Skip to content

Commit

Permalink
Merge pull request apache#23795: Revert 23234: issue apache#23794
Browse files Browse the repository at this point in the history
  • Loading branch information
reuvenlax authored Oct 22, 2022
1 parent 37763c9 commit d38f577
Show file tree
Hide file tree
Showing 2 changed files with 71 additions and 17 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.options;

import com.fasterxml.jackson.annotation.JsonIgnore;
import java.util.concurrent.ScheduledExecutorService;
import org.apache.beam.sdk.util.UnboundedScheduledExecutorService;

/**
* Options for configuring the {@link ScheduledExecutorService} used throughout the Java runtime.
*/
public interface ExecutorOptions extends PipelineOptions {

/**
* The {@link ScheduledExecutorService} instance to use to create threads, can be overridden to
* specify a {@link ScheduledExecutorService} that is compatible with the user's environment. If
* unset, the default is to create an {@link UnboundedScheduledExecutorService}.
*/
@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 an UnboundedScheduledExecutorService.")
@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> {
@Override
public ScheduledExecutorService create(PipelineOptions options) {
/* 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.
*/
return new UnboundedScheduledExecutorService();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,10 @@
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.sdk.util.UnboundedScheduledExecutorService;
import org.checkerframework.checker.nullness.qual.Nullable;

/** Options used to configure Google Cloud Storage. */
Expand All @@ -48,20 +48,22 @@ 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.
* ExecutorService that is compatible with the user's environment. If unset, the default is to use
* {@link ExecutorOptions#getScheduledExecutorService()}.
*
* @deprecated use {@link ExecutorOptions#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)
@Hidden
@Deprecated
ExecutorService getExecutorService();

/**
* @deprecated use {@link ExecutorOptions#setScheduledExecutorService} instead. If set, it may
* result in multiple ExecutorServices, and therefore thread pools, in the runtime.
*/
@Deprecated
void setExecutorService(ExecutorService value);

/** GCS endpoint to use. If unspecified, uses the default endpoint. */
Expand Down Expand Up @@ -132,14 +134,7 @@ public interface GcsOptions extends ApplicationNameOptions, GcpOptions, Pipeline
class ExecutorServiceFactory implements DefaultValueFactory<ExecutorService> {
@Override
public ExecutorService create(PipelineOptions options) {
/* 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.
*/
return new UnboundedScheduledExecutorService();
return options.as(ExecutorOptions.class).getScheduledExecutorService();
}
}

Expand Down

0 comments on commit d38f577

Please sign in to comment.