Skip to content
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

[6571] Windmill service flag #7693

Merged
merged 16 commits into from
Feb 6, 2019
Merged
Show file tree
Hide file tree
Changes from 12 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@
import org.apache.beam.sdk.Pipeline.PipelineVisitor;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.beam.sdk.runners.AppliedPTransform;
Expand Down Expand Up @@ -318,6 +319,32 @@ public Job translate(List<DataflowPackage> packages) {

WorkerPool workerPool = new WorkerPool();

// If streaming engine is enabled set the proper experiments so that it is enabled on the
// back end as well. If streaming engine is not enabled make sure the experiments are also
// not enabled.
if (options.isEnableStreamingEngine()) {
List<String> experiments = options.getExperiments();
if (experiments == null) {
experiments = new ArrayList<String>();
}
if (!experiments.contains(GcpOptions.STREAMING_ENGINE_EXPERIMENT)) {
experiments.add(GcpOptions.STREAMING_ENGINE_EXPERIMENT);
}
if (!experiments.contains(GcpOptions.WINDMILL_SERVICE_EXPERIMENT)) {
experiments.add(GcpOptions.WINDMILL_SERVICE_EXPERIMENT);
}
options.setExperiments(experiments);
} else {
List<String> experiments = options.getExperiments();
if (experiments != null) {
if (experiments.contains(GcpOptions.STREAMING_ENGINE_EXPERIMENT)
|| experiments.contains(GcpOptions.WINDMILL_SERVICE_EXPERIMENT)) {
throw new IllegalArgumentException(
"Streaming engine both disabled and enabled: enableStreamingEngine is set to false, but enable_windmill_service and/or enable_streaming_engine are present. It is recommended you only set enableStreamingEngine.");
}
}
}
dustin12 marked this conversation as resolved.
Show resolved Hide resolved

if (options.isStreaming()) {
job.setType("JOB_TYPE_STREAMING");
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -533,7 +533,7 @@ public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions(
this.workUnitClient = workUnitClient;
this.options = options;
this.sdkHarnessRegistry = sdkHarnessRegistry;
this.windmillServiceEnabled = StreamingDataflowWorkerOptions.streamingEngineEnabled(options);
this.windmillServiceEnabled = options.isEnableStreamingEngine();
this.memoryMonitor = MemoryMonitor.fromOptions(options);
this.statusPages = WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor);
if (windmillServiceEnabled) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,18 +94,6 @@ public interface StreamingDataflowWorkerOptions extends DataflowWorkerHarnessOpt

void setWindmillServiceStreamingRpcBatchLimit(int value);

/** Experiment to turn on the Streaming Engine experiment. */
public static final String STREAMING_ENGINE_EXPERIMENT = "enable_streaming_engine";

/** @deprecated Use STREAMING_ENGINE_EXPERIMENT instead. */
@Deprecated public static final String WINDMILL_SERVICE_EXPERIMENT = "enable_windmill_service";

/** Returns true if the job is running with streaming engine enabled. */
public static boolean streamingEngineEnabled(StreamingDataflowWorkerOptions options) {
return hasExperiment(options, STREAMING_ENGINE_EXPERIMENT)
|| hasExperiment(options, WINDMILL_SERVICE_EXPERIMENT);
}

/**
* Factory for creating local Windmill address. Reads from system propery 'windmill.hostport' for
* backwards compatibility.
Expand Down Expand Up @@ -180,7 +168,7 @@ public WindmillServerStub create(PipelineOptions options) {
StreamingDataflowWorkerOptions streamingOptions =
options.as(StreamingDataflowWorkerOptions.class);
if (streamingOptions.getWindmillServiceEndpoint() != null
|| streamingEngineEnabled(streamingOptions)
|| streamingOptions.isEnableStreamingEngine()
|| streamingOptions.getLocalWindmillHostport().startsWith("grpc:")) {
try {
return new GrpcWindmillServer(streamingOptions);
Expand All @@ -200,7 +188,7 @@ public static class WindmillServiceStreamingRpcBatchLimitFactory
public Integer create(PipelineOptions options) {
StreamingDataflowWorkerOptions streamingOptions =
options.as(StreamingDataflowWorkerOptions.class);
if (streamingEngineEnabled(streamingOptions)
if (streamingOptions.isEnableStreamingEngine()
&& hasExperiment(streamingOptions, "windmill_service_streaming_rpc_batching")) {
return Integer.MAX_VALUE;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequestExtension;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.sdk.util.BackOffUtils;
Expand Down Expand Up @@ -175,14 +176,14 @@ private GrpcWindmillServer(String name, boolean enableStreamingEngine) {
if (experiments == null) {
experiments = new ArrayList<>();
}
experiments.add(StreamingDataflowWorkerOptions.STREAMING_ENGINE_EXPERIMENT);
experiments.add(GcpOptions.STREAMING_ENGINE_EXPERIMENT);
options.setExperiments(experiments);
}
this.stubList.add(CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel(name)));
}

private boolean streamingEngineEnabled() {
return StreamingDataflowWorkerOptions.streamingEngineEnabled(this.options);
return options.isEnableStreamingEngine();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
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.ExperimentalOptions;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.util.BackOffAdapter;
import org.apache.beam.sdk.util.FluentBackoff;
Expand Down Expand Up @@ -126,6 +127,19 @@ public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions {

void setGcpCredential(Credentials value);

/** Experiment to turn on the Streaming Engine experiment. */
String STREAMING_ENGINE_EXPERIMENT = "enable_streaming_engine";
dustin12 marked this conversation as resolved.
Show resolved Hide resolved

/** @deprecated Use STREAMING_ENGINE_EXPERIMENT instead. */
@Deprecated String WINDMILL_SERVICE_EXPERIMENT = "enable_windmill_service";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

private static final

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same


@Description(
"If true will use Streaming Engine. Defaults to false unless the experiments enable_streaming_engine or enable_windmill_service are set.")
@Default.InstanceFactory(EnableStreamingEngineFactory.class)
boolean isEnableStreamingEngine();

void setEnableStreamingEngine(boolean value);

/**
* Attempts to infer the default project based upon the environment this application is executing
* within. Currently this only supports getting the default project from gcloud.
Expand Down Expand Up @@ -216,6 +230,15 @@ public Credentials create(PipelineOptions options) {
}
}

/** EneableStreamingEngine defaults to false unless one of the two experiments is set. */
class EnableStreamingEngineFactory implements DefaultValueFactory<Boolean> {
@Override
public Boolean create(PipelineOptions options) {
return ExperimentalOptions.hasExperiment(options, STREAMING_ENGINE_EXPERIMENT)
|| ExperimentalOptions.hasExperiment(options, WINDMILL_SERVICE_EXPERIMENT);
}
}

/**
* A GCS path for storing temporary files in GCP.
*
Expand Down
3 changes: 3 additions & 0 deletions sdks/python/apache_beam/options/pipeline_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,9 @@ def _add_argparse_args(cls, parser):
'Experimental. '
'See https://cloud.google.com/dataflow/pipelines/'
'updating-a-pipeline')
parser.add_argument('--enable_streaming_engine',
dustin12 marked this conversation as resolved.
Show resolved Hide resolved
default=False,
help='Enable Windmill Service for this Dataflow job. ')
parser.add_argument('--gcp_kms_key',
default=None,
help='Set a Google Cloud KMS key name to be used in '
Expand Down
10 changes: 10 additions & 0 deletions sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
from apache_beam.internal import pickler
from apache_beam.internal.gcp import json_value
from apache_beam.options.pipeline_options import DebugOptions
from apache_beam.options.pipeline_options import GoogleCloudOptions
from apache_beam.options.pipeline_options import SetupOptions
from apache_beam.options.pipeline_options import StandardOptions
from apache_beam.options.pipeline_options import TestOptions
Expand Down Expand Up @@ -361,6 +362,15 @@ def run_pipeline(self, pipeline, options):
experiments = list(set(experiments + debug_options.experiments))
debug_options.experiments = experiments

# Elevate "enable_streaming_engine" to pipeline option, but using the
# existing experiment.
google_cloud_options = options.view_as(GoogleCloudOptions)
if google_cloud_options.enable_streaming_engine:
experiments = ["enable_windmill_service", "enable_streaming_engine"]
if debug_options.experiments is not None:
experiments = list(set(experiments + debug_options.experiments))
debug_options.experiments = experiments
dustin12 marked this conversation as resolved.
Show resolved Hide resolved

self.job = apiclient.Job(options, self.proto_pipeline)

# Dataflow runner requires a KV type for GBK inputs, hence we enforce that
Expand Down