lukecwik commented on code in PR #23234:
URL: https://github.com/apache/beam/pull/23234#discussion_r983855738


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExecutorOptions.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.beam.sdk.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public interface ExecutorOptions extends PipelineOptions{
+
+  /**
+   * The ScheduledExecutorService instance to use to create threads, can be 
overridden to specify a
+   * ScheduledExecutorService that is compatible with the user's environment. 
If unset, the default
+   * is to create an ScheduledExecutorService with a core number of threads 
equal to Math.max(4,
+   * Runtime.getRuntime().availableProcessors())

Review Comment:
   ```suggestion
      * is to create an ScheduledExecutorService with a core number of threads 
equal to {@code Math.max(4,
      * Runtime.getRuntime().availableProcessors())}.
   ```



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExecutorOptions.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.beam.sdk.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public interface ExecutorOptions extends PipelineOptions{
+
+  /**
+   * The ScheduledExecutorService instance to use to create threads, can be 
overridden to specify a
+   * ScheduledExecutorService that is compatible with the user's environment. 
If unset, the default
+   * is to create an ScheduledExecutorService with a core number of threads 
equal to Math.max(4,
+   * Runtime.getRuntime().availableProcessors())
+   */
+  @JsonIgnore
+  @Description(
+      "The ScheduledExecutorService instance to use to create threads, can be 
overridden to specify "
+          + "a ScheduledExecutorService that is compatible with the user's 
environment. If unset, "
+          + "the default is to create a ScheduledExecutorService with a core 
number of threads "
+          + "equal to Math.max(4, Runtime.getRuntime().availableProcessors())")
+  @Default.InstanceFactory(ScheduledExecutorServiceFactory.class)
+  @Hidden
+  ScheduledExecutorService getScheduledExecutorService();
+
+  void setScheduledExecutorService(ScheduledExecutorService value);
+
+
+  /**
+   * Returns the default {@link ScheduledExecutorService} to use within the 
Apache Beam SDK.
+   */
+  class ScheduledExecutorServiceFactory implements 
DefaultValueFactory<ScheduledExecutorService> {
+    @SuppressWarnings("deprecation") // IS_APP_ENGINE is deprecated for 
internal use only.
+    @Override
+    public ScheduledExecutorService create(PipelineOptions options) {
+      ThreadFactoryBuilder threadFactoryBuilder = new ThreadFactoryBuilder();
+      
threadFactoryBuilder.setThreadFactory(MoreExecutors.platformThreadFactory());
+      threadFactoryBuilder.setDaemon(true);
+      /* The SDK requires an unbounded thread pool because a step may create X 
writers
+       * each requiring their own thread to perform the writes otherwise a 
writer may
+       * block causing deadlock for the step because the writers buffer is 
full.
+       * Also, the MapTaskExecutor launches the steps in reverse order and 
completes
+       * them in forward order thus requiring enough threads so that each 
step's writers
+       * can be active.
+       */

Review Comment:
   nit: combine these two comment blocks into one?



##########
sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java:
##########
@@ -53,21 +49,26 @@ public interface GcsOptions extends ApplicationNameOptions, 
GcpOptions, Pipeline
   /**
    * The ExecutorService instance to use to create threads, can be overridden 
to specify an
    * ExecutorService that is compatible with the user's environment. If unset, 
the default is to
-   * create an ExecutorService with an unbounded number of threads; this is 
compatible with Google
-   * AppEngine.
+   * create a ScheduledExecutorService with a core number of threads equal to
+   * Math.max(4, Runtime.getRuntime().availableProcessors()). Deprecated in 
favor of
+   * getScheduledExecutorService instead
    */
   @JsonIgnore
   @Description(
       "The ExecutorService instance to use to create multiple threads. Can be 
overridden "
           + "to specify an ExecutorService that is compatible with the user's 
environment. If unset, "
-          + "the default is to create an ExecutorService with an unbounded 
number of threads; this "
-          + "is compatible with Google AppEngine.")
-  @Default.InstanceFactory(ExecutorServiceFactory.class)
+          + "the default is to create a ScheduledExecutorService with a core 
number of threads"
+          + "equal to Math.max(4, 
Runtime.getRuntime().availableProcessors()).")
+  @Default.InstanceFactory(ScheduledExecutorServiceFactory.class)

Review Comment:
   You don't want this default since this will create two different scheduled 
executor services. Instead update the original ExecutorServiceFactory class to 
do:
   options.as(ExecutorOptions.class).getScheduledExecutorService();



##########
sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java:
##########
@@ -32,15 +29,14 @@
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.ExecutorOptions;
 import org.apache.beam.sdk.options.Hidden;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.InstanceBuilder;
-import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
-import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Options used to configure Google Cloud Storage. */
-public interface GcsOptions extends ApplicationNameOptions, GcpOptions, 
PipelineOptions {
+public interface GcsOptions extends ApplicationNameOptions, GcpOptions, 
ExecutorOptions, PipelineOptions {

Review Comment:
   You don't need to extend ExecutorOptions



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExecutorOptions.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.beam.sdk.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public interface ExecutorOptions extends PipelineOptions{

Review Comment:
   ```suggestion
   public interface ExecutorOptions extends PipelineOptions {
   ```



##########
sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java:
##########
@@ -53,21 +49,26 @@ public interface GcsOptions extends ApplicationNameOptions, 
GcpOptions, Pipeline
   /**
    * The ExecutorService instance to use to create threads, can be overridden 
to specify an
    * ExecutorService that is compatible with the user's environment. If unset, 
the default is to
-   * create an ExecutorService with an unbounded number of threads; this is 
compatible with Google
-   * AppEngine.
+   * create a ScheduledExecutorService with a core number of threads equal to

Review Comment:
   Since you marked the methods as deprecated, please add the `@deprecated` 
javadoc comment telling folks to use ExecutorOptions#getScheduledExecutorService
   
   Ditto for set.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExecutorOptions.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.beam.sdk.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public interface ExecutorOptions extends PipelineOptions{

Review Comment:
   class comment



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExecutorOptions.java:
##########
@@ -0,0 +1,56 @@
+package org.apache.beam.sdk.options;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public interface ExecutorOptions extends PipelineOptions{
+
+  /**
+   * The ScheduledExecutorService instance to use to create threads, can be 
overridden to specify a
+   * ScheduledExecutorService that is compatible with the user's environment. 
If unset, the default
+   * is to create an ScheduledExecutorService with a core number of threads 
equal to Math.max(4,
+   * Runtime.getRuntime().availableProcessors())
+   */
+  @JsonIgnore
+  @Description(
+      "The ScheduledExecutorService instance to use to create threads, can be 
overridden to specify "
+          + "a ScheduledExecutorService that is compatible with the user's 
environment. If unset, "
+          + "the default is to create a ScheduledExecutorService with a core 
number of threads "
+          + "equal to Math.max(4, Runtime.getRuntime().availableProcessors())")
+  @Default.InstanceFactory(ScheduledExecutorServiceFactory.class)
+  @Hidden
+  ScheduledExecutorService getScheduledExecutorService();
+
+  void setScheduledExecutorService(ScheduledExecutorService value);
+
+
+  /**
+   * Returns the default {@link ScheduledExecutorService} to use within the 
Apache Beam SDK.
+   */
+  class ScheduledExecutorServiceFactory implements 
DefaultValueFactory<ScheduledExecutorService> {
+    @SuppressWarnings("deprecation") // IS_APP_ENGINE is deprecated for 
internal use only.
+    @Override
+    public ScheduledExecutorService create(PipelineOptions options) {
+      ThreadFactoryBuilder threadFactoryBuilder = new ThreadFactoryBuilder();
+      
threadFactoryBuilder.setThreadFactory(MoreExecutors.platformThreadFactory());
+      threadFactoryBuilder.setDaemon(true);
+      /* The SDK requires an unbounded thread pool because a step may create X 
writers
+       * each requiring their own thread to perform the writes otherwise a 
writer may
+       * block causing deadlock for the step because the writers buffer is 
full.
+       * Also, the MapTaskExecutor launches the steps in reverse order and 
completes
+       * them in forward order thus requiring enough threads so that each 
step's writers
+       * can be active.
+       */
+
+      /*The minimum of max(4, processors) was chosen as a default working 
configuration found in

Review Comment:
   nit: formatting



##########
sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java:
##########
@@ -53,21 +49,26 @@ public interface GcsOptions extends ApplicationNameOptions, 
GcpOptions, Pipeline
   /**
    * The ExecutorService instance to use to create threads, can be overridden 
to specify an
    * ExecutorService that is compatible with the user's environment. If unset, 
the default is to
-   * create an ExecutorService with an unbounded number of threads; this is 
compatible with Google
-   * AppEngine.
+   * create a ScheduledExecutorService with a core number of threads equal to
+   * Math.max(4, Runtime.getRuntime().availableProcessors()). Deprecated in 
favor of
+   * getScheduledExecutorService instead

Review Comment:
   You can remove this completely.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to