-
Notifications
You must be signed in to change notification settings - Fork 4.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Update BQIO to a single scheduled executor service reduce threads #23234
Changes from all commits
a7fb056
1e062e2
f2c95a9
5b81a10
97fe2c2
e606856
62b4d5d
1de7feb
45fa683
d68b2a1
863f547
b3ce303
de9acee
100cb78
3446728
e8b1301
8d9bb75
a59e30a
eb88210
cc869f7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 { | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. comment and |
||
/** | ||
* 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. | ||
*/ | ||
johnjcasey marked this conversation as resolved.
Show resolved
Hide resolved
|
||
return new UnboundedScheduledExecutorService(); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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. */ | ||
|
@@ -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(); | ||
|
||
johnjcasey marked this conversation as resolved.
Show resolved
Hide resolved
|
||
/** | ||
* @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. */ | ||
|
@@ -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. | ||
*/ | ||
Comment on lines
-135
to
-141
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Move this comment into ScheduledExecutorServiceFactory just above |
||
return new UnboundedScheduledExecutorService(); | ||
return options.as(ExecutorOptions.class).getScheduledExecutorService(); | ||
} | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
class comment?