Merge pull request #10402 from Ardagan/Blogpost2.17.0
Release 2.17.0 Blogpost note
diff --git a/.gitignore b/.gitignore
index 8f7e531..5732b9c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -90,3 +90,6 @@
# JetBrains Education files
!**/study_project.xml
**/.coursecreator/**/*
+
+.pytest_cache
+.pytest_cache/**/*
\ No newline at end of file
diff --git a/.test-infra/jenkins/CommonTestProperties.groovy b/.test-infra/jenkins/CommonTestProperties.groovy
index 0d750ee..7bf585e 100644
--- a/.test-infra/jenkins/CommonTestProperties.groovy
+++ b/.test-infra/jenkins/CommonTestProperties.groovy
@@ -21,6 +21,7 @@
class CommonTestProperties {
enum SDK {
PYTHON,
+ PYTHON_37,
JAVA
}
@@ -42,6 +43,11 @@
DATAFLOW: "TestDataflowRunner",
DIRECT: "DirectRunner",
PORTABLE: "PortableRunner"
+ ],
+ PYTHON_37: [
+ DATAFLOW: "TestDataflowRunner",
+ DIRECT: "DirectRunner",
+ PORTABLE: "PortableRunner"
]
]
@@ -51,8 +57,7 @@
this.option = option
}
-
- String getDepenedencyBySDK(SDK sdk) {
+ String getDependencyBySDK(SDK sdk) {
RUNNER_DEPENDENCY_MAP.get(sdk.toString()).get(this.toString())
}
@@ -62,4 +67,4 @@
PR,
POST_COMMIT
}
-}
\ No newline at end of file
+}
diff --git a/.test-infra/jenkins/LoadTestsBuilder.groovy b/.test-infra/jenkins/LoadTestsBuilder.groovy
index c259033..d7ed2d5 100644
--- a/.test-infra/jenkins/LoadTestsBuilder.groovy
+++ b/.test-infra/jenkins/LoadTestsBuilder.groovy
@@ -42,11 +42,8 @@
shell("echo *** ${title} ***")
gradle {
rootBuildScriptDir(commonJobProperties.checkoutDir)
- tasks(getGradleTaskName(sdk))
+ setGradleTask(delegate, runner, sdk, options, mainClass)
commonJobProperties.setGradleSwitches(delegate)
- switches("-PloadTest.mainClass=\"${mainClass}\"")
- switches("-Prunner=${runner.getDepenedencyBySDK(sdk)}")
- switches("-PloadTest.args=\"${parseOptions(options)}\"")
}
}
}
@@ -59,10 +56,21 @@
}
}
+ private static void setGradleTask(context, Runner runner, SDK sdk, Map<String, ?> options, String mainClass) {
+ context.tasks(getGradleTaskName(sdk))
+ context.switches("-PloadTest.mainClass=\"${mainClass}\"")
+ context.switches("-Prunner=${runner.getDependencyBySDK(sdk)}")
+ context.switches("-PloadTest.args=\"${parseOptions(options)}\"")
+
+ if (sdk == SDK.PYTHON_37) {
+ context.switches("-PpythonVersion=3.7")
+ }
+ }
+
private static String getGradleTaskName(SDK sdk) {
if (sdk == SDK.JAVA) {
return ':sdks:java:testing:load-tests:run'
- } else if (sdk == SDK.PYTHON) {
+ } else if (sdk == SDK.PYTHON || sdk == SDK.PYTHON_37) {
return ':sdks:python:apache_beam:testing:load_tests:run'
} else {
throw new RuntimeException("No task name defined for SDK: $SDK")
diff --git a/.test-infra/jenkins/NexmarkBuilder.groovy b/.test-infra/jenkins/NexmarkBuilder.groovy
index 32a4e13..9cdef21 100644
--- a/.test-infra/jenkins/NexmarkBuilder.groovy
+++ b/.test-infra/jenkins/NexmarkBuilder.groovy
@@ -81,7 +81,7 @@
rootBuildScriptDir(commonJobProperties.checkoutDir)
tasks(':sdks:java:testing:nexmark:run')
commonJobProperties.setGradleSwitches(delegate)
- switches("-Pnexmark.runner=${runner.getDepenedencyBySDK(sdk)}")
+ switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}")
switches("-Pnexmark.args=\"${parseOptions(options)}\"")
}
}
diff --git a/.test-infra/jenkins/README.md b/.test-infra/jenkins/README.md
index 0b7b7c9..471d914 100644
--- a/.test-infra/jenkins/README.md
+++ b/.test-infra/jenkins/README.md
@@ -129,6 +129,7 @@
| beam_LoadTests_Java_ParDo_Dataflow_Batch | [cron](https://builds.apache.org/job/beam_LoadTests_Java_ParDo_Dataflow_Batch/), [phrase](https://builds.apache.org/job/beam_LoadTests_Java_ParDo_Dataflow_Batch_PR/) | `Run Load Tests Java ParDo Dataflow Batch` | [](https://builds.apache.org/job/beam_LoadTests_Java_ParDo_Dataflow_Batch/) |
| beam_LoadTests_Java_ParDo_Dataflow_Streaming | [cron](https://builds.apache.org/job/beam_LoadTests_Java_ParDo_Dataflow_Streaming/), [phrase](https://builds.apache.org/job/beam_LoadTests_Java_ParDo_Dataflow_Streaming_PR/) | `Run Load Tests Java ParDo Dataflow Streaming` | [](https://builds.apache.org/job/beam_LoadTests_Java_ParDo_Dataflow_Streaming/) |
| beam_LoadTests_Python_ParDo_Dataflow_Batch | [cron](https://builds.apache.org/job/beam_LoadTests_Python_ParDo_Dataflow_Batch/), [phrase](https://builds.apache.org/job/beam_LoadTests_Python_ParDo_Dataflow_Batch_PR/) | `Run Python Load Tests ParDo Dataflow Batch` | [](https://builds.apache.org/job/beam_LoadTests_Python_ParDo_Dataflow_Batch/) |
+| beam_LoadTests_Python_37_ParDo_Dataflow_Batch | [cron](https://builds.apache.org/job/beam_LoadTests_Python_37_ParDo_Dataflow_Batch/), [phrase](https://builds.apache.org/job/beam_LoadTests_Python_37_ParDo_Dataflow_Batch_PR/) | `Run Python 3.7 Load Tests ParDo Dataflow Batch` | [](https://builds.apache.org/job/beam_LoadTests_Python_37_ParDo_Dataflow_Batch/) |
| beam_LoadTests_Python_ParDo_Flink_Batch | [cron](https://builds.apache.org/job/beam_LoadTests_Python_ParDo_Flink_Batch/), [phrase](https://builds.apache.org/job/beam_LoadTests_Python_ParDo_Flink_Batch_PR/) | `Run Python Load Tests ParDo Flink Batch` | [](https://builds.apache.org/job/beam_LoadTests_Python_ParDo_Flink_Batch/) |
### Inventory Jobs
diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy
new file mode 100644
index 0000000..71c5d6f
--- /dev/null
+++ b/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+import CommonJobProperties as commonJobProperties
+import LoadTestsBuilder as loadTestsBuilder
+import PhraseTriggeringPostCommitBuilder
+
+def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC'))
+
+def loadTestConfigurations = { datasetName -> [
+ [
+ title : 'ParDo Python Load test: 2GB 100 byte records 10 times',
+ test : 'apache_beam.testing.load_tests.pardo_test:ParDoTest.testParDo',
+ runner : CommonTestProperties.Runner.DATAFLOW,
+ pipelineOptions: [
+ job_name : 'load-tests-python37-dataflow-batch-pardo-1-' + now,
+ project : 'apache-beam-testing',
+ temp_location : 'gs://temp-storage-for-perf-tests/loadtests',
+ publish_to_big_query : true,
+ metrics_dataset : datasetName,
+ metrics_table : 'python37_dataflow_batch_pardo_1',
+ input_options : '\'{' +
+ '"num_records": 20000000,' +
+ '"key_size": 10,' +
+ '"value_size": 90}\'',
+ iterations : 10,
+ number_of_counter_operations: 0,
+ number_of_counters : 0,
+ num_workers : 5,
+ autoscaling_algorithm: 'NONE',
+ ]
+ ],
+]}
+
+def batchLoadTestJob = { scope, triggeringContext ->
+ scope.description('Runs Python 3.7 ParDo load tests on Dataflow runner in batch mode')
+ commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 120)
+
+ def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext)
+ for (testConfiguration in loadTestConfigurations(datasetName)) {
+ loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.PYTHON_37, testConfiguration.pipelineOptions, testConfiguration.test)
+ }
+}
+
+PhraseTriggeringPostCommitBuilder.postCommitJob(
+ 'beam_LoadTests_Python_37_ParDo_Dataflow_Batch',
+ 'Run Python 3.7 Load Tests ParDo Dataflow Batch',
+ 'Load Tests Python 3.7 ParDo Dataflow Batch suite',
+ this
+) {
+ batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR)
+}
+
+CronJobBuilder.cronJob('beam_LoadTests_Python_37_ParDo_Dataflow_Batch', 'H 13 * * *', this) {
+ batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT)
+}
diff --git a/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy b/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy
index 80b2aa3..b526bce 100644
--- a/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy
+++ b/.test-infra/jenkins/job_PostCommit_PortableJar_Flink.groovy
@@ -25,7 +25,7 @@
description('Tests creation and execution of portable pipeline Jars on the Flink runner.')
// Set common parameters.
- commonJobProperties.setTopLevelMainJobProperties(delegate)
+ commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 120)
// Gradle goals for this job.
steps {
diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
index e16d9e5..0d28c6b 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -384,7 +384,7 @@
def postgres_version = "42.2.2"
def powermock_version = "2.0.2"
def proto_google_common_protos_version = "1.17.0"
- def protobuf_version = "3.6.0"
+ def protobuf_version = "3.11.1"
def quickcheck_version = "0.8"
def spark_version = "2.4.4"
def spark_structured_streaming_version = "2.4.0"
@@ -492,6 +492,7 @@
jackson_module_scala : "com.fasterxml.jackson.module:jackson-module-scala_2.11:$jackson_version",
jaxb_api : "javax.xml.bind:jaxb-api:$jaxb_api_version",
joda_time : "joda-time:joda-time:2.10.3",
+ jsonassert : "org.skyscreamer:jsonassert:1.5.0",
jsr305 : "com.google.code.findbugs:jsr305:3.0.2",
junit : "junit:junit:4.13-beta-3",
kafka : "org.apache.kafka:kafka_2.11:$kafka_version",
@@ -1832,6 +1833,9 @@
}
}
project.clean.dependsOn project.cleanPython
+ // Force this subproject's clean to run before the main :clean, to avoid
+ // racing on deletes.
+ project.rootProject.clean.dependsOn project.clean
// Return a joined String from a Map that contains all commandline args of
// IT test.
diff --git a/model/fn-execution/src/main/proto/beam_fn_api.proto b/model/fn-execution/src/main/proto/beam_fn_api.proto
index c868bab..3824e90 100644
--- a/model/fn-execution/src/main/proto/beam_fn_api.proto
+++ b/model/fn-execution/src/main/proto/beam_fn_api.proto
@@ -207,18 +207,13 @@
// Either an absolute timestamp or a relative timestamp can represent a
// scheduled execution time.
message DelayedBundleApplication {
- // Recommended time at which the application should be scheduled to execute
- // by the runner. Times in the past may be scheduled to execute immediately.
- // TODO(BEAM-8536): Migrate usage of absolute time to requested_time_delay.
- google.protobuf.Timestamp requested_execution_time = 1;
-
// (Required) The application that should be scheduled.
- BundleApplication application = 2;
+ BundleApplication application = 1;
// Recommended time delay at which the application should be scheduled to
// execute by the runner. Time delay that equals 0 may be scheduled to execute
// immediately. The unit of time delay should be microsecond.
- google.protobuf.Duration requested_time_delay = 3;
+ google.protobuf.Duration requested_time_delay = 2;
}
// A request to process a given bundle.
@@ -431,7 +426,7 @@
//
// Set to 0 to "checkpoint" as soon as possible (keeping as little work as
// possible and returning the remainder).
- float fraction_of_remainder = 1;
+ double fraction_of_remainder = 1;
// A set of allowed element indices where the SDK may split. When this is
// empty, there are no constraints on where to split.
@@ -486,12 +481,12 @@
// The last element of the input channel that should be entirely considered
// part of the primary, identified by its absolute index in the (ordered)
// channel.
- int32 last_primary_element = 2;
+ int64 last_primary_element = 2;
// The first element of the input channel that should be entirely considered
// part of the residual, identified by its absolute index in the (ordered)
// channel.
- int32 first_residual_element = 3;
+ int64 first_residual_element = 3;
}
// Partitions of input data channels into primary and residual elements,
diff --git a/model/fn-execution/src/main/proto/beam_provision_api.proto b/model/fn-execution/src/main/proto/beam_provision_api.proto
index 442e626..e6b4ed8 100644
--- a/model/fn-execution/src/main/proto/beam_provision_api.proto
+++ b/model/fn-execution/src/main/proto/beam_provision_api.proto
@@ -29,6 +29,7 @@
option java_package = "org.apache.beam.model.fnexecution.v1";
option java_outer_classname = "ProvisionApi";
+import "endpoints.proto";
import "google/protobuf/struct.proto";
// A service to provide runtime provisioning information to the SDK harness
@@ -71,6 +72,12 @@
// (required) The artifact retrieval token produced by
// ArtifactStagingService.CommitManifestResponse.
string retrieval_token = 6;
+
+ // (optional) The endpoint that the runner is hosting for the SDK to submit
+ // status reports to during pipeline execution. This field will only be
+ // populated if the runner supports SDK status reports. For more details see
+ // https://s.apache.org/beam-fn-api-harness-status
+ org.apache.beam.model.pipeline.v1.ApiServiceDescriptor status_endpoint = 7;
}
// Resources specify limits for local resources, such memory and cpu. It
diff --git a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
index 9de15ac..c9a4288 100644
--- a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
+++ b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
@@ -263,6 +263,44 @@
---
+# ParamWindowedValueCoder with constant value of:
+# timestamp: Instant.ofEpochMilli(1000)
+# windows: [IntervalWindow(10, 20)]
+# pane info: PaneInfo(false, true, PaneInfo.Timing.ON_TIME, 30, 40)
+coder:
+ urn: "beam:coder:param_windowed_value:v1"
+ payload: "\x80\x00\x00\x00\x00\x00\x03è\x00\x00\x00\x01\x80\x00\x00\x00\x00\x00\x00\x14\n&\x1E(\x00"
+ components: [{urn: "beam:coder:varint:v1"},
+ {urn: "beam:coder:interval_window:v1"}]
+examples:
+ "\u0002": {
+ value: 2,
+ timestamp: 1000,
+ pane: {is_first: False, is_last: True, timing: ON_TIME, index: 30, on_time_index: 40},
+ windows: [{end: 20, span: 10}]
+ }
+
+---
+
+# ParamWindowedValueCoder with constant value of:
+# timestamp: BoundedWindow.TIMESTAMP_MIN_VALUE
+# windows: [GlobalWindow.INSTANCE]
+# pane info: PaneInfo.NO_FIRING
+coder:
+ urn: "beam:coder:param_windowed_value:v1"
+ payload: "\x7Fß;dZ\x1C¬\t\x00\x00\x00\x01\x0F\x00"
+ components: [{urn: "beam:coder:varint:v1"},
+ {urn: "beam:coder:global_window:v1"}]
+examples:
+ "\u0002": {
+ value: 2,
+ timestamp: -9223372036854775,
+ pane: {is_first: True, is_last: True, timing: UNKNOWN, index: 0, on_time_index: 0},
+ windows: ["global"]
+ }
+
+---
+
coder:
urn: "beam:coder:double:v1"
examples:
diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index 6143930..df5d59d 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -164,6 +164,11 @@
// there is none, or it is not relevant (such as use by the Fn API)
// then it may be omitted.
DisplayData display_data = 6;
+
+ // (Optional) Environment where the current PTransform should be executed in.
+ // Runner that executes the pipeline may choose to override this if needed. If
+ // not specified, environment will be decided by the runner.
+ string environment_id = 7;
}
message StandardPTransforms {
@@ -364,8 +369,8 @@
// The payload for the primitive ParDo transform.
message ParDoPayload {
- // (Required) The SdkFunctionSpec of the DoFn.
- SdkFunctionSpec do_fn = 1;
+ // (Required) The FunctionSpec of the DoFn.
+ FunctionSpec do_fn = 1;
// (Required) Additional pieces of context the DoFn may require that
// are not otherwise represented in the payload.
@@ -439,7 +444,7 @@
message CombiningStateSpec {
string accumulator_coder_id = 1;
- SdkFunctionSpec combine_fn = 2;
+ FunctionSpec combine_fn = 2;
}
message MapStateSpec {
@@ -467,8 +472,8 @@
// The payload for the primitive Read transform.
message ReadPayload {
- // (Required) The SdkFunctionSpec of the source for this Read.
- SdkFunctionSpec source = 1;
+ // (Required) The FunctionSpec of the source for this Read.
+ FunctionSpec source = 1;
// (Required) Whether the source is bounded or unbounded
IsBounded.Enum is_bounded = 2;
@@ -479,15 +484,15 @@
// The payload for the WindowInto transform.
message WindowIntoPayload {
- // (Required) The SdkFunctionSpec of the WindowFn.
- SdkFunctionSpec window_fn = 1;
+ // (Required) The FunctionSpec of the WindowFn.
+ FunctionSpec window_fn = 1;
}
// The payload for the special-but-not-primitive Combine transform.
message CombinePayload {
- // (Required) The SdkFunctionSpec of the CombineFn.
- SdkFunctionSpec combine_fn = 1;
+ // (Required) The FunctionSpec of the CombineFn.
+ FunctionSpec combine_fn = 1;
// (Required) A reference to the Coder to use for accumulators of the CombineFn
string accumulator_coder_id = 2;
@@ -562,11 +567,11 @@
// The payload for the special-but-not-primitive WriteFiles transform.
message WriteFilesPayload {
- // (Required) The SdkFunctionSpec of the FileBasedSink.
- SdkFunctionSpec sink = 1;
+ // (Required) The FunctionSpec of the FileBasedSink.
+ FunctionSpec sink = 1;
// (Required) The format function.
- SdkFunctionSpec format_function = 2;
+ FunctionSpec format_function = 2;
bool windowed_writes = 3;
@@ -588,7 +593,7 @@
// (Optional) If this coder is parametric, such as ListCoder(VarIntCoder),
// this is a list of the components. In order for encodings to be identical,
- // the SdkFunctionSpec and all components must be identical, recursively.
+ // the FunctionSpec and all components must be identical, recursively.
repeated string component_coder_ids = 2;
}
@@ -672,6 +677,16 @@
// Components: The element coder and the window coder, in that order
WINDOWED_VALUE = 8 [(beam_urn) = "beam:coder:windowed_value:v1"];
+ // A windowed value coder with parameterized timestamp, windows and pane info.
+ // Encodes an element with only the value of the windowed value.
+ // Decodes the value and assigns the parameterized timestamp, windows and pane info to the
+ // windowed value.
+ // Components: The element coder and the window coder, in that order
+ // The payload of this coder is an encoded windowed value using the
+ // beam:coder:windowed_value:v1 coder parameterized by a beam:coder:bytes:v1
+ // element coder and the window coder that this param_windowed_value coder uses.
+ PARAM_WINDOWED_VALUE = 14 [(beam_urn) = "beam:coder:param_windowed_value:v1"];
+
// Encodes an iterable of elements, some of which may be stored elsewhere.
//
// The encoding for a state-backed iterable is the same as that for
@@ -741,10 +756,10 @@
// TODO: consider inlining field on PCollection
message WindowingStrategy {
- // (Required) The SdkFunctionSpec of the UDF that assigns windows,
+ // (Required) The FunctionSpec of the UDF that assigns windows,
// merges windows, and shifts timestamps before they are
// combined according to the OutputTime.
- SdkFunctionSpec window_fn = 1;
+ FunctionSpec window_fn = 1;
// (Required) Whether or not the window fn is merging.
//
@@ -787,6 +802,11 @@
//
// This knowledge is required for some optimizations
bool assigns_to_one_window = 10;
+
+ // (Optional) Environment where the current window_fn should be applied in.
+ // Runner that executes the pipeline may choose to override this if needed.
+ // If not specified, environment will be decided by the runner.
+ string environment_id = 11;
}
// Whether or not a PCollection's WindowFn is non-merging, merging, or
@@ -1043,20 +1063,35 @@
// URN)
FunctionSpec access_pattern = 1;
- // (Required) The SdkFunctionSpec of the UDF that adapts a particular
+ // (Required) The FunctionSpec of the UDF that adapts a particular
// access_pattern to a user-facing view type.
//
// For example, View.asSingleton() may include a `view_fn` that adapts a
// specially-designed multimap to a single value per window.
- SdkFunctionSpec view_fn = 2;
+ FunctionSpec view_fn = 2;
- // (Required) The SdkFunctionSpec of the UDF that maps a main input window
+ // (Required) The FunctionSpec of the UDF that maps a main input window
// to a side input window.
//
// For example, when the main input is in fixed windows of one hour, this
// can specify that the side input should be accessed according to the day
// in which that hour falls.
- SdkFunctionSpec window_mapping_fn = 3;
+ FunctionSpec window_mapping_fn = 3;
+}
+
+// Settings that decide the coder type of wire coder.
+message WireCoderSetting {
+ // (Required) The URN of the wire coder.
+ // Note that only windowed value coder or parameterized windowed value coder are supported.
+ string urn = 1;
+
+ // (Optional) The data specifying any parameters to the URN. If
+ // the URN is beam:coder:windowed_value:v1, this may be omitted. If the URN is
+ // beam:coder:param_windowed_value:v1, the payload is an encoded windowed
+ // value using the beam:coder:windowed_value:v1 coder parameterized by
+ // a beam:coder:bytes:v1 element coder and the window coder that this
+ // param_windowed_value coder uses.
+ bytes payload = 2;
}
// An environment for executing UDFs. By default, an SDK container URL, but
@@ -1099,18 +1134,6 @@
map<string, string> params = 2; // Arbitrary extra parameters to pass
}
-// A specification of a user defined function.
-//
-message SdkFunctionSpec {
-
- // (Required) A full specification of this function.
- FunctionSpec spec = 1;
-
- // (Required) Reference to an execution environment capable of
- // invoking this function.
- string environment_id = 2;
-}
-
extend google.protobuf.EnumValueOptions {
// An extension to be used for specifying the standard URN of various
// pipeline entities, e.g. transforms, functions, coders etc.
@@ -1258,7 +1281,7 @@
oneof root {
Coder coder = 2;
CombinePayload combine_payload = 3;
- SdkFunctionSpec sdk_function_spec = 4;
+ FunctionSpec function_spec = 4;
ParDoPayload par_do_payload = 6;
PTransform ptransform = 7;
PCollection pcollection = 8;
@@ -1266,7 +1289,6 @@
SideInput side_input = 11;
WindowIntoPayload window_into_payload = 12;
WindowingStrategy windowing_strategy = 13;
- FunctionSpec function_spec = 14;
}
}
@@ -1280,6 +1302,9 @@
// because ExecutableStages use environments directly. This may change in the future.
Environment environment = 1;
+ // set the wire coder of this executable stage
+ WireCoderSetting wire_coder_setting = 9;
+
// (Required) Input PCollection id. This must be present as a value in the inputs of any
// PTransform the ExecutableStagePayload is the payload of.
string input = 2;
diff --git a/model/pipeline/src/main/proto/schema.proto b/model/pipeline/src/main/proto/schema.proto
index 1e62d17..2cf404e 100644
--- a/model/pipeline/src/main/proto/schema.proto
+++ b/model/pipeline/src/main/proto/schema.proto
@@ -87,4 +87,56 @@
string urn = 1;
bytes payload = 2;
FieldType representation = 3;
+ FieldType argument_type = 4;
+ FieldValue argument = 5;
}
+
+message Row {
+ repeated FieldValue values = 1;
+}
+
+message FieldValue {
+ oneof field_value {
+ AtomicTypeValue atomic_value = 1;
+ ArrayTypeValue array_value = 2;
+ IterableTypeValue iterable_value = 3;
+ MapTypeValue map_value = 4;
+ Row row_value = 5;
+ LogicalTypeValue logical_type_value = 6;
+ }
+}
+
+message AtomicTypeValue {
+ oneof value {
+ int32 byte = 1;
+ int32 int16 = 2;
+ int32 int32 = 3;
+ int64 int64 = 4;
+ float float = 5;
+ double double = 6;
+ string string = 7;
+ bool boolean = 8;
+ bytes bytes = 9;
+ }
+}
+
+message ArrayTypeValue {
+ repeated FieldValue element = 1;
+}
+
+message IterableTypeValue {
+ repeated FieldValue element = 1;
+}
+
+message MapTypeValue {
+ repeated MapTypeEntry entries = 1;
+}
+
+message MapTypeEntry {
+ FieldValue key = 1;
+ FieldValue value = 2;
+}
+
+message LogicalTypeValue {
+ FieldValue value = 1;
+}
\ No newline at end of file
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
index 9a56496..6f90f58 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java
@@ -129,8 +129,8 @@
this.keyCoder = ((KvCoder<K, V>) input.getCoder()).getKeyCoder();
this.valueCoder = ((KvCoder<K, V>) input.getCoder()).getValueCoder();
this.stateInternalsFactory = stateBackend.newStateInternalsFactory(keyCoder);
- TimerInternals.TimerDataCoder timerCoder =
- TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder());
+ TimerInternals.TimerDataCoderV2 timerCoder =
+ TimerInternals.TimerDataCoderV2.of(windowingStrategy.getWindowFn().windowCoder());
this.timerInternals = new ApexTimerInternals<>(timerCoder);
}
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
index 9d4b110..79bb6ef 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java
@@ -187,8 +187,8 @@
this.inputCoder = inputCoder;
this.outputCoders = outputCoders;
- TimerInternals.TimerDataCoder timerCoder =
- TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder());
+ TimerInternals.TimerDataCoderV2 timerCoder =
+ TimerInternals.TimerDataCoderV2.of(windowingStrategy.getWindowFn().windowCoder());
this.currentKeyTimerInternals = new ApexTimerInternals<>(timerCoder);
this.doFnSchemaInformation = doFnSchemaInformation;
this.sideInputMapping = sideInputMapping;
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java
index b4028e7..682cbed 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java
@@ -57,7 +57,7 @@
private transient Instant currentOutputWatermark;
private transient Coder<K> keyCoder;
- public ApexTimerInternals(TimerDataCoder timerDataCoder) {
+ public ApexTimerInternals(TimerDataCoderV2 timerDataCoder) {
this.eventTimeTimeTimers = new TimerSet(timerDataCoder);
this.processingTimeTimers = new TimerSet(timerDataCoder);
}
@@ -77,8 +77,14 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- TimerData timerData = TimerData.of(timerId, namespace, target, timeDomain);
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ TimerData timerData =
+ TimerData.of(timerId, timerFamilyId, namespace, target, outputTimestamp, timeDomain);
setTimer(timerData);
}
@@ -196,10 +202,10 @@
protected static class TimerSet implements Serializable {
private final Map<Slice, Set<Slice>> activeTimers = new HashMap<>();
- private final TimerDataCoder timerDataCoder;
+ private final TimerDataCoderV2 timerDataCoder;
private long minTimestamp = Long.MAX_VALUE;
- protected TimerSet(TimerDataCoder timerDataCoder) {
+ protected TimerSet(TimerDataCoderV2 timerDataCoder) {
this.timerDataCoder = timerDataCoder;
}
diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java
index 0bd890a..1d7e3f8 100644
--- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java
+++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java
@@ -29,7 +29,7 @@
import org.apache.beam.runners.apex.translation.operators.ApexTimerInternals.TimerProcessor;
import org.apache.beam.runners.core.StateNamespaces;
import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.TimerInternals.TimerDataCoder;
+import org.apache.beam.runners.core.TimerInternals.TimerDataCoderV2;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.state.TimeDomain;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
@@ -46,7 +46,7 @@
final Map<String, Collection<TimerData>> firedTimers = new HashMap<>();
TimerProcessor<String> timerProcessor = firedTimers::put;
- TimerDataCoder timerDataCoder = TimerDataCoder.of(GlobalWindow.Coder.INSTANCE);
+ TimerDataCoderV2 timerDataCoder = TimerDataCoderV2.of(GlobalWindow.Coder.INSTANCE);
String key1 = "key1";
Instant instant0 = new Instant(0);
Instant instant1 = new Instant(1);
@@ -85,7 +85,7 @@
@Test
public void testDeleteTimer() {
- TimerDataCoder timerDataCoder = TimerDataCoder.of(GlobalWindow.Coder.INSTANCE);
+ TimerDataCoderV2 timerDataCoder = TimerDataCoderV2.of(GlobalWindow.Coder.INSTANCE);
String key1 = "key1";
Instant instant0 = new Instant(0);
Instant instant1 = new Instant(1);
@@ -121,7 +121,7 @@
@Test
public void testSerialization() {
- TimerDataCoder timerDataCoder = TimerDataCoder.of(GlobalWindow.Coder.INSTANCE);
+ TimerDataCoderV2 timerDataCoder = TimerDataCoderV2.of(GlobalWindow.Coder.INSTANCE);
TimerData timerData =
TimerData.of(
"arbitrary-id", StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME);
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
index 2bf5264..6f156dd 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
@@ -125,6 +125,26 @@
};
}
+ static CoderTranslator<WindowedValue.ParamWindowedValueCoder<?>> paramWindowedValue() {
+ return new CoderTranslator<WindowedValue.ParamWindowedValueCoder<?>>() {
+ @Override
+ public List<? extends Coder<?>> getComponents(WindowedValue.ParamWindowedValueCoder<?> from) {
+ return ImmutableList.of(from.getValueCoder(), from.getWindowCoder());
+ }
+
+ @Override
+ public byte[] getPayload(WindowedValue.ParamWindowedValueCoder<?> from) {
+ return WindowedValue.ParamWindowedValueCoder.getPayload(from);
+ }
+
+ @Override
+ public WindowedValue.ParamWindowedValueCoder<?> fromComponents(
+ List<Coder<?>> components, byte[] payload) {
+ return WindowedValue.ParamWindowedValueCoder.fromComponents(components, payload);
+ }
+ };
+ }
+
static CoderTranslator<RowCoder> row() {
return new CoderTranslator<RowCoder>() {
@Override
@@ -134,7 +154,7 @@
@Override
public byte[] getPayload(RowCoder from) {
- return SchemaTranslation.schemaToProto(from.getSchema()).toByteArray();
+ return SchemaTranslation.schemaToProto(from.getSchema(), true).toByteArray();
}
@Override
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java
index 76881e2..e5edc35 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java
@@ -27,7 +27,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
@@ -234,15 +233,10 @@
.build();
}
- public static SdkFunctionSpec toProto(
- GlobalCombineFn<?, ?, ?> combineFn, SdkComponents components) {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(JAVA_SERIALIZED_COMBINE_FN_URN)
- .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(combineFn)))
- .build())
+ public static FunctionSpec toProto(GlobalCombineFn<?, ?, ?> combineFn, SdkComponents components) {
+ return FunctionSpec.newBuilder()
+ .setUrn(JAVA_SERIALIZED_COMBINE_FN_URN)
+ .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(combineFn)))
.build();
}
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
index 79b0111..ed94642 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
@@ -23,56 +23,21 @@
import java.util.Optional;
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.DockerPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExternalPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.ProcessPayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ReadPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.StandardEnvironments;
-import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
import org.apache.beam.sdk.util.ReleaseInfo;
import org.apache.beam.sdk.util.common.ReflectHelpers;
import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
/** Utilities for interacting with portability {@link Environment environments}. */
public class Environments {
- private static final ImmutableMap<String, EnvironmentIdExtractor> KNOWN_URN_SPEC_EXTRACTORS =
- ImmutableMap.<String, EnvironmentIdExtractor>builder()
- .put(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN, Environments::combineExtractor)
- .put(
- PTransformTranslation.COMBINE_PER_KEY_PRECOMBINE_TRANSFORM_URN,
- Environments::combineExtractor)
- .put(
- PTransformTranslation.COMBINE_PER_KEY_MERGE_ACCUMULATORS_TRANSFORM_URN,
- Environments::combineExtractor)
- .put(
- PTransformTranslation.COMBINE_PER_KEY_EXTRACT_OUTPUTS_TRANSFORM_URN,
- Environments::combineExtractor)
- .put(PTransformTranslation.PAR_DO_TRANSFORM_URN, Environments::parDoExtractor)
- .put(PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN, Environments::parDoExtractor)
- .put(
- PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN,
- Environments::parDoExtractor)
- .put(
- PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN,
- Environments::parDoExtractor)
- .put(
- PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN,
- Environments::parDoExtractor)
- .put(PTransformTranslation.READ_TRANSFORM_URN, Environments::readExtractor)
- .put(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN, Environments::windowExtractor)
- .build();
-
- private static final EnvironmentIdExtractor DEFAULT_SPEC_EXTRACTOR = transform -> null;
-
private static final ObjectMapper MAPPER =
new ObjectMapper()
.registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader()));
@@ -181,73 +146,31 @@
}
public static Optional<Environment> getEnvironment(String ptransformId, Components components) {
- try {
- PTransform ptransform = components.getTransformsOrThrow(ptransformId);
- String envId =
- KNOWN_URN_SPEC_EXTRACTORS
- .getOrDefault(ptransform.getSpec().getUrn(), DEFAULT_SPEC_EXTRACTOR)
- .getEnvironmentId(ptransform);
- if (Strings.isNullOrEmpty(envId)) {
- // Some PTransform payloads may have an unspecified (empty) Environment ID, for example a
- // WindowIntoPayload with a known WindowFn. Others will never have an Environment ID, such
- // as a GroupByKeyPayload, and the Default extractor returns null in this case.
- return Optional.empty();
- } else {
- return Optional.of(components.getEnvironmentsOrThrow(envId));
- }
- } catch (IOException e) {
- throw new RuntimeException(e);
+ PTransform ptransform = components.getTransformsOrThrow(ptransformId);
+ String envId = ptransform.getEnvironmentId();
+ if (Strings.isNullOrEmpty(envId)) {
+ // Some PTransform payloads may have an unspecified (empty) Environment ID, for example a
+ // WindowIntoPayload with a known WindowFn. Others will never have an Environment ID, such
+ // as a GroupByKeyPayload, and we return null in this case.
+ return Optional.empty();
+ } else {
+ return Optional.of(components.getEnvironmentsOrThrow(envId));
}
}
public static Optional<Environment> getEnvironment(
PTransform ptransform, RehydratedComponents components) {
- try {
- String envId =
- KNOWN_URN_SPEC_EXTRACTORS
- .getOrDefault(ptransform.getSpec().getUrn(), DEFAULT_SPEC_EXTRACTOR)
- .getEnvironmentId(ptransform);
- if (!Strings.isNullOrEmpty(envId)) {
- // Some PTransform payloads may have an empty (default) Environment ID, for example a
- // WindowIntoPayload with a known WindowFn. Others will never have an Environment ID, such
- // as a GroupByKeyPayload, and the Default extractor returns null in this case.
- return Optional.of(components.getEnvironment(envId));
- } else {
- return Optional.empty();
- }
- } catch (IOException e) {
- throw new RuntimeException(e);
+ String envId = ptransform.getEnvironmentId();
+ if (Strings.isNullOrEmpty(envId)) {
+ return Optional.empty();
+ } else {
+ // Some PTransform payloads may have an empty (default) Environment ID, for example a
+ // WindowIntoPayload with a known WindowFn. Others will never have an Environment ID, such
+ // as a GroupByKeyPayload, and we return null in this case.
+ return Optional.of(components.getEnvironment(envId));
}
}
- private interface EnvironmentIdExtractor {
- @Nullable
- String getEnvironmentId(PTransform transform) throws IOException;
- }
-
- private static String parDoExtractor(PTransform pTransform)
- throws InvalidProtocolBufferException {
- return ParDoPayload.parseFrom(pTransform.getSpec().getPayload()).getDoFn().getEnvironmentId();
- }
-
- private static String combineExtractor(PTransform pTransform)
- throws InvalidProtocolBufferException {
- return CombinePayload.parseFrom(pTransform.getSpec().getPayload())
- .getCombineFn()
- .getEnvironmentId();
- }
-
- private static String readExtractor(PTransform transform) throws InvalidProtocolBufferException {
- return ReadPayload.parseFrom(transform.getSpec().getPayload()).getSource().getEnvironmentId();
- }
-
- private static String windowExtractor(PTransform transform)
- throws InvalidProtocolBufferException {
- return WindowIntoPayload.parseFrom(transform.getSpec().getPayload())
- .getWindowFn()
- .getEnvironmentId();
- }
-
private static class ProcessPayloadReferenceJSON {
@Nullable private String os;
@Nullable private String arch;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
index 1a842b6..d58346b 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
@@ -66,16 +66,17 @@
return namespaceCounter.getAndIncrement();
}
- public static <OutputT> SingleOutputExpandableTransform<OutputT> of(
- String urn, byte[] payload, String endpoint) {
+ public static <InputT extends PInput, OutputT>
+ SingleOutputExpandableTransform<InputT, OutputT> of(
+ String urn, byte[] payload, String endpoint) {
Endpoints.ApiServiceDescriptor apiDesc =
Endpoints.ApiServiceDescriptor.newBuilder().setUrl(endpoint).build();
return new SingleOutputExpandableTransform<>(urn, payload, apiDesc, getFreshNamespaceIndex());
}
/** Expandable transform for output type of PCollection. */
- public static class SingleOutputExpandableTransform<OutputT>
- extends ExpandableTransform<PCollection<OutputT>> {
+ public static class SingleOutputExpandableTransform<InputT extends PInput, OutputT>
+ extends ExpandableTransform<InputT, PCollection<OutputT>> {
SingleOutputExpandableTransform(
String urn,
byte[] payload,
@@ -90,14 +91,20 @@
return Iterables.getOnlyElement(output.values());
}
- public MultiOutputExpandableTransform withMultiOutputs() {
- return new MultiOutputExpandableTransform(
+ public MultiOutputExpandableTransform<InputT> withMultiOutputs() {
+ return new MultiOutputExpandableTransform<>(
+ getUrn(), getPayload(), getEndpoint(), getNamespaceIndex());
+ }
+
+ public <T> SingleOutputExpandableTransform<InputT, T> withOutputType() {
+ return new SingleOutputExpandableTransform<>(
getUrn(), getPayload(), getEndpoint(), getNamespaceIndex());
}
}
/** Expandable transform for output type of PCollectionTuple. */
- public static class MultiOutputExpandableTransform extends ExpandableTransform<PCollectionTuple> {
+ public static class MultiOutputExpandableTransform<InputT extends PInput>
+ extends ExpandableTransform<InputT, PCollectionTuple> {
MultiOutputExpandableTransform(
String urn,
byte[] payload,
@@ -119,8 +126,8 @@
}
/** Base Expandable Transform which calls ExpansionService to expand itself. */
- public abstract static class ExpandableTransform<OutputT extends POutput>
- extends PTransform<PInput, OutputT> {
+ public abstract static class ExpandableTransform<InputT extends PInput, OutputT extends POutput>
+ extends PTransform<InputT, OutputT> {
private final String urn;
private final byte[] payload;
private final Endpoints.ApiServiceDescriptor endpoint;
@@ -142,7 +149,7 @@
}
@Override
- public OutputT expand(PInput input) {
+ public OutputT expand(InputT input) {
Pipeline p = input.getPipeline();
SdkComponents components = SdkComponents.create(p.getOptions());
RunnerApi.PTransform.Builder ptransformBuilder =
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java
index 937e982..39be910 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java
@@ -37,7 +37,7 @@
/** Translator for ExpandableTransform. */
public static class ExternalTranslator
- implements PTransformTranslation.TransformTranslator<External.ExpandableTransform<?>> {
+ implements PTransformTranslation.TransformTranslator<External.ExpandableTransform<?, ?>> {
public static PTransformTranslation.TransformTranslator create() {
return new ExternalTranslator();
}
@@ -125,6 +125,7 @@
transformBuilder
.setUniqueName(proto.getUniqueName())
.setSpec(proto.getSpec())
+ .setEnvironmentId(proto.getEnvironmentId())
.addAllSubtransforms(proto.getSubtransformsList());
for (Map.Entry<String, String> inputEntry : proto.getInputsMap().entrySet()) {
transformBuilder.putInputs(
@@ -144,6 +145,7 @@
.setUniqueName(expandedTransform.getUniqueName())
.setSpec(expandedTransform.getSpec())
.addAllSubtransforms(expandedTransform.getSubtransformsList())
+ .setEnvironmentId(expandedTransform.getEnvironmentId())
.putAllInputs(expandedTransform.getInputsMap());
for (Map.Entry<String, String> outputEntry : expandedTransform.getOutputsMap().entrySet()) {
rootTransformBuilder.putOutputs(
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java
index 854f523..8ff9063 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java
@@ -34,6 +34,7 @@
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder;
+import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.BiMap;
@@ -60,6 +61,9 @@
.put(LengthPrefixCoder.class, ModelCoders.LENGTH_PREFIX_CODER_URN)
.put(GlobalWindow.Coder.class, ModelCoders.GLOBAL_WINDOW_CODER_URN)
.put(FullWindowedValueCoder.class, ModelCoders.WINDOWED_VALUE_CODER_URN)
+ .put(
+ WindowedValue.ParamWindowedValueCoder.class,
+ ModelCoders.PARAM_WINDOWED_VALUE_CODER_URN)
.put(DoubleCoder.class, ModelCoders.DOUBLE_CODER_URN)
.put(RowCoder.class, ModelCoders.ROW_CODER_URN)
.build();
@@ -80,6 +84,7 @@
.put(Timer.Coder.class, CoderTranslators.timer())
.put(LengthPrefixCoder.class, CoderTranslators.lengthPrefix())
.put(FullWindowedValueCoder.class, CoderTranslators.fullWindowedValue())
+ .put(WindowedValue.ParamWindowedValueCoder.class, CoderTranslators.paramWindowedValue())
.put(DoubleCoder.class, CoderTranslators.atomic(DoubleCoder.class))
.put(RowCoder.class, CoderTranslators.row())
.build();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
index 486e39c..3d6d4dd 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
@@ -25,6 +25,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.Coder;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.StandardCoders;
+import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
/** Utilities and constants ot interact with coders that are part of the Beam Model. */
@@ -53,6 +54,8 @@
getUrn(StandardCoders.Enum.INTERVAL_WINDOW);
public static final String WINDOWED_VALUE_CODER_URN = getUrn(StandardCoders.Enum.WINDOWED_VALUE);
+ public static final String PARAM_WINDOWED_VALUE_CODER_URN =
+ getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE);
public static final String ROW_CODER_URN = getUrn(StandardCoders.Enum.ROW);
@@ -70,7 +73,8 @@
INTERVAL_WINDOW_CODER_URN,
WINDOWED_VALUE_CODER_URN,
DOUBLE_CODER_URN,
- ROW_CODER_URN);
+ ROW_CODER_URN,
+ PARAM_WINDOWED_VALUE_CODER_URN);
public static Set<String> urns() {
return MODEL_CODER_URNS;
@@ -90,6 +94,18 @@
.build();
}
+ public static Coder paramWindowedValueCoder(
+ String elementCoderId, String windowCoderId, byte[] payload) {
+ return Coder.newBuilder()
+ .setSpec(
+ FunctionSpec.newBuilder()
+ .setUrn(PARAM_WINDOWED_VALUE_CODER_URN)
+ .setPayload(ByteString.copyFrom(payload)))
+ .addComponentCoderIds(elementCoderId)
+ .addComponentCoderIds(windowCoderId)
+ .build();
+ }
+
/** Components of a Windowed Value {@link Coder} with names. */
@AutoValue
public abstract static class WindowedValueCoderComponents {
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java
index d9ad758..d9ddb93 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java
@@ -72,12 +72,12 @@
}
/**
- * Converts a {@link org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec} into a {@link
+ * Converts a {@link org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec} into a {@link
* ViewFn} using the URN.
*/
- public static ViewFn<?, ?> viewFnFromProto(RunnerApi.SdkFunctionSpec viewFn)
+ public static ViewFn<?, ?> viewFnFromProto(RunnerApi.FunctionSpec viewFn)
throws InvalidProtocolBufferException {
- RunnerApi.FunctionSpec spec = viewFn.getSpec();
+ RunnerApi.FunctionSpec spec = viewFn;
checkArgument(
spec.getUrn().equals(ParDoTranslation.CUSTOM_JAVA_VIEW_FN_URN),
"Can't deserialize unknown %s type %s",
@@ -89,12 +89,12 @@
}
/**
- * Converts a {@link org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec} into a {@link
+ * Converts a {@link org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec} into a {@link
* WindowMappingFn} using the URN.
*/
- public static WindowMappingFn<?> windowMappingFnFromProto(
- RunnerApi.SdkFunctionSpec windowMappingFn) throws InvalidProtocolBufferException {
- RunnerApi.FunctionSpec spec = windowMappingFn.getSpec();
+ public static WindowMappingFn<?> windowMappingFnFromProto(RunnerApi.FunctionSpec windowMappingFn)
+ throws InvalidProtocolBufferException {
+ RunnerApi.FunctionSpec spec = windowMappingFn;
checkArgument(
spec.getUrn().equals(ParDoTranslation.CUSTOM_JAVA_WINDOW_MAPPING_FN_URN),
"Can't deserialize unknown %s type %s",
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
index 8c73964..c21cd38 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
@@ -38,6 +38,7 @@
import org.apache.beam.runners.core.construction.ExternalTranslation.ExternalTranslator;
import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoTranslator;
import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.display.DisplayData;
@@ -48,6 +49,7 @@
import org.apache.beam.sdk.values.PValue;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSortedSet;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
@@ -278,7 +280,7 @@
if (spec != null) {
transformBuilder.setSpec(spec);
}
-
+ transformBuilder.setEnvironmentId(components.getOnlyEnvironmentId());
return transformBuilder.build();
}
}
@@ -292,6 +294,11 @@
private static final Map<Class<? extends PTransform>, TransformPayloadTranslator>
KNOWN_PAYLOAD_TRANSLATORS = loadTransformPayloadTranslators();
+ // TODO: BEAM-9001 - set environment ID in all transforms and allow runners to override.
+ private static List<String> sdkTransformsWithEnvironment =
+ ImmutableList.of(
+ PAR_DO_TRANSFORM_URN, COMBINE_PER_KEY_TRANSFORM_URN, ASSIGN_WINDOWS_TRANSFORM_URN);
+
private static Map<Class<? extends PTransform>, TransformPayloadTranslator>
loadTransformPayloadTranslators() {
HashMap<Class<? extends PTransform>, TransformPayloadTranslator> translators =
@@ -342,6 +349,16 @@
.translate(appliedPTransform, components);
if (spec != null) {
transformBuilder.setSpec(spec);
+
+ if (sdkTransformsWithEnvironment.contains(spec.getUrn())) {
+ transformBuilder.setEnvironmentId(components.getOnlyEnvironmentId());
+ } else if (spec.getUrn().equals(READ_TRANSFORM_URN)
+ && (appliedPTransform.getTransform().getClass() == Read.Bounded.class)) {
+ // Only assigning environment to Bounded reads. Not assigning an environment to Unbounded
+ // reads since they are a Runner translated transform, unless, in the future, we have an
+ // adapter available for splittable DoFn.
+ transformBuilder.setEnvironmentId(components.getOnlyEnvironmentId());
+ }
}
return transformBuilder.build();
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
index 280e2f3..f26f49ba 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
@@ -37,7 +37,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Parameter.Type;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput.Builder;
import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
@@ -127,6 +126,7 @@
.setUrn(PAR_DO_TRANSFORM_URN)
.setPayload(payload.toByteString())
.build());
+ builder.setEnvironmentId(components.getOnlyEnvironmentId());
String mainInputName = getMainInputName(builder, payload);
PCollection<KV<?, ?>> mainInput =
@@ -207,7 +207,7 @@
return payloadForParDoLike(
new ParDoLike() {
@Override
- public SdkFunctionSpec translateDoFn(SdkComponents newComponents) {
+ public FunctionSpec translateDoFn(SdkComponents newComponents) {
return ParDoTranslation.translateDoFn(
parDo.getFn(),
parDo.getMainOutputTag(),
@@ -508,7 +508,7 @@
case BAG_SPEC:
return StateSpecs.bag(components.getCoder(stateSpec.getBagSpec().getElementCoderId()));
case COMBINING_SPEC:
- FunctionSpec combineFnSpec = stateSpec.getCombiningSpec().getCombineFn().getSpec();
+ FunctionSpec combineFnSpec = stateSpec.getCombiningSpec().getCombineFn();
if (!combineFnSpec.getUrn().equals(CombineTranslation.JAVA_SERIALIZED_COMBINE_FN_URN)) {
throw new UnsupportedOperationException(
@@ -576,36 +576,32 @@
}
}
- public static SdkFunctionSpec translateDoFn(
+ public static FunctionSpec translateDoFn(
DoFn<?, ?> fn,
TupleTag<?> tag,
Map<String, PCollectionView<?>> sideInputMapping,
DoFnSchemaInformation doFnSchemaInformation,
SdkComponents components) {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(CUSTOM_JAVA_DO_FN_URN)
- .setPayload(
- ByteString.copyFrom(
- SerializableUtils.serializeToByteArray(
- DoFnWithExecutionInformation.of(
- fn, tag, sideInputMapping, doFnSchemaInformation))))
- .build())
+ return FunctionSpec.newBuilder()
+ .setUrn(CUSTOM_JAVA_DO_FN_URN)
+ .setPayload(
+ ByteString.copyFrom(
+ SerializableUtils.serializeToByteArray(
+ DoFnWithExecutionInformation.of(
+ fn, tag, sideInputMapping, doFnSchemaInformation))))
.build();
}
public static DoFnWithExecutionInformation doFnWithExecutionInformationFromProto(
- SdkFunctionSpec fnSpec) {
+ FunctionSpec fnSpec) {
checkArgument(
- fnSpec.getSpec().getUrn().equals(CUSTOM_JAVA_DO_FN_URN),
+ fnSpec.getUrn().equals(CUSTOM_JAVA_DO_FN_URN),
"Expected %s to be %s with URN %s, but URN was %s",
DoFn.class.getSimpleName(),
FunctionSpec.class.getSimpleName(),
CUSTOM_JAVA_DO_FN_URN,
- fnSpec.getSpec().getUrn());
- byte[] serializedFn = fnSpec.getSpec().getPayload().toByteArray();
+ fnSpec.getUrn());
+ byte[] serializedFn = fnSpec.getPayload().toByteArray();
return (DoFnWithExecutionInformation)
SerializableUtils.deserializeFromByteArray(serializedFn, "Custom DoFn With Execution Info");
}
@@ -662,14 +658,10 @@
return builder.build();
}
- public static SdkFunctionSpec translateViewFn(ViewFn<?, ?> viewFn, SdkComponents components) {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(CUSTOM_JAVA_VIEW_FN_URN)
- .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(viewFn)))
- .build())
+ public static FunctionSpec translateViewFn(ViewFn<?, ?> viewFn, SdkComponents components) {
+ return FunctionSpec.newBuilder()
+ .setUrn(CUSTOM_JAVA_VIEW_FN_URN)
+ .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(viewFn)))
.build();
}
@@ -696,22 +688,17 @@
return payload.getSplittable();
}
- public static SdkFunctionSpec translateWindowMappingFn(
+ public static FunctionSpec translateWindowMappingFn(
WindowMappingFn<?> windowMappingFn, SdkComponents components) {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(CUSTOM_JAVA_WINDOW_MAPPING_FN_URN)
- .setPayload(
- ByteString.copyFrom(SerializableUtils.serializeToByteArray(windowMappingFn)))
- .build())
+ return FunctionSpec.newBuilder()
+ .setUrn(CUSTOM_JAVA_WINDOW_MAPPING_FN_URN)
+ .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(windowMappingFn)))
.build();
}
/** These methods drive to-proto translation from Java and from rehydrated ParDos. */
public interface ParDoLike {
- SdkFunctionSpec translateDoFn(SdkComponents newComponents);
+ FunctionSpec translateDoFn(SdkComponents newComponents);
List<RunnerApi.Parameter> translateParameters();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java
index f5b9c7f..81bafab 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java
@@ -27,7 +27,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.IsBounded;
import org.apache.beam.model.pipeline.v1.RunnerApi.ReadPayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.Read;
@@ -65,7 +64,7 @@
.build();
}
- public static SdkFunctionSpec toProto(Source<?> source, SdkComponents components) {
+ public static FunctionSpec toProto(Source<?> source, SdkComponents components) {
if (source instanceof BoundedSource) {
return toProto((BoundedSource) source, components);
} else if (source instanceof UnboundedSource) {
@@ -76,14 +75,10 @@
}
}
- private static SdkFunctionSpec toProto(BoundedSource<?> source, SdkComponents components) {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(JAVA_SERIALIZED_BOUNDED_SOURCE)
- .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(source)))
- .build())
+ private static FunctionSpec toProto(BoundedSource<?> source, SdkComponents components) {
+ return FunctionSpec.newBuilder()
+ .setUrn(JAVA_SERIALIZED_BOUNDED_SOURCE)
+ .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(source)))
.build();
}
@@ -92,7 +87,7 @@
checkArgument(payload.getIsBounded().equals(IsBounded.Enum.BOUNDED));
return (BoundedSource<?>)
SerializableUtils.deserializeFromByteArray(
- payload.getSource().getSpec().getPayload().toByteArray(), "BoundedSource");
+ payload.getSource().getPayload().toByteArray(), "BoundedSource");
}
public static <T> BoundedSource<T> boundedSourceFromTransform(
@@ -118,15 +113,10 @@
.getPayload());
}
- private static SdkFunctionSpec toProto(UnboundedSource<?, ?> source, SdkComponents components) {
- return SdkFunctionSpec.newBuilder()
- // Do not assign an environment. Unbounded reads are a Runner translated transform,
- // unless, in the future, we have an adapter available for splittable DoFn.
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(JAVA_SERIALIZED_UNBOUNDED_SOURCE)
- .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(source)))
- .build())
+ private static FunctionSpec toProto(UnboundedSource<?, ?> source, SdkComponents components) {
+ return FunctionSpec.newBuilder()
+ .setUrn(JAVA_SERIALIZED_UNBOUNDED_SOURCE)
+ .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(source)))
.build();
}
@@ -134,7 +124,7 @@
checkArgument(payload.getIsBounded().equals(IsBounded.Enum.UNBOUNDED));
return (UnboundedSource<?, ?>)
SerializableUtils.deserializeFromByteArray(
- payload.getSource().getSpec().getPayload().toByteArray(), "UnboundedSource");
+ payload.getSource().getPayload().toByteArray(), "UnboundedSource");
}
public static PCollection.IsBounded sourceIsBounded(AppliedPTransform<?, ?, ?> transform) {
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
index 2373188..f89874e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
@@ -30,7 +30,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Parameter;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
@@ -366,7 +365,7 @@
ParDoTranslation.payloadForParDoLike(
new ParDoLike() {
@Override
- public SdkFunctionSpec translateDoFn(SdkComponents newComponents) {
+ public FunctionSpec translateDoFn(SdkComponents newComponents) {
// Schemas not yet supported on splittable DoFn.
return ParDoTranslation.translateDoFn(
fn,
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
index 072dbc7..d994762 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
@@ -47,9 +47,13 @@
/** Returns a timer for the given timestamp with a user specified payload. */
public static <T> Timer<T> of(Instant timestamp, @Nullable T payload) {
- return new AutoValue_Timer(timestamp, payload);
+ return new AutoValue_Timer(timestamp, timestamp, payload);
}
+ /** Returns a timer for the given timestamp with a user specified payload and outputTimestamp. */
+ public static <T> Timer<T> of(Instant timestamp, Instant outputTimestamp, @Nullable T payload) {
+ return new AutoValue_Timer(timestamp, outputTimestamp, payload);
+ }
/**
* Returns the timestamp of when the timer is scheduled to fire.
*
@@ -58,6 +62,9 @@
*/
public abstract Instant getTimestamp();
+ /* Returns the outputTimestamps */
+ public abstract Instant getOutputTimestamp();
+
/** A user supplied payload. */
@Nullable
public abstract T getPayload();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java
index a57aa9b..63f662f 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java
@@ -25,7 +25,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.OutputTime;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.StandardWindowFns.FixedWindowsPayload;
import org.apache.beam.model.pipeline.v1.StandardWindowFns.GlobalWindowsPayload;
import org.apache.beam.model.pipeline.v1.StandardWindowFns.SessionsPayload;
@@ -216,25 +215,19 @@
* RunnerApi.MessageWithComponents#getFunctionSpec()} is a {@link RunnerApi.FunctionSpec} for the
* input {@link WindowFn}.
*/
- public static SdkFunctionSpec toProto(WindowFn<?, ?> windowFn, SdkComponents components) {
+ public static FunctionSpec toProto(WindowFn<?, ?> windowFn, SdkComponents components) {
ByteString serializedFn = ByteString.copyFrom(SerializableUtils.serializeToByteArray(windowFn));
if (windowFn instanceof GlobalWindows) {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(FunctionSpec.newBuilder().setUrn(GLOBAL_WINDOWS_URN))
- .build();
+ return FunctionSpec.newBuilder().setUrn(GLOBAL_WINDOWS_URN).build();
} else if (windowFn instanceof FixedWindows) {
FixedWindowsPayload fixedWindowsPayload =
FixedWindowsPayload.newBuilder()
.setSize(Durations.fromMillis(((FixedWindows) windowFn).getSize().getMillis()))
.setOffset(Timestamps.fromMillis(((FixedWindows) windowFn).getOffset().getMillis()))
.build();
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(FIXED_WINDOWS_URN)
- .setPayload(fixedWindowsPayload.toByteString()))
+ return FunctionSpec.newBuilder()
+ .setUrn(FIXED_WINDOWS_URN)
+ .setPayload(fixedWindowsPayload.toByteString())
.build();
} else if (windowFn instanceof SlidingWindows) {
SlidingWindowsPayload slidingWindowsPayload =
@@ -243,32 +236,23 @@
.setOffset(Timestamps.fromMillis(((SlidingWindows) windowFn).getOffset().getMillis()))
.setPeriod(Durations.fromMillis(((SlidingWindows) windowFn).getPeriod().getMillis()))
.build();
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(SLIDING_WINDOWS_URN)
- .setPayload(slidingWindowsPayload.toByteString()))
+ return FunctionSpec.newBuilder()
+ .setUrn(SLIDING_WINDOWS_URN)
+ .setPayload(slidingWindowsPayload.toByteString())
.build();
} else if (windowFn instanceof Sessions) {
SessionsPayload sessionsPayload =
SessionsPayload.newBuilder()
.setGapSize(Durations.fromMillis(((Sessions) windowFn).getGapDuration().getMillis()))
.build();
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(SESSION_WINDOWS_URN)
- .setPayload(sessionsPayload.toByteString()))
+ return FunctionSpec.newBuilder()
+ .setUrn(SESSION_WINDOWS_URN)
+ .setPayload(sessionsPayload.toByteString())
.build();
} else {
- return SdkFunctionSpec.newBuilder()
- .setEnvironmentId(components.getOnlyEnvironmentId())
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(SERIALIZED_JAVA_WINDOWFN_URN)
- .setPayload(serializedFn))
+ return FunctionSpec.newBuilder()
+ .setUrn(SERIALIZED_JAVA_WINDOWFN_URN)
+ .setPayload(serializedFn)
.build();
}
}
@@ -295,7 +279,7 @@
*/
public static RunnerApi.WindowingStrategy toProto(
WindowingStrategy<?, ?> windowingStrategy, SdkComponents components) throws IOException {
- SdkFunctionSpec windowFnSpec = toProto(windowingStrategy.getWindowFn(), components);
+ FunctionSpec windowFnSpec = toProto(windowingStrategy.getWindowFn(), components);
RunnerApi.WindowingStrategy.Builder windowingStrategyProto =
RunnerApi.WindowingStrategy.newBuilder()
@@ -308,7 +292,8 @@
.setAssignsToOneWindow(windowingStrategy.getWindowFn().assignsToOneWindow())
.setOnTimeBehavior(toProto(windowingStrategy.getOnTimeBehavior()))
.setWindowCoderId(
- components.registerCoder(windowingStrategy.getWindowFn().windowCoder()));
+ components.registerCoder(windowingStrategy.getWindowFn().windowCoder()))
+ .setEnvironmentId(components.getOnlyEnvironmentId());
return windowingStrategyProto.build();
}
@@ -340,7 +325,7 @@
RunnerApi.WindowingStrategy proto, RehydratedComponents components)
throws InvalidProtocolBufferException {
- SdkFunctionSpec windowFnSpec = proto.getWindowFn();
+ FunctionSpec windowFnSpec = proto.getWindowFn();
WindowFn<?, ?> windowFn = windowFnFromProto(windowFnSpec);
TimestampCombiner timestampCombiner = timestampCombinerFromProto(proto.getOutputTime());
AccumulationMode accumulationMode = fromProto(proto.getAccumulationMode());
@@ -358,34 +343,32 @@
.withOnTimeBehavior(onTimeBehavior);
}
- public static WindowFn<?, ?> windowFnFromProto(SdkFunctionSpec windowFnSpec) {
+ public static WindowFn<?, ?> windowFnFromProto(FunctionSpec windowFnSpec) {
try {
- String s = windowFnSpec.getSpec().getUrn();
+ String s = windowFnSpec.getUrn();
if (s.equals(getUrn(GlobalWindowsPayload.Enum.PROPERTIES))) {
return new GlobalWindows();
} else if (s.equals(getUrn(FixedWindowsPayload.Enum.PROPERTIES))) {
- FixedWindowsPayload fixedParams =
- FixedWindowsPayload.parseFrom(windowFnSpec.getSpec().getPayload());
+ FixedWindowsPayload fixedParams = FixedWindowsPayload.parseFrom(windowFnSpec.getPayload());
return FixedWindows.of(Duration.millis(Durations.toMillis(fixedParams.getSize())))
.withOffset(Duration.millis(Timestamps.toMillis(fixedParams.getOffset())));
} else if (s.equals(getUrn(SlidingWindowsPayload.Enum.PROPERTIES))) {
SlidingWindowsPayload slidingParams =
- SlidingWindowsPayload.parseFrom(windowFnSpec.getSpec().getPayload());
+ SlidingWindowsPayload.parseFrom(windowFnSpec.getPayload());
return SlidingWindows.of(Duration.millis(Durations.toMillis(slidingParams.getSize())))
.every(Duration.millis(Durations.toMillis(slidingParams.getPeriod())))
.withOffset(Duration.millis(Timestamps.toMillis(slidingParams.getOffset())));
} else if (s.equals(getUrn(SessionsPayload.Enum.PROPERTIES))) {
- SessionsPayload sessionParams =
- SessionsPayload.parseFrom(windowFnSpec.getSpec().getPayload());
+ SessionsPayload sessionParams = SessionsPayload.parseFrom(windowFnSpec.getPayload());
return Sessions.withGapDuration(
Duration.millis(Durations.toMillis(sessionParams.getGapSize())));
} else if (s.equals(SERIALIZED_JAVA_WINDOWFN_URN)) {
return (WindowFn<?, ?>)
SerializableUtils.deserializeFromByteArray(
- windowFnSpec.getSpec().getPayload().toByteArray(), "WindowFn");
+ windowFnSpec.getPayload().toByteArray(), "WindowFn");
} else {
throw new IllegalArgumentException(
- "Unknown or unsupported WindowFn: " + windowFnSpec.getSpec().getUrn());
+ "Unknown or unsupported WindowFn: " + windowFnSpec.getUrn());
}
} catch (InvalidProtocolBufferException e) {
throw new IllegalArgumentException(
@@ -393,7 +376,7 @@
"%s for %s with URN %s did not contain expected proto message for payload",
FunctionSpec.class.getSimpleName(),
WindowFn.class.getSimpleName(),
- windowFnSpec.getSpec().getUrn()),
+ windowFnSpec.getUrn()),
e);
}
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java
index e86c450..0df16a2 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java
@@ -30,7 +30,6 @@
import java.util.Map;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.WriteFilesPayload;
import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
@@ -66,7 +65,7 @@
return payloadForWriteFilesLike(
new WriteFilesLike() {
@Override
- public SdkFunctionSpec translateSink(SdkComponents newComponents) {
+ public FunctionSpec translateSink(SdkComponents newComponents) {
// TODO: register the environment
return toProto(transform.getSink());
}
@@ -96,31 +95,27 @@
components);
}
- private static SdkFunctionSpec toProto(FileBasedSink<?, ?, ?> sink) {
+ private static FunctionSpec toProto(FileBasedSink<?, ?, ?> sink) {
return toProto(CUSTOM_JAVA_FILE_BASED_SINK_URN, sink);
}
- private static SdkFunctionSpec toProto(String urn, Serializable serializable) {
- return SdkFunctionSpec.newBuilder()
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(urn)
- .setPayload(
- ByteString.copyFrom(SerializableUtils.serializeToByteArray(serializable)))
- .build())
+ private static FunctionSpec toProto(String urn, Serializable serializable) {
+ return FunctionSpec.newBuilder()
+ .setUrn(urn)
+ .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(serializable)))
.build();
}
@VisibleForTesting
- static FileBasedSink<?, ?, ?> sinkFromProto(SdkFunctionSpec sinkProto) throws IOException {
+ static FileBasedSink<?, ?, ?> sinkFromProto(FunctionSpec sinkProto) throws IOException {
checkArgument(
- sinkProto.getSpec().getUrn().equals(CUSTOM_JAVA_FILE_BASED_SINK_URN),
+ sinkProto.getUrn().equals(CUSTOM_JAVA_FILE_BASED_SINK_URN),
"Cannot extract %s instance from %s with URN %s",
FileBasedSink.class.getSimpleName(),
FunctionSpec.class.getSimpleName(),
- sinkProto.getSpec().getUrn());
+ sinkProto.getUrn());
- byte[] serializedSink = sinkProto.getSpec().getPayload().toByteArray();
+ byte[] serializedSink = sinkProto.getPayload().toByteArray();
return (FileBasedSink<?, ?, ?>)
SerializableUtils.deserializeFromByteArray(
@@ -252,7 +247,7 @@
}
@Override
- public SdkFunctionSpec translateSink(SdkComponents newComponents) {
+ public FunctionSpec translateSink(SdkComponents newComponents) {
// TODO: re-register the environment with the new components
return payload.getSink();
}
@@ -304,7 +299,7 @@
/** These methods drive to-proto translation from Java and from rehydrated WriteFiles. */
private interface WriteFilesLike {
- SdkFunctionSpec translateSink(SdkComponents newComponents);
+ FunctionSpec translateSink(SdkComponents newComponents);
Map<String, RunnerApi.SideInput> translateSideInputs(SdkComponents components);
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java
index 8bdd718..dd2d374 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java
@@ -17,6 +17,8 @@
*/
package org.apache.beam.runners.core.construction.graph;
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;
@@ -31,6 +33,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
+import org.apache.beam.model.pipeline.v1.RunnerApi.WireCoderSetting;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
@@ -70,6 +73,14 @@
Environment getEnvironment();
/**
+ * Returns the {@link WireCoderSetting} this stage executes in.
+ *
+ * <p>A {@link WireCoderSetting} consists of settings which is used to configure the type of the
+ * wire coder.
+ */
+ WireCoderSetting getWireCoderSetting();
+
+ /**
* Returns the root {@link PCollectionNode} of this {@link ExecutableStage}. This {@link
* ExecutableStage} executes by reading elements from a Remote gRPC Read Node.
*
@@ -134,6 +145,7 @@
ExecutableStagePayload.Builder payload = ExecutableStagePayload.newBuilder();
payload.setEnvironment(getEnvironment());
+ payload.setWireCoderSetting(getWireCoderSetting());
// Populate inputs and outputs of the stage payload and outer PTransform simultaneously.
PCollectionNode input = getInputPCollection();
@@ -208,6 +220,7 @@
static ExecutableStage fromPayload(ExecutableStagePayload payload) {
Components components = payload.getComponents();
Environment environment = payload.getEnvironment();
+ WireCoderSetting wireCoderSetting = payload.getWireCoderSetting();
PCollectionNode input =
PipelineNode.pCollection(
@@ -233,6 +246,20 @@
.map(id -> PipelineNode.pCollection(id, components.getPcollectionsOrThrow(id)))
.collect(Collectors.toList());
return ImmutableExecutableStage.of(
- components, environment, input, sideInputs, userStates, timers, transforms, outputs);
+ components,
+ environment,
+ input,
+ sideInputs,
+ userStates,
+ timers,
+ transforms,
+ outputs,
+ wireCoderSetting);
}
+
+ /** The default wire coder, i.e., WINDOWED_VALUE coder. */
+ WireCoderSetting DEFAULT_WIRE_CODER_SETTING =
+ WireCoderSetting.newBuilder()
+ .setUrn(getUrn(RunnerApi.StandardCoders.Enum.WINDOWED_VALUE))
+ .build();
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
index 61d9546..00cca32 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
@@ -415,7 +415,8 @@
stage.getUserStates(),
stage.getTimers(),
pTransformNodes,
- stage.getOutputPCollections());
+ stage.getOutputPCollections(),
+ stage.getWireCoderSetting());
}
/**
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java
index 2334458..87f2076 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.construction.graph;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import java.util.ArrayDeque;
@@ -138,7 +139,8 @@
userStates,
timers,
fusedTransforms.build(),
- materializedPCollections);
+ materializedPCollections,
+ DEFAULT_WIRE_CODER_SETTING);
}
private static Environment getStageEnvironment(
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java
index a996ac0..0092056 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java
@@ -22,6 +22,7 @@
import java.util.stream.Collectors;
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.model.pipeline.v1.RunnerApi.WireCoderSetting;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
@@ -37,7 +38,8 @@
Collection<UserStateReference> userStates,
Collection<TimerReference> timers,
Collection<PTransformNode> transforms,
- Collection<PCollectionNode> outputs) {
+ Collection<PCollectionNode> outputs,
+ WireCoderSetting wireCoderSetting) {
Components prunedComponents =
components
.toBuilder()
@@ -47,7 +49,15 @@
.collect(Collectors.toMap(PTransformNode::getId, PTransformNode::getTransform)))
.build();
return of(
- prunedComponents, environment, input, sideInputs, userStates, timers, transforms, outputs);
+ prunedComponents,
+ environment,
+ input,
+ sideInputs,
+ userStates,
+ timers,
+ transforms,
+ outputs,
+ wireCoderSetting);
}
public static ImmutableExecutableStage of(
@@ -58,7 +68,8 @@
Collection<UserStateReference> userStates,
Collection<TimerReference> timers,
Collection<PTransformNode> transforms,
- Collection<PCollectionNode> outputs) {
+ Collection<PCollectionNode> outputs,
+ WireCoderSetting wireCoderSetting) {
return new AutoValue_ImmutableExecutableStage(
components,
environment,
@@ -67,7 +78,8 @@
ImmutableSet.copyOf(userStates),
ImmutableSet.copyOf(timers),
ImmutableSet.copyOf(transforms),
- ImmutableSet.copyOf(outputs));
+ ImmutableSet.copyOf(outputs),
+ wireCoderSetting);
}
@Override
@@ -94,4 +106,7 @@
@Override
public abstract Collection<PCollectionNode> getOutputPCollections();
+
+ @Override
+ public abstract WireCoderSetting getWireCoderSetting();
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java
index 157ece0..def7de8 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java
@@ -308,7 +308,8 @@
stage.getUserStates(),
stage.getTimers(),
updatedTransforms,
- updatedOutputs);
+ updatedOutputs,
+ stage.getWireCoderSetting());
}
/**
@@ -325,6 +326,7 @@
output.getKey(), originalToPartial.get(output.getValue()).getId());
}
}
+ updatedTransformBuilder.setEnvironmentId(transform.getEnvironmentId());
return updatedTransformBuilder.build();
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
index 77f0211..a320412 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
@@ -162,6 +162,7 @@
FunctionSpec.newBuilder()
.setUrn(PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN)
.setPayload(splittableParDo.getSpec().getPayload()));
+ pairWithRestriction.setEnvironmentId(splittableParDo.getEnvironmentId());
rval.getComponentsBuilder()
.putTransforms(pairWithRestrictionId, pairWithRestriction.build());
}
@@ -180,6 +181,7 @@
FunctionSpec.newBuilder()
.setUrn(PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN)
.setPayload(splittableParDo.getSpec().getPayload()));
+ splitAndSize.setEnvironmentId(splittableParDo.getEnvironmentId());
rval.getComponentsBuilder().putTransforms(splitAndSizeId, splitAndSize.build());
}
@@ -201,6 +203,7 @@
.setUrn(
PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN)
.setPayload(splittableParDo.getSpec().getPayload()));
+ processSizedElementsAndRestrictions.setEnvironmentId(splittableParDo.getEnvironmentId());
rval.getComponentsBuilder()
.putTransforms(
processSizedElementsAndRestrictionsId,
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java
index 4cdf60b..bfa1fa1e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetector.java
@@ -20,7 +20,7 @@
import io.github.classgraph.ClassGraph;
import java.io.File;
import java.util.List;
-import java.util.stream.Stream;
+import java.util.stream.Collectors;
/**
* Attempts to detect all the resources to be staged using classgraph library.
@@ -43,9 +43,10 @@
* @return A list of absolute paths to the resources the class loader uses.
*/
@Override
- public Stream<String> detect(ClassLoader classLoader) {
- List<File> classpathContents = classGraph.addClassLoader(classLoader).getClasspathFiles();
+ public List<String> detect(ClassLoader classLoader) {
+ List<File> classpathContents =
+ classGraph.disableNestedJarScanning().addClassLoader(classLoader).getClasspathFiles();
- return classpathContents.stream().map(File::getAbsolutePath);
+ return classpathContents.stream().map(File::getAbsolutePath).collect(Collectors.toList());
}
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java
index c3ecde4..dda68f1 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResources.java
@@ -50,11 +50,10 @@
ClassLoader classLoader, PipelineOptions options) {
PipelineResourcesOptions artifactsRelatedOptions = options.as(PipelineResourcesOptions.class);
- return artifactsRelatedOptions
- .getPipelineResourcesDetector()
- .detect(classLoader)
- .filter(isStageable())
- .collect(Collectors.toList());
+ List<String> detectedResources =
+ artifactsRelatedOptions.getPipelineResourcesDetector().detect(classLoader);
+
+ return detectedResources.stream().filter(isStageable()).collect(Collectors.toList());
}
/**
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java
index 46b9459..103c694 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetector.java
@@ -18,10 +18,15 @@
package org.apache.beam.runners.core.construction.resources;
import java.io.Serializable;
-import java.util.stream.Stream;
+import java.util.List;
/** Interface for an algorithm detecting classpath resources for pipelines. */
public interface PipelineResourcesDetector extends Serializable {
- Stream<String> detect(ClassLoader classLoader);
+ List<String> detect(ClassLoader classLoader);
+
+ /** Provides pipeline resources detection algorithm. */
+ interface Factory {
+ PipelineResourcesDetector getPipelineResourcesDetector();
+ }
}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetectorAbstractFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetectorAbstractFactory.java
deleted file mode 100644
index 2006955..0000000
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesDetectorAbstractFactory.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.runners.core.construction.resources;
-
-/** Provides pipeline resources detection algorithm. */
-public interface PipelineResourcesDetectorAbstractFactory {
- PipelineResourcesDetector getPipelineResourcesDetector();
-}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java
index 21f0c24..884664a 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesOptions.java
@@ -28,16 +28,24 @@
/** Pipeline options dedicated to detecting classpath resources. */
public interface PipelineResourcesOptions extends PipelineOptions {
+ /**
+ * The class of the pipeline resources detector factory that should be created and used to create
+ * the detector. If not set explicitly, a default class will be used to instantiate the factory.
+ */
+ @JsonIgnore
@Description(
"The class of the pipeline resources detector factory that should be created and used to create "
+ "the detector. If not set explicitly, a default class will be used to instantiate the factory.")
@Default.Class(ClasspathScanningResourcesDetectorFactory.class)
- Class<? extends PipelineResourcesDetectorAbstractFactory>
- getPipelineResourcesDetectorFactoryClass();
+ Class<? extends PipelineResourcesDetector.Factory> getPipelineResourcesDetectorFactoryClass();
void setPipelineResourcesDetectorFactoryClass(
- Class<? extends PipelineResourcesDetectorAbstractFactory> factoryClass);
+ Class<? extends PipelineResourcesDetector.Factory> factoryClass);
+ /**
+ * Instance of a pipeline resources detection algorithm. If not set explicitly, a default
+ * implementation will be used.
+ */
@JsonIgnore
@Description(
"Instance of a pipeline resources detection algorithm. If not set explicitly, a default implementation will be used")
@@ -46,14 +54,18 @@
void setPipelineResourcesDetector(PipelineResourcesDetector pipelineResourcesDetector);
+ /**
+ * Creates {@link PipelineResourcesDetector} instance based on provided pipeline options or
+ * default values set for them.
+ */
class PipelineResourcesDetectorFactory implements DefaultValueFactory<PipelineResourcesDetector> {
@Override
public PipelineResourcesDetector create(PipelineOptions options) {
PipelineResourcesOptions resourcesOptions = options.as(PipelineResourcesOptions.class);
- PipelineResourcesDetectorAbstractFactory resourcesToStage =
- InstanceBuilder.ofType(PipelineResourcesDetectorAbstractFactory.class)
+ PipelineResourcesDetector.Factory resourcesToStage =
+ InstanceBuilder.ofType(PipelineResourcesDetector.Factory.class)
.fromClass(resourcesOptions.getPipelineResourcesDetectorFactoryClass())
.fromFactoryMethod("create")
.build();
@@ -62,8 +74,8 @@
}
}
- class ClasspathScanningResourcesDetectorFactory
- implements PipelineResourcesDetectorAbstractFactory {
+ /** Constructs the default {@link PipelineResourcesDetector} instance. */
+ class ClasspathScanningResourcesDetectorFactory implements PipelineResourcesDetector.Factory {
public static ClasspathScanningResourcesDetectorFactory create() {
return new ClasspathScanningResourcesDetectorFactory();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
index c2c6804..b2adc2b 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
@@ -51,6 +51,7 @@
import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder;
+import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
@@ -79,6 +80,7 @@
.add(
FullWindowedValueCoder.of(
IterableCoder.of(VarLongCoder.of()), IntervalWindowCoder.of()))
+ .add(WindowedValue.ParamWindowedValueCoder.of(IterableCoder.of(VarLongCoder.of())))
.add(DoubleCoder.of())
.add(
RowCoder.of(
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java
index 73b26a3..ee026a4 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java
@@ -108,7 +108,7 @@
assertEquals(
combineFn,
SerializableUtils.deserializeFromByteArray(
- combineProto.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn"));
+ combineProto.getCombineFn().getPayload().toByteArray(), "CombineFn"));
}
}
@@ -151,7 +151,7 @@
assertEquals(
combineFn,
SerializableUtils.deserializeFromByteArray(
- combineProto.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn"));
+ combineProto.getCombineFn().getPayload().toByteArray(), "CombineFn"));
}
@Test
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
index 2de2fe8..56fae2f 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
@@ -106,6 +106,9 @@
.put(
getUrn(StandardCoders.Enum.WINDOWED_VALUE),
WindowedValue.FullWindowedValueCoder.class)
+ .put(
+ getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE),
+ WindowedValue.ParamWindowedValueCoder.class)
.put(getUrn(StandardCoders.Enum.ROW), RowCoder.class)
.build();
@@ -272,7 +275,8 @@
return convertedElements;
} else if (s.equals(getUrn(StandardCoders.Enum.GLOBAL_WINDOW))) {
return GlobalWindow.INSTANCE;
- } else if (s.equals(getUrn(StandardCoders.Enum.WINDOWED_VALUE))) {
+ } else if (s.equals(getUrn(StandardCoders.Enum.WINDOWED_VALUE))
+ || s.equals(getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE))) {
Map<String, Object> kvMap = (Map<String, Object>) value;
Coder valueCoder = ((WindowedValue.FullWindowedValueCoder) coder).getValueCoder();
Coder windowCoder = ((WindowedValue.FullWindowedValueCoder) coder).getWindowCoder();
@@ -438,6 +442,9 @@
} else if (s.equals(getUrn(StandardCoders.Enum.WINDOWED_VALUE))) {
assertEquals(expectedValue, actualValue);
+ } else if (s.equals(getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE))) {
+ assertEquals(expectedValue, actualValue);
+
} else if (s.equals(getUrn(StandardCoders.Enum.DOUBLE))) {
assertEquals(expectedValue, actualValue);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java
index 5cbe98a..68ff4ce 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java
@@ -24,32 +24,19 @@
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
-import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.DockerPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.ProcessPayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ReadPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.StandardEnvironments;
-import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.CountingSource;
-import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFnSchemaInformation;
import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.PartitioningWindowFn;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
@@ -112,9 +99,10 @@
}
@Test
- public void getEnvironmentParDo() throws IOException {
+ public void getEnvironmentPTransform() throws IOException {
SdkComponents components = SdkComponents.create();
- components.registerEnvironment(Environments.createDockerEnvironment("java"));
+ Environment env = Environments.createDockerEnvironment("java");
+ components.registerEnvironment(env);
ParDoPayload payload =
ParDoTranslation.translateParDo(
ParDo.of(
@@ -128,145 +116,18 @@
components);
RehydratedComponents rehydratedComponents =
RehydratedComponents.forComponents(components.toComponents());
- PTransform builder =
+ PTransform ptransform =
PTransform.newBuilder()
.setSpec(
FunctionSpec.newBuilder()
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(payload.toByteString())
.build())
+ .setEnvironmentId(components.getOnlyEnvironmentId())
.build();
- Environment env = Environments.getEnvironment(builder, rehydratedComponents).get();
+ Environment env1 = Environments.getEnvironment(ptransform, rehydratedComponents).get();
assertThat(
- env,
- equalTo(
- components
- .toComponents()
- .getEnvironmentsOrThrow(payload.getDoFn().getEnvironmentId())));
- }
-
- @Test
- public void getEnvironmentWindowIntoKnown() throws IOException {
- SdkComponents components = SdkComponents.create();
- components.registerEnvironment(Environments.createDockerEnvironment("java"));
- WindowIntoPayload payload =
- WindowIntoPayload.newBuilder()
- .setWindowFn(
- WindowingStrategyTranslation.toProto(
- FixedWindows.of(Duration.standardMinutes(5L)), components))
- .build();
- RehydratedComponents rehydratedComponents =
- RehydratedComponents.forComponents(components.toComponents());
- PTransform builder =
- PTransform.newBuilder()
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
- .setPayload(payload.toByteString())
- .build())
- .build();
- Environment env = Environments.getEnvironment(builder, rehydratedComponents).get();
- assertThat(
- env,
- equalTo(
- components
- .toComponents()
- .getEnvironmentsOrThrow(payload.getWindowFn().getEnvironmentId())));
- }
-
- @Test
- public void getEnvironmentWindowIntoCustom() throws IOException {
- SdkComponents components = SdkComponents.create();
- components.registerEnvironment(Environments.createDockerEnvironment("java"));
- WindowIntoPayload payload =
- WindowIntoPayload.newBuilder()
- .setWindowFn(
- WindowingStrategyTranslation.toProto(
- new PartitioningWindowFn<Object, BoundedWindow>() {
- @Override
- public BoundedWindow assignWindow(Instant timestamp) {
- return null;
- }
-
- @Override
- public boolean isCompatible(WindowFn<?, ?> other) {
- return false;
- }
-
- @Override
- public Coder<BoundedWindow> windowCoder() {
- return null;
- }
- },
- components))
- .build();
- RehydratedComponents rehydratedComponents =
- RehydratedComponents.forComponents(components.toComponents());
- PTransform builder =
- PTransform.newBuilder()
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
- .setPayload(payload.toByteString())
- .build())
- .build();
- Environment env = Environments.getEnvironment(builder, rehydratedComponents).get();
- assertThat(
- env,
- equalTo(
- components
- .toComponents()
- .getEnvironmentsOrThrow(payload.getWindowFn().getEnvironmentId())));
- }
-
- @Test
- public void getEnvironmentRead() throws IOException {
- SdkComponents components = SdkComponents.create();
- components.registerEnvironment(Environments.createDockerEnvironment("java"));
- ReadPayload payload = ReadTranslation.toProto(Read.from(CountingSource.upTo(10)), components);
- RehydratedComponents rehydratedComponents =
- RehydratedComponents.forComponents(components.toComponents());
- PTransform builder =
- PTransform.newBuilder()
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN)
- .setPayload(payload.toByteString())
- .build())
- .build();
- Environment env = Environments.getEnvironment(builder, rehydratedComponents).get();
- assertThat(
- env,
- equalTo(
- components
- .toComponents()
- .getEnvironmentsOrThrow(payload.getSource().getEnvironmentId())));
- }
-
- @Test
- public void getEnvironmentCombine() throws IOException {
- SdkComponents components = SdkComponents.create();
- components.registerEnvironment(Environments.createDockerEnvironment("java"));
- CombinePayload payload =
- CombinePayload.newBuilder()
- .setCombineFn(CombineTranslation.toProto(Sum.ofLongs(), components))
- .build();
- RehydratedComponents rehydratedComponents =
- RehydratedComponents.forComponents(components.toComponents());
- PTransform builder =
- PTransform.newBuilder()
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN)
- .setPayload(payload.toByteString())
- .build())
- .build();
- Environment env = Environments.getEnvironment(builder, rehydratedComponents).get();
- assertThat(
- env,
- equalTo(
- components
- .toComponents()
- .getEnvironmentsOrThrow(payload.getCombineFn().getEnvironmentId())));
+ env1,
+ equalTo(components.toComponents().getEnvironmentsOrThrow(ptransform.getEnvironmentId())));
}
}
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java
index 397b444..b399472 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java
@@ -139,8 +139,8 @@
testPipeline
.apply(Create.of("1", "2", "2", "3", "3", "3"))
.apply(
- External.<KV<String, Integer>>of(
- "beam:transforms:xlang:count", new byte[] {}, target))
+ External.of("beam:transforms:xlang:count", new byte[] {}, target)
+ .<KV<String, Long>>withOutputType())
.apply(
"toString",
MapElements.into(TypeDescriptors.strings())
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java
index 9324bab..6022d70 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java
@@ -85,7 +85,7 @@
@Test
public void toAndFromProto() throws Exception {
- SchemaApi.Schema schemaProto = SchemaTranslation.schemaToProto(schema);
+ SchemaApi.Schema schemaProto = SchemaTranslation.schemaToProto(schema, true);
Schema decodedSchema = SchemaTranslation.fromProto(schemaProto);
assertThat(decodedSchema, equalTo(schema));
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
index f7a0509..863165c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.construction.graph;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -33,7 +34,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
@@ -65,12 +65,13 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("foo"))
+ .setDoFn(FunctionSpec.newBuilder())
.putSideInputs("side_input", SideInput.getDefaultInstance())
.putStateSpecs("user_state", StateSpec.getDefaultInstance())
.putTimerSpecs("timer", TimerSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("foo")
.build();
PCollection input = PCollection.newBuilder().setUniqueName("input.out").build();
PCollection sideInput = PCollection.newBuilder().setUniqueName("sideInput.in").build();
@@ -104,7 +105,8 @@
Collections.singleton(userStateRef),
Collections.singleton(timerRef),
Collections.singleton(PipelineNode.pTransform("pt", pt)),
- Collections.singleton(PipelineNode.pCollection("output.out", output)));
+ Collections.singleton(PipelineNode.pCollection("output.out", output)),
+ DEFAULT_WIRE_CODER_SETTING);
PTransform stagePTransform = stage.toPTransform("foo");
assertThat(stagePTransform.getOutputsMap(), hasValue("output.out"));
@@ -130,9 +132,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform windowTransform =
PTransform.newBuilder()
@@ -143,9 +146,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
Components components =
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
index 87c0f72..0a32f9e 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
@@ -41,7 +41,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
@@ -117,9 +116,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("read.out", pc(name))
.putTransforms(
@@ -133,9 +133,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("parDo.out", pc("parDo.out"))
.putTransforms(
@@ -149,10 +150,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("window.out", pc("window.out"))
.build();
@@ -238,9 +239,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("read.out", pc("read.out"))
.putTransforms(
@@ -265,9 +267,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("parDo.out", pc("parDo.out"))
.build();
@@ -314,9 +317,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("read.out", pc("read.out"))
.putTransforms(
@@ -330,9 +334,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("go.out", pc("go.out"))
.putTransforms(
@@ -346,10 +351,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("py.out", pc("py.out"))
.build();
@@ -414,9 +419,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("pyRead.out", pc("pyRead.out"))
.putTransforms(
@@ -440,9 +446,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("goRead.out", pc("goRead.out"))
.putTransforms(
@@ -468,9 +475,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("pyParDo.out", pc("pyParDo.out"))
.putTransforms(
@@ -484,9 +492,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("goParDo.out", pc("goParDo.out"))
.putEnvironments("go", Environments.createDockerEnvironment("go"))
@@ -587,9 +596,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("pyRead.out", pc("pyRead.out"))
.putTransforms(
@@ -613,9 +623,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("goRead.out", pc("goRead.out"))
.putTransforms(
@@ -641,9 +652,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("goParDo.out", pc("goParDo.out"))
.putEnvironments("go", Environments.createDockerEnvironment("go"))
@@ -708,9 +720,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("read.out", pc("read.out"))
.putTransforms(
@@ -724,10 +737,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("py").build())
+ .setDoFn(FunctionSpec.newBuilder().build())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("parDo.out", pc("parDo.out"))
.build();
@@ -791,9 +804,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("read.out", pc("read.out"))
.putTransforms(
@@ -817,9 +831,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("sideRead.out", pc("sideRead.out"))
.putTransforms(
@@ -833,10 +848,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString())
.build())
+ .setEnvironmentId("py")
.build())
.putPcollections("leftParDo.out", pc("leftParDo.out"))
.putTransforms(
@@ -850,10 +866,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString())
.build())
+ .setEnvironmentId("py")
.build())
.putPcollections("rightParDo.out", pc("rightParDo.out"))
.putTransforms(
@@ -868,11 +885,12 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.putSideInputs("side", SideInput.getDefaultInstance())
.build()
.toByteString())
.build())
+ .setEnvironmentId("py")
.build())
.putPcollections("sideParDo.out", pc("sideParDo.out"))
.putEnvironments("py", Environments.createDockerEnvironment("py"))
@@ -930,9 +948,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform statefulTransform =
PTransform.newBuilder()
@@ -944,10 +963,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putStateSpecs("state", StateSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
Components components =
@@ -998,9 +1018,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform timerTransform =
PTransform.newBuilder()
@@ -1014,10 +1035,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putTimerSpecs("timer", TimerSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
Components components =
@@ -1067,11 +1089,12 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putStateSpecs("state", StateSpec.getDefaultInstance())
.putTimerSpecs("timer", TimerSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
Components components =
@@ -1122,9 +1145,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("read.out", pc("read.out"))
.putTransforms(
@@ -1138,9 +1162,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("go.out", pc("go.out"))
.putTransforms(
@@ -1154,10 +1179,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("py.out", pc("py.out"))
.putTransforms(
@@ -1216,9 +1241,10 @@
.setUrn(PTransformTranslation.FLATTEN_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build();
PTransform read1Transform =
@@ -1231,9 +1257,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build();
PTransform read2Transform =
PTransform.newBuilder()
@@ -1245,9 +1272,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build();
PTransform impulse1Transform =
@@ -1259,7 +1287,7 @@
.setUrn(PTransformTranslation.IMPULSE_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
.build();
@@ -1272,7 +1300,7 @@
.setUrn(PTransformTranslation.IMPULSE_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
.build();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
index af918d4..8dc0f1e 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
@@ -34,7 +34,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
@@ -123,10 +122,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("go")
.build())
.putPcollections("go.out", PCollection.newBuilder().setUniqueName("go.out").build())
.putTransforms(
@@ -139,10 +138,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("py")
.build())
.putPcollections("py.out", PCollection.newBuilder().setUniqueName("py.out").build())
.putEnvironments("go", Environments.createDockerEnvironment("go"))
@@ -205,9 +204,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform windowTransform =
PTransform.newBuilder()
@@ -218,9 +218,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
@@ -262,9 +263,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform statefulTransform =
PTransform.newBuilder()
@@ -275,10 +277,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putStateSpecs("state", StateSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
@@ -321,9 +324,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform timerTransform =
PTransform.newBuilder()
@@ -334,10 +338,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putTimerSpecs("timer", TimerSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
@@ -382,9 +387,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform parDoTransform =
PTransform.newBuilder()
@@ -395,9 +401,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform flattenTransform =
PTransform.newBuilder()
@@ -415,9 +422,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
@@ -464,9 +472,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform otherEnvRead =
PTransform.newBuilder()
@@ -477,9 +486,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("rare"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("rare")
.build();
PTransform flattenTransform =
PTransform.newBuilder()
@@ -497,9 +507,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
Components components =
@@ -561,10 +572,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString())
.build())
+ .setEnvironmentId("py")
.build();
PTransform goRead =
PTransform.newBuilder()
@@ -575,10 +587,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString())
.build())
+ .setEnvironmentId("go")
.build();
PTransform pyParDo =
@@ -590,10 +603,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("py"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString())
.build())
+ .setEnvironmentId("py")
.build();
PTransform goWindow =
PTransform.newBuilder()
@@ -604,10 +618,11 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(SdkFunctionSpec.newBuilder().setEnvironmentId("go"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString())
.build())
+ .setEnvironmentId("go")
.build();
PCollection flattenPc = PCollection.newBuilder().setUniqueName("flatten.out").build();
@@ -682,9 +697,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PCollection parDoOutput = PCollection.newBuilder().setUniqueName("parDo.out").build();
@@ -704,10 +720,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("rare"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("rare")
.build())
.putPcollections(
"window.out", PCollection.newBuilder().setUniqueName("window.out").build())
@@ -749,9 +765,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
QueryablePipeline.forPrimitivesIn(
@@ -770,10 +787,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build())
.putPcollections(
"parDo.out", PCollection.newBuilder().setUniqueName("parDo.out").build())
@@ -787,10 +804,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("rare"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("rare")
.build())
.putPcollections(
"window.out", PCollection.newBuilder().setUniqueName("window.out").build())
@@ -828,9 +845,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
@@ -863,11 +881,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putSideInputs("side_input", SideInput.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build())
.putPcollections(
"parDo.out", PCollection.newBuilder().setUniqueName("parDo.out").build())
@@ -881,10 +899,10 @@
.setUrn(PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN)
.setPayload(
WindowIntoPayload.newBuilder()
- .setWindowFn(
- SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setWindowFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build())
.putPcollections(
"window.out", PCollection.newBuilder().setUniqueName("window.out").build())
@@ -912,9 +930,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform parDoTransform =
@@ -928,10 +947,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putSideInputs("side_input", SideInput.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PCollection sideInputPCollection =
PCollection.newBuilder().setUniqueName("side_read.out").build();
@@ -996,9 +1016,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform processMain =
PTransform.newBuilder()
@@ -1010,10 +1031,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putSideInputs("side", SideInput.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PCollection sidePC = PCollection.newBuilder().setUniqueName("sidePC").build();
@@ -1051,9 +1073,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PTransform parDoTransform =
PTransform.newBuilder()
@@ -1064,10 +1087,11 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.putStateSpecs("state_spec", StateSpec.getDefaultInstance())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
PCollection userStateMainInputPCollection =
PCollection.newBuilder().setUniqueName("read.out").build();
@@ -1124,9 +1148,10 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(SdkFunctionSpec.newBuilder().setEnvironmentId("common"))
+ .setDoFn(FunctionSpec.newBuilder())
.build()
.toByteString()))
+ .setEnvironmentId("common")
.build();
QueryablePipeline p =
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
index fd034d8..578d506 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.construction.graph;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.equalTo;
@@ -58,7 +59,7 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(
ParDoPayload.newBuilder()
- .setDoFn(RunnerApi.SdkFunctionSpec.newBuilder().setEnvironmentId("foo"))
+ .setDoFn(RunnerApi.FunctionSpec.newBuilder())
.putSideInputs("side_input", RunnerApi.SideInput.getDefaultInstance())
.putStateSpecs("user_state", RunnerApi.StateSpec.getDefaultInstance())
.putTimerSpecs("timer", RunnerApi.TimerSpec.getDefaultInstance())
@@ -98,7 +99,8 @@
Collections.singleton(userStateRef),
Collections.singleton(timerRef),
Collections.singleton(PipelineNode.pTransform("pt", pt)),
- Collections.singleton(PipelineNode.pCollection("output.out", output)));
+ Collections.singleton(PipelineNode.pCollection("output.out", output)),
+ DEFAULT_WIRE_CODER_SETTING);
assertThat(stage.getComponents().containsTransforms("pt"), is(true));
assertThat(stage.getComponents().containsTransforms("other_pt"), is(false));
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
index daae554..8e83eef 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.construction.graph;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables.getOnlyElement;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.empty;
@@ -119,7 +120,8 @@
ImmutableList.of(),
ImmutableList.of(),
ImmutableList.of(PipelineNode.pTransform("one", one)),
- ImmutableList.of(PipelineNode.pCollection(oneOut.getUniqueName(), oneOut)));
+ ImmutableList.of(PipelineNode.pCollection(oneOut.getUniqueName(), oneOut)),
+ DEFAULT_WIRE_CODER_SETTING);
ExecutableStage twoStage =
ImmutableExecutableStage.of(
components,
@@ -129,7 +131,8 @@
ImmutableList.of(),
ImmutableList.of(),
ImmutableList.of(PipelineNode.pTransform("two", two)),
- ImmutableList.of(PipelineNode.pCollection(twoOut.getUniqueName(), twoOut)));
+ ImmutableList.of(PipelineNode.pCollection(twoOut.getUniqueName(), twoOut)),
+ DEFAULT_WIRE_CODER_SETTING);
PTransformNode redTransform = PipelineNode.pTransform("red", red);
PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
@@ -237,7 +240,8 @@
ImmutableList.of(),
ImmutableList.of(
PipelineNode.pTransform("one", one), PipelineNode.pTransform("shared", shared)),
- ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)));
+ ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
+ DEFAULT_WIRE_CODER_SETTING);
ExecutableStage twoStage =
ImmutableExecutableStage.of(
components,
@@ -248,7 +252,8 @@
ImmutableList.of(),
ImmutableList.of(
PipelineNode.pTransform("two", two), PipelineNode.pTransform("shared", shared)),
- ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)));
+ ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
+ DEFAULT_WIRE_CODER_SETTING);
PTransformNode redTransform = PipelineNode.pTransform("red", red);
PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
@@ -367,7 +372,8 @@
ImmutableList.of(),
ImmutableList.of(),
ImmutableList.of(PipelineNode.pTransform("one", one), sharedTransform),
- ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)));
+ ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
+ DEFAULT_WIRE_CODER_SETTING);
PTransformNode redTransform = PipelineNode.pTransform("red", red);
PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
@@ -540,7 +546,8 @@
PipelineNode.pTransform("otherShared", otherShared)),
ImmutableList.of(
PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut),
- PipelineNode.pCollection(otherSharedOut.getUniqueName(), otherSharedOut)));
+ PipelineNode.pCollection(otherSharedOut.getUniqueName(), otherSharedOut)),
+ DEFAULT_WIRE_CODER_SETTING);
ExecutableStage oneStage =
ImmutableExecutableStage.of(
components,
@@ -551,7 +558,8 @@
ImmutableList.of(),
ImmutableList.of(
PipelineNode.pTransform("one", one), PipelineNode.pTransform("shared", shared)),
- ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)));
+ ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
+ DEFAULT_WIRE_CODER_SETTING);
ExecutableStage twoStage =
ImmutableExecutableStage.of(
components,
@@ -564,7 +572,8 @@
PipelineNode.pTransform("two", two),
PipelineNode.pTransform("otherShared", otherShared)),
ImmutableList.of(
- PipelineNode.pCollection(otherSharedOut.getUniqueName(), otherSharedOut)));
+ PipelineNode.pCollection(otherSharedOut.getUniqueName(), otherSharedOut)),
+ DEFAULT_WIRE_CODER_SETTING);
PTransformNode redTransform = PipelineNode.pTransform("red", red);
PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java
index 63cea67..afd0472 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java
@@ -21,6 +21,7 @@
import static org.hamcrest.CoreMatchers.hasItem;
import static org.hamcrest.CoreMatchers.hasItems;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInRelativeOrder;
import static org.hamcrest.Matchers.not;
import static org.junit.Assert.assertFalse;
@@ -33,9 +34,7 @@
import java.util.List;
import java.util.jar.JarOutputStream;
import java.util.jar.Manifest;
-import java.util.stream.Collectors;
import org.apache.beam.sdk.testing.RestoreSystemProperties;
-import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -47,21 +46,14 @@
@Rule public transient RestoreSystemProperties systemProperties = new RestoreSystemProperties();
- private ClasspathScanningResourcesDetector detector;
-
- private ClassLoader classLoader;
-
- @Before
- public void setUp() {
- detector = new ClasspathScanningResourcesDetector(new ClassGraph());
- }
-
@Test
public void shouldDetectDirectories() throws Exception {
File folder = tmpFolder.newFolder("folder1");
- classLoader = new URLClassLoader(new URL[] {folder.toURI().toURL()});
+ ClassLoader classLoader = new URLClassLoader(new URL[] {folder.toURI().toURL()});
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
- List<String> result = detector.detect(classLoader).collect(Collectors.toList());
+ List<String> result = detector.detect(classLoader);
assertThat(result, hasItem(containsString(folder.getAbsolutePath())));
}
@@ -69,13 +61,33 @@
@Test
public void shouldDetectJarFiles() throws Exception {
File jarFile = createTestTmpJarFile("test");
- classLoader = new URLClassLoader(new URL[] {jarFile.toURI().toURL()});
+ ClassLoader classLoader = new URLClassLoader(new URL[] {jarFile.toURI().toURL()});
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
- List<String> result = detector.detect(classLoader).collect(Collectors.toList());
+ List<String> result = detector.detect(classLoader);
assertThat(result, hasItem(containsString(jarFile.getAbsolutePath())));
}
+ @Test
+ public void shouldDetectResourcesInOrderTheyAppearInURLClassLoader() throws Exception {
+ File file1 = createTestTmpJarFile("test1");
+ File file2 = createTestTmpJarFile("test2");
+ ClassLoader classLoader =
+ new URLClassLoader(new URL[] {file1.toURI().toURL(), file2.toURI().toURL()});
+
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
+
+ List<String> result = detector.detect(classLoader);
+
+ assertThat(
+ result,
+ containsInRelativeOrder(
+ containsString(file1.getAbsolutePath()), containsString(file2.getAbsolutePath())));
+ }
+
private File createTestTmpJarFile(String name) throws IOException {
File jarFile = tmpFolder.newFile(name);
try (JarOutputStream os = new JarOutputStream(new FileOutputStream(jarFile), new Manifest())) {}
@@ -85,9 +97,11 @@
@Test
public void shouldNotDetectOrdinaryFiles() throws Exception {
File textFile = tmpFolder.newFile("ordinaryTextFile.txt");
- classLoader = new URLClassLoader(new URL[] {textFile.toURI().toURL()});
+ ClassLoader classLoader = new URLClassLoader(new URL[] {textFile.toURI().toURL()});
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
- List<String> result = detector.detect(classLoader).collect(Collectors.toList());
+ List<String> result = detector.detect(classLoader);
assertThat(result, not(hasItem(containsString(textFile.getAbsolutePath()))));
}
@@ -96,8 +110,10 @@
public void shouldDetectClassPathResourceFromJavaClassPathEnvVariable() throws IOException {
String path = tmpFolder.newFolder("folder").getAbsolutePath();
System.setProperty("java.class.path", path);
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
- List<String> resources = detector.detect(null).collect(Collectors.toList());
+ List<String> resources = detector.detect(null);
assertThat(resources, hasItems(containsString(path)));
}
@@ -105,9 +121,11 @@
@Test
public void shouldNotDetectClassPathResourceThatIsNotAFile() throws Exception {
String url = "http://www.google.com/all-the-secrets.jar";
- classLoader = new URLClassLoader(new URL[] {new URL(url)});
+ ClassLoader classLoader = new URLClassLoader(new URL[] {new URL(url)});
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
- List<String> result = detector.detect(classLoader).collect(Collectors.toList());
+ List<String> result = detector.detect(classLoader);
assertThat(result, not(hasItem(containsString(url))));
}
@@ -120,9 +138,10 @@
@Test
public void shouldStillDetectResourcesEvenIfClassloaderIsUseless() {
ClassLoader uselessClassLoader = Mockito.mock(ClassLoader.class);
+ ClasspathScanningResourcesDetector detector =
+ new ClasspathScanningResourcesDetector(new ClassGraph());
- List<String> detectedResources =
- detector.detect(uselessClassLoader).collect(Collectors.toList());
+ List<String> detectedResources = detector.detect(uselessClassLoader);
assertFalse(detectedResources.isEmpty());
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
index 7b01c04..286e60b 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
@@ -103,11 +103,19 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- setTimer(TimerData.of(timerId, namespace, target, timeDomain));
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ setTimer(TimerData.of(timerId, timerFamilyId, namespace, target, outputTimestamp, timeDomain));
}
- /** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */
+ /**
+ * @deprecated use {@link #setTimer(StateNamespace, String, String, Instant, Instant,
+ * TimeDomain)}.
+ */
@Deprecated
@Override
public void setTimer(TimerData timerData) {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java
index 2949548..fc395de 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java
@@ -22,7 +22,7 @@
import java.io.OutputStream;
import java.util.List;
import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.TimerInternals.TimerDataCoder;
+import org.apache.beam.runners.core.TimerInternals.TimerDataCoderV2;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.IterableCoder;
@@ -54,7 +54,7 @@
this.keyCoder = keyCoder;
this.elemCoder = elemCoder;
this.windowCoder = windowCoder;
- this.timersCoder = IterableCoder.of(TimerDataCoder.of(windowCoder));
+ this.timersCoder = IterableCoder.of(TimerDataCoderV2.of(windowCoder));
this.elemsCoder = IterableCoder.of(FullWindowedValueCoder.of(elemCoder, windowCoder));
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index 2b105fb..472a9d2 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -992,7 +992,7 @@
* user has no way to compute a good choice of time.
*/
private void setUnderlyingTimer(Instant target) {
- timerInternals.setTimer(namespace, timerId, target, spec.getTimeDomain());
+ timerInternals.setTimer(namespace, timerId, "", target, target, spec.getTimeDomain());
}
private Instant getCurrentTime() {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java
index 14a9502..f69c74a 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java
@@ -208,7 +208,12 @@
// make sure this fires after any window.maxTimestamp() timers
gcTime = gcTime.plus(GC_DELAY_MS);
timerInternals.setTimer(
- StateNamespaces.window(windowCoder, window), GC_TIMER_ID, gcTime, TimeDomain.EVENT_TIME);
+ StateNamespaces.window(windowCoder, window),
+ GC_TIMER_ID,
+ "",
+ gcTime,
+ window.maxTimestamp(),
+ TimeDomain.EVENT_TIME);
}
@Override
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java
index a766143..f9f23ca 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java
@@ -54,9 +54,18 @@
*
* <p>It is an error to set a timer for two different time domains.
*/
- void setTimer(StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain);
+ void setTimer(
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain);
- /** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */
+ /**
+ * @deprecated use {@link #setTimer(StateNamespace, String, String, Instant, Instant,
+ * TimeDomain)}.
+ */
@Deprecated
void setTimer(TimerData timerData);
@@ -161,10 +170,19 @@
public abstract String getTimerId();
+ public abstract String getTimerFamilyId();
+
public abstract StateNamespace getNamespace();
public abstract Instant getTimestamp();
+ /**
+ * Timestamp the timer assigns to outputted elements from {@link
+ * org.apache.beam.sdk.transforms.DoFn.OnTimer} method. For event time timers, output watermark
+ * is held at this timestamp until the timer fires.
+ */
+ public abstract Instant getOutputTimestamp();
+
public abstract TimeDomain getDomain();
// When adding a new field, make sure to add it to the compareTo() method.
@@ -174,8 +192,25 @@
* generated.
*/
public static TimerData of(
+ String timerId,
+ String timerFamilyId,
+ StateNamespace namespace,
+ Instant timestamp,
+ Instant outputTimestamp,
+ TimeDomain domain) {
+ return new AutoValue_TimerInternals_TimerData(
+ timerId, timerFamilyId, namespace, timestamp, outputTimestamp, domain);
+ }
+
+ /**
+ * Construct a {@link TimerData} for the given parameters, where the timer ID is automatically
+ * generated. Construct a {@link TimerData} for the given parameters except for {@code
+ * outputTimestamp}. {@code outputTimestamp} is set to timer {@code timestamp}.
+ */
+ public static TimerData of(
String timerId, StateNamespace namespace, Instant timestamp, TimeDomain domain) {
- return new AutoValue_TimerInternals_TimerData(timerId, namespace, timestamp, domain);
+ return new AutoValue_TimerInternals_TimerData(
+ timerId, timerId, namespace, timestamp, timestamp, domain);
}
/**
@@ -207,8 +242,10 @@
ComparisonChain chain =
ComparisonChain.start()
.compare(this.getTimestamp(), that.getTimestamp())
+ .compare(this.getOutputTimestamp(), that.getOutputTimestamp())
.compare(this.getDomain(), that.getDomain())
- .compare(this.getTimerId(), that.getTimerId());
+ .compare(this.getTimerId(), that.getTimerId())
+ .compare(this.getTimerFamilyId(), that.getTimerFamilyId());
if (chain.result() == 0 && !this.getNamespace().equals(that.getNamespace())) {
// Obtaining the stringKey may be expensive; only do so if required
chain = chain.compare(getNamespace().stringKey(), that.getNamespace().stringKey());
@@ -218,6 +255,56 @@
}
/** A {@link Coder} for {@link TimerData}. */
+ class TimerDataCoderV2 extends StructuredCoder<TimerData> {
+ private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of();
+ private static final InstantCoder INSTANT_CODER = InstantCoder.of();
+ private final Coder<? extends BoundedWindow> windowCoder;
+
+ public static TimerDataCoderV2 of(Coder<? extends BoundedWindow> windowCoder) {
+ return new TimerDataCoderV2(windowCoder);
+ }
+
+ private TimerDataCoderV2(Coder<? extends BoundedWindow> windowCoder) {
+ this.windowCoder = windowCoder;
+ }
+
+ @Override
+ public void encode(TimerData timer, OutputStream outStream) throws CoderException, IOException {
+ STRING_CODER.encode(timer.getTimerId(), outStream);
+ STRING_CODER.encode(timer.getTimerFamilyId(), outStream);
+ STRING_CODER.encode(timer.getNamespace().stringKey(), outStream);
+ INSTANT_CODER.encode(timer.getTimestamp(), outStream);
+ INSTANT_CODER.encode(timer.getOutputTimestamp(), outStream);
+ STRING_CODER.encode(timer.getDomain().name(), outStream);
+ }
+
+ @Override
+ public TimerData decode(InputStream inStream) throws CoderException, IOException {
+ String timerId = STRING_CODER.decode(inStream);
+ String timerFamilyId = STRING_CODER.decode(inStream);
+ StateNamespace namespace =
+ StateNamespaces.fromString(STRING_CODER.decode(inStream), windowCoder);
+ Instant timestamp = INSTANT_CODER.decode(inStream);
+ Instant outputTimestamp = INSTANT_CODER.decode(inStream);
+ TimeDomain domain = TimeDomain.valueOf(STRING_CODER.decode(inStream));
+ return TimerData.of(timerId, timerFamilyId, namespace, timestamp, outputTimestamp, domain);
+ }
+
+ @Override
+ public List<? extends Coder<?>> getCoderArguments() {
+ return Arrays.asList(windowCoder);
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ verifyDeterministic(this, "window coder must be deterministic", windowCoder);
+ }
+ }
+
+ /**
+ * A {@link Coder} for {@link TimerData}. To make it encoding and decoding backward compatible for
+ * DataFlow
+ */
class TimerDataCoder extends StructuredCoder<TimerData> {
private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of();
private static final InstantCoder INSTANT_CODER = InstantCoder.of();
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
index 6be35b7..dd106329 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
@@ -71,8 +71,8 @@
Instant laterTimestamp = new Instant(42);
underTest.advanceInputWatermark(new Instant(0));
- underTest.setTimer(NS1, ID1, earlyTimestamp, TimeDomain.EVENT_TIME);
- underTest.setTimer(NS1, ID1, laterTimestamp, TimeDomain.EVENT_TIME);
+ underTest.setTimer(NS1, ID1, ID1, earlyTimestamp, earlyTimestamp, TimeDomain.EVENT_TIME);
+ underTest.setTimer(NS1, ID1, ID1, laterTimestamp, laterTimestamp, TimeDomain.EVENT_TIME);
underTest.advanceInputWatermark(earlyTimestamp.plus(1L));
assertThat(underTest.removeNextEventTimer(), nullValue());
@@ -86,7 +86,7 @@
public void testDeletionIdempotent() throws Exception {
InMemoryTimerInternals underTest = new InMemoryTimerInternals();
Instant timestamp = new Instant(42);
- underTest.setTimer(NS1, ID1, timestamp, TimeDomain.EVENT_TIME);
+ underTest.setTimer(NS1, ID1, ID1, timestamp, timestamp, TimeDomain.EVENT_TIME);
underTest.deleteTimer(NS1, ID1);
underTest.deleteTimer(NS1, ID1);
}
@@ -97,7 +97,7 @@
Instant timestamp = new Instant(42);
underTest.advanceInputWatermark(new Instant(0));
- underTest.setTimer(NS1, ID1, timestamp, TimeDomain.EVENT_TIME);
+ underTest.setTimer(NS1, ID1, ID1, timestamp, timestamp, TimeDomain.EVENT_TIME);
underTest.deleteTimer(NS1, ID1);
underTest.advanceInputWatermark(new Instant(43));
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
index b790314..10972d6 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
@@ -156,6 +156,8 @@
.setTimer(
StateNamespaces.window(new GlobalWindows().windowCoder(), GlobalWindow.INSTANCE),
DoFnWithTimers.TIMER_ID,
+ "",
+ currentTime.plus(DoFnWithTimers.TIMER_OFFSET),
currentTime.plus(DoFnWithTimers.TIMER_OFFSET),
TimeDomain.EVENT_TIME);
}
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java
index d1e5221..ab2978f 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java
@@ -22,7 +22,7 @@
import static org.junit.Assert.assertThat;
import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.TimerInternals.TimerDataCoder;
+import org.apache.beam.runners.core.TimerInternals.TimerDataCoderV2;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.state.TimeDomain;
import org.apache.beam.sdk.testing.CoderProperties;
@@ -40,13 +40,13 @@
@Test
public void testTimerDataCoder() throws Exception {
CoderProperties.coderDecodeEncodeEqual(
- TimerDataCoder.of(GlobalWindow.Coder.INSTANCE),
+ TimerDataCoderV2.of(GlobalWindow.Coder.INSTANCE),
TimerData.of(
"arbitrary-id", StateNamespaces.global(), new Instant(0), TimeDomain.EVENT_TIME));
Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
CoderProperties.coderDecodeEncodeEqual(
- TimerDataCoder.of(windowCoder),
+ TimerDataCoderV2.of(windowCoder),
TimerData.of(
"another-id",
StateNamespaces.window(
@@ -57,7 +57,7 @@
@Test
public void testCoderIsSerializableWithWellKnownCoderType() {
- CoderProperties.coderSerializable(TimerDataCoder.of(GlobalWindow.Coder.INSTANCE));
+ CoderProperties.coderSerializable(TimerDataCoderV2.of(GlobalWindow.Coder.INSTANCE));
}
@Test
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
index 8f3ab48..0261bf6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
@@ -47,11 +47,20 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, timeDomain));
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ timerUpdateBuilder.setTimer(
+ TimerData.of(timerId, timerFamilyId, namespace, target, outputTimestamp, timeDomain));
}
- /** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */
+ /**
+ * @deprecated use {@link #setTimer(StateNamespace, String, String, Instant, Instant,
+ * TimeDomain)}.
+ */
@Deprecated
@Override
public void setTimer(TimerData timerData) {
diff --git a/runners/flink/1.7/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/1.7/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
index e29f97e..807faf5 100644
--- a/runners/flink/1.7/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/1.7/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -20,6 +20,8 @@
import java.io.EOFException;
import java.io.IOException;
import java.util.Objects;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
import org.apache.beam.sdk.coders.Coder;
@@ -40,9 +42,26 @@
private final Coder<T> coder;
+ /**
+ * {@link SerializablePipelineOptions} deserialization will cause {@link
+ * org.apache.beam.sdk.io.FileSystems} registration needed for {@link
+ * org.apache.beam.sdk.transforms.Reshuffle} translation.
+ */
+ @SuppressWarnings("unused")
+ @Nullable
+ private final SerializablePipelineOptions pipelineOptions;
+
public CoderTypeSerializer(Coder<T> coder) {
Preconditions.checkNotNull(coder);
this.coder = coder;
+ this.pipelineOptions = null;
+ }
+
+ public CoderTypeSerializer(
+ Coder<T> coder, @Nullable SerializablePipelineOptions pipelineOptions) {
+ Preconditions.checkNotNull(coder);
+ this.coder = coder;
+ this.pipelineOptions = pipelineOptions;
}
@Override
diff --git a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
index 2ff1cda..276e49c 100644
--- a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -19,6 +19,8 @@
import java.io.EOFException;
import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
import org.apache.beam.sdk.coders.Coder;
@@ -41,9 +43,26 @@
private final Coder<T> coder;
+ /**
+ * {@link SerializablePipelineOptions} deserialization will cause {@link
+ * org.apache.beam.sdk.io.FileSystems} registration needed for {@link
+ * org.apache.beam.sdk.transforms.Reshuffle} translation.
+ */
+ @SuppressWarnings("unused")
+ @Nullable
+ private final SerializablePipelineOptions pipelineOptions;
+
public CoderTypeSerializer(Coder<T> coder) {
Preconditions.checkNotNull(coder);
this.coder = coder;
+ this.pipelineOptions = null;
+ }
+
+ public CoderTypeSerializer(
+ Coder<T> coder, @Nullable SerializablePipelineOptions pipelineOptions) {
+ Preconditions.checkNotNull(coder);
+ this.coder = coder;
+ this.pipelineOptions = pipelineOptions;
}
@Override
diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle
index c7f8cc0..1c617d8 100644
--- a/runners/flink/flink_runner.gradle
+++ b/runners/flink/flink_runner.gradle
@@ -128,6 +128,7 @@
configurations {
validatesRunner
+ miniCluster
}
dependencies {
@@ -170,6 +171,7 @@
validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest")
validatesRunner project(path: ":runners:core-java", configuration: "testRuntime")
validatesRunner project(project.path)
+ miniCluster "org.apache.flink:flink-runtime-web_2.11:$flink_version"
}
class ValidatesRunnerConfig {
@@ -223,3 +225,29 @@
// Generates :runners:flink:1.9:runQuickstartJavaFlinkLocal
createJavaExamplesArchetypeValidationTask(type: 'Quickstart', runner: 'FlinkLocal')
+
+/**
+ * Updates the documentation with the current pipeline options.
+ */
+def createPipelineOptionsTableTask(String target) {
+ tasks.register("generatePipelineOptionsTable${target}", JavaExec) {
+ group = 'Website'
+ description = "Generates a table with pipeline options for the Flink Runner documentation page"
+ classpath = sourceSets.test.runtimeClasspath
+ main = 'org.apache.beam.runners.flink.website.PipelineOptionsTableGenerator'
+ args = [target]
+ standardOutput = new ByteArrayOutputStream()
+ doLast {
+ def dest = file("${project(':website').getProjectDir()}/src/_includes/flink_${target.toLowerCase()}_pipeline_options.html")
+ if (!dest.exists()) {
+ throw new GradleException("Pipeline options file is not in expected location: ${dest}")
+ }
+ dest.write(standardOutput.toString())
+ }
+ }
+}
+createPipelineOptionsTableTask('Java')
+createPipelineOptionsTableTask('Python')
+// Update the pipeline options documentation before running the tests
+test.dependsOn(generatePipelineOptionsTableJava)
+test.dependsOn(generatePipelineOptionsTablePython)
diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle
index f0cae9d..27a116f 100644
--- a/runners/flink/job-server/flink_job_server.gradle
+++ b/runners/flink/job-server/flink_job_server.gradle
@@ -181,20 +181,76 @@
]
)
-task testPipelineJar() {
- dependsOn shadowJar
- dependsOn ":sdks:python:container:py35:docker"
- doLast{
- exec {
- executable "sh"
- def options = [
- "--flink_job_server_jar ${shadowJar.archivePath}",
- "--env_dir ${project.rootProject.buildDir}/gradleenv/${project.path.hashCode()}",
- "--python_root_dir ${project.rootDir}/sdks/python",
- "--python_version 3.5",
- "--python_container_image apachebeam/python3.5_sdk:${project['python_sdk_version']}",
- ]
- args "-c", "../../job-server/test_pipeline_jar.sh ${options.join(' ')}"
+def addTestJavaJarCreator(String pyVersion) {
+ def pyBuildPath = pyVersion.startsWith("2") ? "2" : pyVersion.replaceAll("\\.", "")
+ project.tasks.create(name: "testJavaJarCreatorPy${pyBuildPath}") {
+ dependsOn shadowJar
+ dependsOn ":sdks:python:container:py${pyBuildPath}:docker"
+ doLast{
+ exec {
+ executable "sh"
+ def options = [
+ "--flink_job_server_jar ${shadowJar.archivePath}",
+ "--env_dir ${project.rootProject.buildDir}/gradleenv/${project.path.hashCode()}",
+ "--python_root_dir ${project.rootDir}/sdks/python",
+ "--python_version ${pyVersion}",
+ "--python_container_image apachebeam/python${pyVersion}_sdk:${project['python_sdk_version']}",
+ ]
+ args "-c", "../../job-server/test_pipeline_jar.sh ${options.join(' ')}"
+ }
}
}
}
+
+// miniCluster jar starts an embedded Flink cluster intended for use in testing.
+task miniCluster(type: Jar, dependsOn: shadowJar) {
+ archiveBaseName = "${project.archivesBaseName}-mini-cluster"
+ dependencies {
+ runtime project(path: flinkRunnerProject, configuration: "miniCluster")
+ }
+ from zipTree(shadowJar.archivePath).matching {
+ // If these classes aren't excluded from the mini cluster jar, they will be loaded instead of
+ // the corresponding classes in the submitted job jar, preventing pipeline resources from
+ // loading successfully.
+ exclude "**/FlinkPipelineRunner*"
+ exclude "**/PortablePipelineJarUtils*"
+ }
+ manifest {
+ attributes('Main-Class': 'org.apache.beam.runners.flink.FlinkMiniClusterEntryPoint')
+ }
+ zip64 true // jar needs to contain more than 65535 files
+}
+
+def addTestFlinkUberJarPy(String pyVersion) {
+ def pyBuildPath = pyVersion.startsWith("2") ? "2" : pyVersion.replaceAll("\\.", "")
+ project.tasks.create(name: "testFlinkUberJarPy${pyBuildPath}") {
+ dependsOn miniCluster
+ dependsOn shadowJar
+ dependsOn ":sdks:python:container:py${pyBuildPath}:docker"
+ doLast{
+ exec {
+ executable "sh"
+ def options = [
+ "--flink_job_server_jar ${shadowJar.archivePath}",
+ "--flink_mini_cluster_jar ${miniCluster.archivePath}",
+ "--env_dir ${project.rootProject.buildDir}/gradleenv/${project.path.hashCode()}",
+ "--python_root_dir ${project.rootDir}/sdks/python",
+ "--python_version ${pyVersion}",
+ "--python_container_image apachebeam/python${pyVersion}_sdk:${project['python_sdk_version']}",
+ ]
+ args "-c", "../../job-server/test_flink_uber_jar.sh ${options.join(' ')}"
+ }
+ }
+ }
+}
+
+["2.7", "3.5", "3.6", "3.7"].each{ pyVersion ->
+ addTestJavaJarCreator(pyVersion)
+ addTestFlinkUberJarPy(pyVersion)
+}
+
+task testPipelineJar() {
+ dependsOn testJavaJarCreatorPy37
+ dependsOn testFlinkUberJarPy36
+ dependsOn testFlinkUberJarPy37
+}
diff --git a/runners/flink/job-server/test_flink_uber_jar.sh b/runners/flink/job-server/test_flink_uber_jar.sh
new file mode 100755
index 0000000..f24c32a
--- /dev/null
+++ b/runners/flink/job-server/test_flink_uber_jar.sh
@@ -0,0 +1,141 @@
+#!/bin/bash
+#
+# 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.
+#
+
+# test_flink_uber_jar.sh tests the Python FlinkRunner class.
+
+set -e
+set -v
+
+while [[ $# -gt 0 ]]
+do
+key="$1"
+case $key in
+ --flink_job_server_jar)
+ FLINK_JOB_SERVER_JAR="$2"
+ shift # past argument
+ shift # past value
+ ;;
+ --flink_mini_cluster_jar)
+ FLINK_MINI_CLUSTER_JAR="$2"
+ shift # past argument
+ shift # past value
+ ;;
+ --env_dir)
+ ENV_DIR="$2"
+ shift # past argument
+ shift # past value
+ ;;
+ --python_root_dir)
+ PYTHON_ROOT_DIR="$2"
+ shift # past argument
+ shift # past value
+ ;;
+ --python_version)
+ PYTHON_VERSION="$2"
+ shift # past argument
+ shift # past value
+ ;;
+ --python_container_image)
+ PYTHON_CONTAINER_IMAGE="$2"
+ shift # past argument
+ shift # past value
+ ;;
+ *) # unknown option
+ echo "Unknown option: $1"
+ exit 1
+ ;;
+esac
+done
+
+
+# Go to the root of the repository
+cd "$(git rev-parse --show-toplevel)"
+
+# Verify docker command exists
+command -v docker
+docker -v
+
+# Verify container has already been built
+docker images --format "{{.Repository}}:{{.Tag}}" | grep "$PYTHON_CONTAINER_IMAGE"
+
+# Set up Python environment
+virtualenv -p "python$PYTHON_VERSION" "$ENV_DIR"
+. $ENV_DIR/bin/activate
+pip install --retries 10 -e "$PYTHON_ROOT_DIR"
+
+# Hacky python script to find a free port. Note there is a small chance the chosen port could
+# get taken before being claimed.
+SOCKET_SCRIPT="
+from __future__ import print_function
+import socket
+s = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
+s.bind(('localhost', 0))
+print(s.getsockname()[1])
+s.close()
+"
+FLINK_PORT=$(python -c "$SOCKET_SCRIPT")
+
+echo "Starting Flink mini cluster listening on port $FLINK_PORT"
+java -jar "$FLINK_MINI_CLUSTER_JAR" --rest-port "$FLINK_PORT" &
+
+PIPELINE_PY="
+import apache_beam as beam
+from apache_beam.options.pipeline_options import PipelineOptions
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+from apache_beam.transforms import Create
+from apache_beam.transforms import Map
+
+# To test that our main session is getting plumbed through artifact staging
+# correctly, create a global variable. If the main session is not plumbed
+# through properly, global_var will be undefined and the pipeline will fail.
+global_var = 1
+
+pipeline_options = PipelineOptions()
+pipeline_options.view_as(SetupOptions).save_main_session = True
+pipeline = beam.Pipeline(options=pipeline_options)
+pcoll = (pipeline
+ | Create([0, 1, 2])
+ | Map(lambda x: x + global_var))
+assert_that(pcoll, equal_to([1, 2, 3]))
+
+result = pipeline.run()
+result.wait_until_finish()
+"
+
+(python -c "$PIPELINE_PY" \
+ --runner FlinkRunner \
+ --flink_job_server_jar "$FLINK_JOB_SERVER_JAR" \
+ --parallelism 1 \
+ --environment_type DOCKER \
+ --environment_config "$PYTHON_CONTAINER_IMAGE" \
+ --flink_master "localhost:$FLINK_PORT" \
+ --flink_submit_uber_jar \
+) || TEST_EXIT_CODE=$? # don't fail fast here; clean up before exiting
+
+kill %1 || echo "Failed to shut down Flink mini cluster"
+
+rm -rf "$ENV_DIR"
+
+if [[ "$TEST_EXIT_CODE" -eq 0 ]]; then
+ echo ">>> SUCCESS"
+else
+ echo ">>> FAILURE"
+fi
+exit $TEST_EXIT_CODE
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
index 229eca5..28351d5 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
@@ -34,6 +34,7 @@
import org.apache.beam.runners.core.construction.ReadTranslation;
import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkIdentityFunction;
import org.apache.beam.runners.flink.translation.functions.FlinkMergingNonShuffleReduceFunction;
import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction;
import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
@@ -84,6 +85,7 @@
import org.apache.flink.api.java.operators.GroupCombineOperator;
import org.apache.flink.api.java.operators.GroupReduceOperator;
import org.apache.flink.api.java.operators.Grouping;
+import org.apache.flink.api.java.operators.MapOperator;
import org.apache.flink.api.java.operators.MapPartitionOperator;
import org.apache.flink.api.java.operators.SingleInputUdfOperator;
@@ -306,11 +308,25 @@
@Override
public void translateNode(
Reshuffle<K, InputT> transform, FlinkBatchTranslationContext context) {
-
- DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
+ final DataSet<WindowedValue<KV<K, InputT>>> inputDataSet =
context.getInputDataSet(context.getInput(transform));
-
- context.setOutputDataSet(context.getOutput(transform), inputDataSet.rebalance());
+ // Construct an instance of CoderTypeInformation which contains the pipeline options.
+ // This will be used to initialized FileSystems.
+ @SuppressWarnings("unchecked")
+ final CoderTypeInformation<WindowedValue<KV<K, InputT>>> outputType =
+ ((CoderTypeInformation) inputDataSet.getType())
+ .withPipelineOptions(context.getPipelineOptions());
+ // We insert a NOOP here to initialize the FileSystems via the above CoderTypeInformation.
+ // The output type coder may be relying on file system access. The shuffled data may have to
+ // be deserialized on a different machine using this coder where FileSystems has not been
+ // initialized.
+ final DataSet<WindowedValue<KV<K, InputT>>> retypedDataSet =
+ new MapOperator<>(
+ inputDataSet,
+ outputType,
+ FlinkIdentityFunction.of(),
+ getCurrentTransformName(context));
+ context.setOutputDataSet(context.getOutput(transform), retypedDataSet.rebalance());
}
}
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkMiniClusterEntryPoint.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkMiniClusterEntryPoint.java
new file mode 100644
index 0000000..fec9a88
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkMiniClusterEntryPoint.java
@@ -0,0 +1,91 @@
+/*
+ * 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.runners.flink;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.kohsuke.args4j.CmdLineException;
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Entry point for starting an embedded Flink cluster. */
+public class FlinkMiniClusterEntryPoint {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkMiniClusterEntryPoint.class);
+
+ static class MiniClusterArgs {
+ @Option(name = "--rest-port")
+ int restPort = 0;
+
+ @Option(name = "--num-task-managers")
+ int numTaskManagers = 1;
+
+ @Option(name = "--num-task-slots-per-taskmanager")
+ int numSlotsPerTaskManager = 1;
+ }
+
+ public static void main(String[] args) throws Exception {
+ MiniClusterArgs miniClusterArgs = parseArgs(args);
+
+ Configuration flinkConfig = new Configuration();
+ flinkConfig.setInteger(RestOptions.PORT, miniClusterArgs.restPort);
+
+ MiniClusterConfiguration clusterConfig =
+ new MiniClusterConfiguration.Builder()
+ .setConfiguration(flinkConfig)
+ .setNumTaskManagers(miniClusterArgs.numTaskManagers)
+ .setNumSlotsPerTaskManager(miniClusterArgs.numSlotsPerTaskManager)
+ .build();
+
+ try (MiniCluster miniCluster = new MiniCluster(clusterConfig)) {
+ miniCluster.start();
+ System.out.println(
+ String.format(
+ "Started Flink mini cluster (%s TaskManagers with %s task slots) with Rest API at %s",
+ miniClusterArgs.numTaskManagers,
+ miniClusterArgs.numSlotsPerTaskManager,
+ miniCluster.getRestAddress()));
+ Thread.sleep(Long.MAX_VALUE);
+ }
+ }
+
+ private static MiniClusterArgs parseArgs(String[] args) {
+ MiniClusterArgs configuration = new MiniClusterArgs();
+ CmdLineParser parser = new CmdLineParser(configuration);
+ try {
+ parser.parseArgument(args);
+ } catch (CmdLineException e) {
+ LOG.error("Unable to parse command line arguments.", e);
+ printUsage(parser);
+ throw new IllegalArgumentException("Unable to parse command line arguments.", e);
+ }
+ return configuration;
+ }
+
+ private static void printUsage(CmdLineParser parser) {
+ System.err.println(
+ String.format(
+ "Usage: java %s arguments...", FlinkMiniClusterEntryPoint.class.getSimpleName()));
+ parser.printUsage(System.err);
+ System.err.println();
+ }
+}
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index add6da3..d2244bc 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -153,20 +153,11 @@
void setStateBackendFactory(Class<? extends FlinkStateBackendFactory> stateBackendFactory);
- @Description("Enable/disable Beam metrics in Flink Runner")
- @Default.Boolean(true)
- Boolean getEnableMetrics();
-
- void setEnableMetrics(Boolean enableMetrics);
-
- @Description(
- "By default, uses Flink accumulators to store the metrics which allows to query metrics from the PipelineResult. "
- + "If set to true, metrics will still be reported but can't be queried via PipelineResult. "
- + "This saves network and memory.")
+ @Description("Disable Beam metrics in Flink Runner")
@Default.Boolean(false)
- Boolean getDisableMetricAccumulator();
+ Boolean getDisableMetrics();
- void setDisableMetricAccumulator(Boolean disableMetricAccumulator);
+ void setDisableMetrics(Boolean enableMetrics);
/** Enables or disables externalized checkpoints. */
@Description(
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
index 9690da4..e389be5 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainer.java
@@ -61,42 +61,43 @@
private static final String METRIC_KEY_SEPARATOR =
GlobalConfiguration.loadConfiguration().getString(MetricOptions.SCOPE_DELIMITER);
+ private final MetricsContainerStepMap metricsContainers;
private final RuntimeContext runtimeContext;
private final Map<String, Counter> flinkCounterCache;
private final Map<String, FlinkDistributionGauge> flinkDistributionGaugeCache;
private final Map<String, FlinkGauge> flinkGaugeCache;
- private final MetricsAccumulator metricsAccumulator;
- public FlinkMetricContainer(RuntimeContext runtimeContext, boolean accumulatorDisabled) {
+ public FlinkMetricContainer(RuntimeContext runtimeContext) {
this.runtimeContext = runtimeContext;
this.flinkCounterCache = new HashMap<>();
this.flinkDistributionGaugeCache = new HashMap<>();
this.flinkGaugeCache = new HashMap<>();
-
- Accumulator<MetricsContainerStepMap, MetricsContainerStepMap> metricsAccumulator;
- if (accumulatorDisabled) {
- // Do not register the accumulator with Flink
- metricsAccumulator = new MetricsAccumulator();
- } else {
- metricsAccumulator = runtimeContext.getAccumulator(ACCUMULATOR_NAME);
- if (metricsAccumulator == null) {
- metricsAccumulator = new MetricsAccumulator();
- try {
- runtimeContext.addAccumulator(ACCUMULATOR_NAME, metricsAccumulator);
- } catch (UnsupportedOperationException e) {
- // Not supported in all environments, e.g. tests
- } catch (Exception e) {
- LOG.error("Failed to create metrics accumulator.", e);
- }
- }
- }
- this.metricsAccumulator = (MetricsAccumulator) metricsAccumulator;
+ this.metricsContainers = new MetricsContainerStepMap();
}
public MetricsContainerImpl getMetricsContainer(String stepName) {
- return metricsAccumulator != null
- ? metricsAccumulator.getLocalValue().getContainer(stepName)
- : null;
+ return metricsContainers.getContainer(stepName);
+ }
+
+ /**
+ * This should be called at the end of the Flink job and sets up an accumulator to push the
+ * metrics to the PipelineResult. This should not be called beforehand, to avoid the overhead
+ * which accumulators cause at runtime.
+ */
+ public void registerMetricsForPipelineResult() {
+ Accumulator<MetricsContainerStepMap, MetricsContainerStepMap> metricsAccumulator =
+ runtimeContext.getAccumulator(ACCUMULATOR_NAME);
+ if (metricsAccumulator == null) {
+ metricsAccumulator = new MetricsAccumulator();
+ try {
+ runtimeContext.addAccumulator(ACCUMULATOR_NAME, metricsAccumulator);
+ } catch (UnsupportedOperationException e) {
+ // Not supported in all environments, e.g. tests
+ } catch (Exception e) {
+ LOG.error("Failed to create metrics accumulator.", e);
+ }
+ }
+ metricsAccumulator.add(metricsContainers);
}
/**
@@ -113,7 +114,7 @@
* given step.
*/
void updateMetrics(String stepName) {
- MetricResults metricResults = asAttemptedOnlyMetricResults(metricsAccumulator.getLocalValue());
+ MetricResults metricResults = asAttemptedOnlyMetricResults(metricsContainers);
MetricQueryResults metricQueryResults =
metricResults.queryMetrics(MetricsFilter.builder().addStep(stepName).build());
updateCounters(metricQueryResults.getCounters());
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/MetricsAccumulator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/MetricsAccumulator.java
index 09966c0..d4f2c48 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/MetricsAccumulator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/MetricsAccumulator.java
@@ -21,7 +21,11 @@
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.accumulators.SimpleAccumulator;
-/** Accumulator of {@link MetricsContainerStepMap}. */
+/**
+ * Accumulator of {@link MetricsContainerStepMap}. This accumulator will only be reported to Flink
+ * when the job ends. This avoids the runtime overhead for accumulators which are continously sent
+ * to the job manager.
+ */
public class MetricsAccumulator implements SimpleAccumulator<MetricsContainerStepMap> {
private MetricsContainerStepMap metricsContainers = new MetricsContainerStepMap();
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java
index 995bacc..736a2dd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/ReaderInvocationUtil.java
@@ -40,7 +40,7 @@
String stepName, PipelineOptions options, FlinkMetricContainer container) {
FlinkPipelineOptions flinkPipelineOptions = options.as(FlinkPipelineOptions.class);
this.stepName = stepName;
- enableMetrics = flinkPipelineOptions.getEnableMetrics();
+ this.enableMetrics = !flinkPipelineOptions.getDisableMetrics();
this.container = container;
}
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
index 01699b6..fa6867f 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -72,6 +72,7 @@
private final Map<String, PCollectionView<?>> sideInputMapping;
private transient DoFnInvoker<InputT, OutputT> doFnInvoker;
+ private transient FlinkMetricContainer metricContainer;
public FlinkDoFnFunction(
DoFn<InputT, OutputT> doFn,
@@ -132,13 +133,8 @@
sideInputMapping);
FlinkPipelineOptions pipelineOptions = serializedOptions.get().as(FlinkPipelineOptions.class);
- if (pipelineOptions.getEnableMetrics()) {
- doFnRunner =
- new DoFnRunnerWithMetricsUpdate<>(
- stepName,
- doFnRunner,
- new FlinkMetricContainer(
- getRuntimeContext(), pipelineOptions.getDisableMetricAccumulator()));
+ if (!pipelineOptions.getDisableMetrics()) {
+ doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, metricContainer);
}
doFnRunner.startBundle();
@@ -157,11 +153,13 @@
// options where they are needed.
FileSystems.setDefaultPipelineOptions(serializedOptions.get());
doFnInvoker = DoFnInvokers.tryInvokeSetupFor(doFn);
+ metricContainer = new FlinkMetricContainer(getRuntimeContext());
}
@Override
public void close() throws Exception {
try {
+ metricContainer.registerMetricsForPipelineResult();
Optional.ofNullable(doFnInvoker).ifPresent(DoFnInvoker::invokeTeardown);
} finally {
FlinkClassloading.deleteStaticCaches();
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
index c1260eb..4e1c709 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
@@ -96,7 +96,7 @@
// Worker-local fields. These should only be constructed and consumed on Flink TaskManagers.
private transient RuntimeContext runtimeContext;
- private transient FlinkMetricContainer container;
+ private transient FlinkMetricContainer metricContainer;
private transient StateRequestHandler stateRequestHandler;
private transient ExecutableStageContext stageContext;
private transient StageBundleFactory stageBundleFactory;
@@ -131,7 +131,7 @@
FileSystems.setDefaultPipelineOptions(options);
executableStage = ExecutableStage.fromPayload(stagePayload);
runtimeContext = getRuntimeContext();
- container = new FlinkMetricContainer(runtimeContext, options.getDisableMetricAccumulator());
+ metricContainer = new FlinkMetricContainer(runtimeContext);
// TODO: Wire this into the distributed cache and make it pluggable.
stageContext = contextFactory.get(jobInfo);
stageBundleFactory = stageContext.getStageBundleFactory(executableStage);
@@ -145,12 +145,12 @@
new BundleProgressHandler() {
@Override
public void onProgress(ProcessBundleProgressResponse progress) {
- container.updateMetrics(stepName, progress.getMonitoringInfosList());
+ metricContainer.updateMetrics(stepName, progress.getMonitoringInfosList());
}
@Override
public void onCompleted(ProcessBundleResponse response) {
- container.updateMetrics(stepName, response.getMonitoringInfosList());
+ metricContainer.updateMetrics(stepName, response.getMonitoringInfosList());
}
};
}
@@ -283,6 +283,7 @@
@Override
public void close() throws Exception {
+ metricContainer.registerMetricsForPipelineResult();
// close may be called multiple times when an exception is thrown
if (stageContext != null) {
try (AutoCloseable bundleFactoryCloser = stageBundleFactory;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkIdentityFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkIdentityFunction.java
new file mode 100644
index 0000000..f9128e7
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkIdentityFunction.java
@@ -0,0 +1,42 @@
+/*
+ * 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.runners.flink.translation.functions;
+
+import org.apache.flink.api.common.functions.MapFunction;
+
+/**
+ * A map function that outputs the input element without any change.
+ *
+ * @param <T> Input element type.
+ */
+public class FlinkIdentityFunction<T> implements MapFunction<T, T> {
+
+ private static final FlinkIdentityFunction<?> INSTANCE = new FlinkIdentityFunction<>();
+
+ @SuppressWarnings("unchecked")
+ public static <T> FlinkIdentityFunction<T> of() {
+ return (FlinkIdentityFunction) INSTANCE;
+ }
+
+ private FlinkIdentityFunction() {}
+
+ @Override
+ public T map(T value) {
+ return value;
+ }
+}
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
index 829e063..6aee09f 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
@@ -72,7 +72,9 @@
private final Map<TupleTag<?>, Coder<?>> outputCoderMap;
private final DoFnSchemaInformation doFnSchemaInformation;
private final Map<String, PCollectionView<?>> sideInputMapping;
+
private transient DoFnInvoker doFnInvoker;
+ private transient FlinkMetricContainer metricContainer;
public FlinkStatefulDoFnFunction(
DoFn<KV<K, V>, OutputT> dofn,
@@ -158,13 +160,8 @@
sideInputMapping);
FlinkPipelineOptions pipelineOptions = serializedOptions.get().as(FlinkPipelineOptions.class);
- if (pipelineOptions.getEnableMetrics()) {
- doFnRunner =
- new DoFnRunnerWithMetricsUpdate<>(
- stepName,
- doFnRunner,
- new FlinkMetricContainer(
- getRuntimeContext(), pipelineOptions.getDisableMetricAccumulator()));
+ if (!pipelineOptions.getDisableMetrics()) {
+ doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, metricContainer);
}
doFnRunner.startBundle();
@@ -227,12 +224,14 @@
// deserialization method. However, this is a hack, and we want to properly initialize the
// options where they are needed.
FileSystems.setDefaultPipelineOptions(serializedOptions.get());
+ metricContainer = new FlinkMetricContainer(getRuntimeContext());
doFnInvoker = DoFnInvokers.tryInvokeSetupFor(dofn);
}
@Override
public void close() throws Exception {
try {
+ metricContainer.registerMetricsForPipelineResult();
Optional.ofNullable(doFnInvoker).ifPresent(DoFnInvoker::invokeTeardown);
} finally {
FlinkClassloading.deleteStaticCaches();
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
index c03bef9..5e76923 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
@@ -19,7 +19,10 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.AtomicType;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -33,10 +36,18 @@
public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
private final Coder<T> coder;
+ @Nullable private final SerializablePipelineOptions pipelineOptions;
public CoderTypeInformation(Coder<T> coder) {
checkNotNull(coder);
this.coder = coder;
+ this.pipelineOptions = null;
+ }
+
+ private CoderTypeInformation(Coder<T> coder, PipelineOptions pipelineOptions) {
+ checkNotNull(coder);
+ this.coder = coder;
+ this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions);
}
public Coder<T> getCoder() {
@@ -70,9 +81,8 @@
}
@Override
- @SuppressWarnings("unchecked")
public TypeSerializer<T> createSerializer(ExecutionConfig config) {
- return new CoderTypeSerializer<>(coder);
+ return new CoderTypeSerializer<>(coder, pipelineOptions);
}
@Override
@@ -80,6 +90,18 @@
return 2;
}
+ /**
+ * Creates a new {@link CoderTypeInformation} with {@link PipelineOptions}, that can be used for
+ * {@link org.apache.beam.sdk.io.FileSystems} registration.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/BEAM-8577">Jira issue.</a>
+ * @param pipelineOptions Options of current pipeline.
+ * @return New type information.
+ */
+ public CoderTypeInformation<T> withPipelineOptions(PipelineOptions pipelineOptions) {
+ return new CoderTypeInformation<>(getCoder(), pipelineOptions);
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
index 850c9c5..b22fd88 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
@@ -20,7 +20,6 @@
import java.io.IOException;
import java.util.List;
import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil;
import org.apache.beam.sdk.io.BoundedSource;
@@ -53,6 +52,7 @@
private boolean inputAvailable = false;
private transient ReaderInvocationUtil<T, BoundedSource.BoundedReader<T>> readerInvoker;
+ private transient FlinkMetricContainer metricContainer;
public SourceInputFormat(
String stepName, BoundedSource<T> initialSource, PipelineOptions options) {
@@ -68,10 +68,7 @@
@Override
public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
- FlinkMetricContainer metricContainer =
- new FlinkMetricContainer(
- getRuntimeContext(),
- options.as(FlinkPipelineOptions.class).getDisableMetricAccumulator());
+ metricContainer = new FlinkMetricContainer(getRuntimeContext());
readerInvoker = new ReaderInvocationUtil<>(stepName, serializedOptions.get(), metricContainer);
@@ -149,6 +146,7 @@
@Override
public void close() throws IOException {
+ metricContainer.registerMetricsForPipelineResult();
// TODO null check can be removed once FLINK-3796 is fixed
if (reader != null) {
reader.close();
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index e474daa..8e63679 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -171,7 +171,7 @@
final KeySelector<WindowedValue<InputT>, ?> keySelector;
- private final TimerInternals.TimerDataCoder timerCoder;
+ private final TimerInternals.TimerDataCoderV2 timerCoder;
/** Max number of elements to include in a bundle. */
private final long maxBundleSize;
@@ -244,7 +244,7 @@
this.keySelector = keySelector;
this.timerCoder =
- TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder());
+ TimerInternals.TimerDataCoderV2.of(windowingStrategy.getWindowFn().windowCoder());
FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class);
@@ -438,9 +438,8 @@
}
doFnRunner = createWrappingDoFnRunner(doFnRunner);
- if (options.getEnableMetrics()) {
- flinkMetricContainer =
- new FlinkMetricContainer(getRuntimeContext(), options.getDisableMetricAccumulator());
+ if (!options.getDisableMetrics()) {
+ flinkMetricContainer = new FlinkMetricContainer(getRuntimeContext());
doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, flinkMetricContainer);
}
@@ -481,6 +480,7 @@
@Override
public void close() throws Exception {
try {
+ flinkMetricContainer.registerMetricsForPipelineResult();
// This is our last change to block shutdown of this operator while
// there are still remaining processing-time timers. Flink will ignore pending
// processing-time timers when upstream operators have shut down and will also
@@ -1088,11 +1088,20 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- setTimer(TimerData.of(timerId, namespace, target, timeDomain));
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ setTimer(
+ TimerData.of(timerId, timerFamilyId, namespace, target, outputTimestamp, timeDomain));
}
- /** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */
+ /**
+ * @deprecated use {@link #setTimer(StateNamespace, String, String, Instant, Instant,
+ * TimeDomain)}.
+ */
@Deprecated
@Override
public void setTimer(TimerData timer) {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
index ba951b1..891cef8 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
@@ -839,7 +839,9 @@
timerInternals.setTimer(
StateNamespaces.window(windowCoder, window),
GC_TIMER_ID,
+ "",
gcTime,
+ window.maxTimestamp(),
TimeDomain.EVENT_TIME);
} finally {
stateBackendLock.unlock();
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index 57eed1a..2539687 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -136,6 +136,9 @@
/** false if checkpointCoder is null or no restore state by starting first. */
private transient boolean isRestored = false;
+ /** Metrics container which will be reported as Flink accumulators at the end of the job. */
+ private transient FlinkMetricContainer metricContainer;
+
@SuppressWarnings("unchecked")
public UnboundedSourceWrapper(
String stepName,
@@ -177,6 +180,7 @@
public void open(Configuration parameters) throws Exception {
FileSystems.setDefaultPipelineOptions(serializedOptions.get());
runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
+ metricContainer = new FlinkMetricContainer(runtimeContext);
// figure out which split sources we're responsible for
int subtaskIndex = runtimeContext.getIndexOfThisSubtask();
@@ -220,10 +224,6 @@
context = ctx;
- FlinkPipelineOptions options = serializedOptions.get().as(FlinkPipelineOptions.class);
- FlinkMetricContainer metricContainer =
- new FlinkMetricContainer(getRuntimeContext(), options.getDisableMetricAccumulator());
-
ReaderInvocationUtil<OutputT, UnboundedSource.UnboundedReader<OutputT>> readerInvoker =
new ReaderInvocationUtil<>(stepName, serializedOptions.get(), metricContainer);
@@ -349,6 +349,7 @@
@Override
public void close() throws Exception {
+ metricContainer.registerMetricsForPipelineResult();
try {
super.close();
if (localReaders != null) {
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java
index 8246125..c0d6ceb 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java
@@ -91,8 +91,7 @@
assertThat(options.getExecutionModeForBatch(), is(ExecutionMode.PIPELINED.name()));
assertThat(options.getSavepointPath(), is(nullValue()));
assertThat(options.getAllowNonRestoredState(), is(false));
- assertThat(options.getEnableMetrics(), is(true));
- assertThat(options.getDisableMetricAccumulator(), is(false));
+ assertThat(options.getDisableMetrics(), is(false));
}
@Test(expected = Exception.class)
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
index afbf46b..a85ad65 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
@@ -21,10 +21,8 @@
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertNotNull;
-import static org.junit.runners.Parameterized.Parameter;
-import static org.junit.runners.Parameterized.Parameters;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.anyObject;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.argThat;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.verify;
@@ -59,28 +57,18 @@
import org.apache.flink.metrics.SimpleCounter;
import org.junit.Before;
import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
import org.mockito.ArgumentMatcher;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
/** Tests for {@link FlinkMetricContainer}. */
-@RunWith(Parameterized.class)
public class FlinkMetricContainerTest {
- @Parameter public boolean useMetricAccumulator;
-
@Mock private RuntimeContext runtimeContext;
@Mock private MetricGroup metricGroup;
FlinkMetricContainer container;
- @Parameters(name = "useMetricAccumulator: {0}")
- public static Object[] data() {
- return new Object[] {true, false};
- }
-
@Before
public void beforeTest() {
MockitoAnnotations.initMocks(this);
@@ -88,7 +76,7 @@
anyString()))
.thenReturn(new MetricsAccumulator());
when(runtimeContext.getMetricGroup()).thenReturn(metricGroup);
- container = new FlinkMetricContainer(runtimeContext, !useMetricAccumulator);
+ container = new FlinkMetricContainer(runtimeContext);
}
@Test
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java
new file mode 100644
index 0000000..779f9c1
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.runners.flink.translation.functions;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnSchemaInformation;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.powermock.reflect.Whitebox;
+
+/** Tests for {@link FlinkDoFnFunction}. */
+public class FlinkDoFnFunctionTest {
+
+ @Test
+ public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
+ FlinkDoFnFunction doFnFunction =
+ new TestDoFnFunction(
+ "step",
+ WindowingStrategy.globalDefault(),
+ Collections.emptyMap(),
+ PipelineOptionsFactory.create(),
+ Collections.emptyMap(),
+ new TupleTag<>(),
+ null,
+ Collections.emptyMap(),
+ DoFnSchemaInformation.create(),
+ Collections.emptyMap());
+
+ doFnFunction.open(new Configuration());
+
+ String metricContainerFieldName = "metricContainer";
+ FlinkMetricContainer monitoredContainer =
+ Mockito.spy(
+ (FlinkMetricContainer)
+ Whitebox.getInternalState(doFnFunction, metricContainerFieldName));
+ Whitebox.setInternalState(doFnFunction, metricContainerFieldName, monitoredContainer);
+
+ doFnFunction.close();
+ Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
+ }
+
+ private static class TestDoFnFunction extends FlinkDoFnFunction {
+
+ public TestDoFnFunction(
+ String stepName,
+ WindowingStrategy windowingStrategy,
+ Map sideInputs,
+ PipelineOptions options,
+ Map outputMap,
+ TupleTag mainOutputTag,
+ Coder inputCoder,
+ Map outputCoderMap,
+ DoFnSchemaInformation doFnSchemaInformation,
+ Map sideInputMapping) {
+ super(
+ new IdentityFn(),
+ stepName,
+ windowingStrategy,
+ sideInputs,
+ options,
+ outputMap,
+ mainOutputTag,
+ inputCoder,
+ outputCoderMap,
+ doFnSchemaInformation,
+ sideInputMapping);
+ }
+
+ @Override
+ public RuntimeContext getRuntimeContext() {
+ return Mockito.mock(RuntimeContext.class);
+ }
+
+ private static class IdentityFn<T> extends DoFn<T, T> {
+ @ProcessElement
+ public void processElement(ProcessContext c) {
+ c.output(c.element());
+ }
+ }
+ }
+}
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
index 93f7cd2..61d8906 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
@@ -32,6 +32,7 @@
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
+import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
import org.apache.beam.runners.fnexecution.control.BundleProgressHandler;
import org.apache.beam.runners.fnexecution.control.ExecutableStageContext;
import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory;
@@ -197,6 +198,11 @@
}
@Override
+ public void split(double fractionOfRemainder) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public void close() throws Exception {
if (once) {
return;
@@ -248,6 +254,21 @@
verifyNoMoreInteractions(stageBundleFactory);
}
+ @Test
+ public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
+ FlinkExecutableStageFunction<Integer> function = getFunction(Collections.emptyMap());
+ function.open(new Configuration());
+
+ String metricContainerFieldName = "metricContainer";
+ FlinkMetricContainer monitoredContainer =
+ Mockito.spy(
+ (FlinkMetricContainer) Whitebox.getInternalState(function, metricContainerFieldName));
+ Whitebox.setInternalState(function, metricContainerFieldName, monitoredContainer);
+
+ function.close();
+ Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
+ }
+
/**
* Creates a {@link FlinkExecutableStageFunction}. Sets the runtime context to {@link
* #runtimeContext}. The context factory is mocked to return {@link #stageContext} every time. The
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java
new file mode 100644
index 0000000..4f9707c
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.runners.flink.translation.functions;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnSchemaInformation;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.powermock.reflect.Whitebox;
+
+/** Tests for {@link FlinkStatefulDoFnFunction}. */
+public class FlinkStatefulDoFnFunctionTest {
+
+ @Test
+ public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
+ FlinkStatefulDoFnFunction doFnFunction =
+ new TestDoFnFunction(
+ "step",
+ WindowingStrategy.globalDefault(),
+ Collections.emptyMap(),
+ PipelineOptionsFactory.create(),
+ Collections.emptyMap(),
+ new TupleTag<>(),
+ null,
+ Collections.emptyMap(),
+ DoFnSchemaInformation.create(),
+ Collections.emptyMap());
+
+ doFnFunction.open(new Configuration());
+
+ String metricContainerFieldName = "metricContainer";
+ FlinkMetricContainer monitoredContainer =
+ Mockito.spy(
+ (FlinkMetricContainer)
+ Whitebox.getInternalState(doFnFunction, metricContainerFieldName));
+ Whitebox.setInternalState(doFnFunction, metricContainerFieldName, monitoredContainer);
+
+ doFnFunction.close();
+ Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
+ }
+
+ private static class TestDoFnFunction extends FlinkStatefulDoFnFunction {
+
+ public TestDoFnFunction(
+ String stepName,
+ WindowingStrategy windowingStrategy,
+ Map sideInputs,
+ PipelineOptions options,
+ Map outputMap,
+ TupleTag mainOutputTag,
+ Coder inputCoder,
+ Map outputCoderMap,
+ DoFnSchemaInformation doFnSchemaInformation,
+ Map sideInputMapping) {
+ super(
+ new IdentityFn(),
+ stepName,
+ windowingStrategy,
+ sideInputs,
+ options,
+ outputMap,
+ mainOutputTag,
+ inputCoder,
+ outputCoderMap,
+ doFnSchemaInformation,
+ sideInputMapping);
+ }
+
+ @Override
+ public RuntimeContext getRuntimeContext() {
+ return Mockito.mock(RuntimeContext.class);
+ }
+
+ private static class IdentityFn<T> extends DoFn<T, T> {
+ @ProcessElement
+ public void processElement(ProcessContext c) {
+ c.output(c.element());
+ }
+ }
+ }
+}
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormatTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormatTest.java
new file mode 100644
index 0000000..4e8affd
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormatTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.runners.flink.translation.wrappers;
+
+import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.powermock.reflect.Whitebox;
+
+/** Tests for {@link SourceInputFormat}. */
+public class SourceInputFormatTest {
+
+ @Test
+ public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
+ SourceInputFormat<Long> sourceInputFormat =
+ new TestSourceInputFormat<>(
+ "step", CountingSource.upTo(10), PipelineOptionsFactory.create());
+
+ sourceInputFormat.open(sourceInputFormat.createInputSplits(1)[0]);
+
+ String metricContainerFieldName = "metricContainer";
+ FlinkMetricContainer monitoredContainer =
+ Mockito.spy(
+ (FlinkMetricContainer)
+ Whitebox.getInternalState(sourceInputFormat, metricContainerFieldName));
+ Whitebox.setInternalState(sourceInputFormat, metricContainerFieldName, monitoredContainer);
+
+ sourceInputFormat.close();
+ Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
+ }
+
+ private static class TestSourceInputFormat<T> extends SourceInputFormat<T> {
+
+ public TestSourceInputFormat(
+ String stepName, BoundedSource initialSource, PipelineOptions options) {
+ super(stepName, initialSource, options);
+ }
+
+ @Override
+ public RuntimeContext getRuntimeContext() {
+ return Mockito.mock(RuntimeContext.class);
+ }
+ }
+}
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
index 220ffc9..235a2e3 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
@@ -39,6 +39,7 @@
import javax.annotation.Nullable;
import org.apache.beam.runners.core.StatefulDoFnRunner;
import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
import org.apache.beam.sdk.Pipeline;
@@ -94,6 +95,7 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
import org.powermock.reflect.Whitebox;
/** Tests for {@link DoFnOperator}. */
@@ -1874,13 +1876,43 @@
assertThrows(Error.class, () -> testHarness.snapshot(0, 0));
}
+ @Test
+ public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
+ DoFnOperator doFnOperator = getOperatorForCleanupInspection();
+ OneInputStreamOperatorTestHarness<WindowedValue<String>, WindowedValue<String>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(doFnOperator);
+
+ testHarness.open();
+
+ String metricContainerFieldName = "flinkMetricContainer";
+ FlinkMetricContainer monitoredContainer =
+ Mockito.spy(
+ (FlinkMetricContainer)
+ Whitebox.getInternalState(doFnOperator, metricContainerFieldName));
+ Whitebox.setInternalState(doFnOperator, metricContainerFieldName, monitoredContainer);
+
+ testHarness.close();
+ Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
+ }
+
/**
* Ensures Jackson cache is cleaned to get rid of any references to the Flink Classloader. See
* https://jira.apache.org/jira/browse/BEAM-6460
*/
@Test
public void testRemoveCachedClassReferences() throws Exception {
+ OneInputStreamOperatorTestHarness<WindowedValue<String>, WindowedValue<String>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(getOperatorForCleanupInspection());
+ LRUMap typeCache =
+ (LRUMap) Whitebox.getInternalState(TypeFactory.defaultInstance(), "_typeCache");
+ assertThat(typeCache.size(), greaterThan(0));
+ testHarness.open();
+ testHarness.close();
+ assertThat(typeCache.size(), is(0));
+ }
+
+ private static DoFnOperator getOperatorForCleanupInspection() {
FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
options.setParallelism(4);
@@ -1901,34 +1933,23 @@
outputTag,
WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE));
- DoFnOperator<String, String> doFnOperator =
- new DoFnOperator<>(
- doFn,
- "stepName",
- windowedValueCoder,
- null,
- Collections.emptyMap(),
- outputTag,
- Collections.emptyList(),
- outputManagerFactory,
- WindowingStrategy.globalDefault(),
- new HashMap<>(), /* side-input mapping */
- Collections.emptyList(), /* side inputs */
- options,
- null,
- null,
- DoFnSchemaInformation.create(),
- Collections.emptyMap());
-
- OneInputStreamOperatorTestHarness<WindowedValue<String>, WindowedValue<String>> testHarness =
- new OneInputStreamOperatorTestHarness<>(doFnOperator);
-
- LRUMap typeCache =
- (LRUMap) Whitebox.getInternalState(TypeFactory.defaultInstance(), "_typeCache");
- assertThat(typeCache.size(), greaterThan(0));
- testHarness.open();
- testHarness.close();
- assertThat(typeCache.size(), is(0));
+ return new DoFnOperator<>(
+ doFn,
+ "stepName",
+ windowedValueCoder,
+ null,
+ Collections.emptyMap(),
+ outputTag,
+ Collections.emptyList(),
+ outputManagerFactory,
+ WindowingStrategy.globalDefault(),
+ new HashMap<>(), /* side-input mapping */
+ Collections.emptyList(), /* side inputs */
+ options,
+ null,
+ null,
+ DoFnSchemaInformation.create(),
+ Collections.emptyMap());
}
private Iterable<WindowedValue<String>> stripStreamRecord(Iterable<?> input) {
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
index 9f7eff4..7fdec35 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
@@ -29,8 +29,8 @@
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.verify;
@@ -315,6 +315,11 @@
}
@Override
+ public void split(double fractionOfRemainder) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public void close() throws Exception {
if (onceEmitted) {
return;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java
index 7b0f9b8..5a04f7e 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java
@@ -36,6 +36,7 @@
import java.util.stream.LongStream;
import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource;
import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
import org.apache.beam.runners.flink.streaming.StreamSources;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.io.CountingSource;
@@ -71,6 +72,7 @@
import org.junit.runners.JUnit4;
import org.junit.runners.Parameterized;
import org.mockito.Mockito;
+import org.powermock.reflect.Whitebox;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -775,6 +777,34 @@
.boxed()
.toArray()));
}
+
+ @Test
+ public void testAccumulatorRegistrationOnOperatorClose() throws Exception {
+ FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+
+ TestCountingSource source = new TestCountingSource(20).withoutSplitting();
+
+ UnboundedSourceWrapper<KV<Integer, Integer>, TestCountingSource.CounterMark> sourceWrapper =
+ new UnboundedSourceWrapper<>("noReader", options, source, 2);
+
+ StreamingRuntimeContext mock = Mockito.mock(StreamingRuntimeContext.class);
+ Mockito.when(mock.getNumberOfParallelSubtasks()).thenReturn(1);
+ Mockito.when(mock.getExecutionConfig()).thenReturn(new ExecutionConfig());
+ Mockito.when(mock.getIndexOfThisSubtask()).thenReturn(0);
+ sourceWrapper.setRuntimeContext(mock);
+
+ sourceWrapper.open(new Configuration());
+
+ String metricContainerFieldName = "metricContainer";
+ FlinkMetricContainer monitoredContainer =
+ Mockito.spy(
+ (FlinkMetricContainer)
+ Whitebox.getInternalState(sourceWrapper, metricContainerFieldName));
+ Whitebox.setInternalState(sourceWrapper, metricContainerFieldName, monitoredContainer);
+
+ sourceWrapper.close();
+ Mockito.verify(monitoredContainer).registerMetricsForPipelineResult();
+ }
}
private static final class TestStreamStatusMaintainer implements StreamStatusMaintainer {
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/website/PipelineOptionsTableGenerator.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/website/PipelineOptionsTableGenerator.java
new file mode 100644
index 0000000..8ca490e
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/website/PipelineOptionsTableGenerator.java
@@ -0,0 +1,179 @@
+/*
+ * 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.runners.flink.website;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.CaseFormat;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/**
+ * A main class which is called by the Gradle generatePipelineOptionsTable* tasks to update the list
+ * of available pipeline options for the Beam website.
+ */
+public class PipelineOptionsTableGenerator {
+
+ private static final List<String> supportedLanguages = ImmutableList.of("java", "python");
+
+ private static class Option {
+ String name;
+ String description;
+ @Nullable String defaultValue;
+
+ public Option(String name, String description, @Nullable String defaultValue) {
+ this.name = name;
+ this.description = description;
+ this.defaultValue = defaultValue;
+ }
+ }
+
+ public static void main(String[] args) {
+ if (args.length != 1) {
+ throw new RuntimeException(
+ "Please specify the language (" + supportedLanguages + ") as the only argument.");
+ }
+ String arg = args[0].toLowerCase();
+ if (!supportedLanguages.contains(arg)) {
+ throw new RuntimeException("The language is not supported: " + arg);
+ }
+ boolean isPython = arg.equals("python");
+
+ printHeader();
+ List<Option> options = extractOptions(isPython);
+ printOptionsTable(options);
+ }
+
+ private static void printHeader() {
+ System.out.println(
+ "<!--\n"
+ + "Licensed under the Apache License, Version 2.0 (the \"License\");\n"
+ + "you may not use this file except in compliance with the License.\n"
+ + "You may obtain a copy of the License at\n"
+ + "\n"
+ + "http://www.apache.org/licenses/LICENSE-2.0\n"
+ + "\n"
+ + "Unless required by applicable law or agreed to in writing, software\n"
+ + "distributed under the License is distributed on an \"AS IS\" BASIS,\n"
+ + "WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n"
+ + "See the License for the specific language governing permissions and\n"
+ + "limitations under the License.\n"
+ + "-->");
+ System.out.println(
+ "<!--\n"
+ + "This is an auto-generated file.\n"
+ + "Use generatePipelineOptionsTableJava and generatePipelineOptionsTablePython respectively\n"
+ + "which should be called before running the tests.\n"
+ + "-->");
+ }
+
+ /**
+ * Returns the extracted list of options via reflections on FlinkPipelineOptions. Options are
+ * returned sorted in alphabetical order since Java does not guarantee any consistent order on the
+ * class methods.
+ */
+ private static List<Option> extractOptions(boolean isPython) {
+ List<Option> options = new ArrayList<>();
+ for (Method method : FlinkPipelineOptions.class.getDeclaredMethods()) {
+ String name;
+ String description;
+ String defaultValue = null;
+ name = method.getName();
+ if (name.matches("^(get|is).*")) {
+ name = name.replaceFirst("^(get|is)", "");
+
+ if (isPython) {
+ name = CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_UNDERSCORE, name);
+ } else {
+ name = Character.toLowerCase(name.charAt(0)) + name.substring(1);
+ }
+
+ Description descriptionAnnotation = method.getAnnotation(Description.class);
+ if (descriptionAnnotation == null) {
+ throw new RuntimeException(
+ "All pipeline options should have a description. Please add one for " + name);
+ }
+ description = descriptionAnnotation.value();
+
+ Optional<String> defaultValueFromAnnotation = getDefaultValueFromAnnotation(method);
+ if (defaultValueFromAnnotation.isPresent()) {
+ defaultValue = defaultValueFromAnnotation.get();
+ }
+
+ options.add(new Option(name, description, defaultValue));
+ }
+ }
+ options.sort(Comparator.comparing(option -> option.name));
+ return options;
+ }
+
+ private static void printOptionsTable(List<Option> options) {
+ System.out.println("<table class=\"table table-bordered\">");
+ for (Option option : options) {
+ System.out.println("<tr>");
+ System.out.println(" <td><code>" + option.name + "</code></td>");
+ System.out.println(" <td>" + option.description + "</td>");
+ if (option.defaultValue != null) {
+ System.out.println(" <td>Default: <code>" + option.defaultValue + "</code></td>");
+ } else {
+ System.out.println(" <td></td>");
+ }
+ System.out.println("</tr>");
+ }
+ System.out.println("</table>");
+ }
+
+ /** Returns a string representation of the {@link Default} value on the passed in method. */
+ private static Optional<String> getDefaultValueFromAnnotation(Method method) {
+ for (Annotation annotation : method.getAnnotations()) {
+ if (annotation instanceof Default.Class) {
+ return Optional.of(((Default.Class) annotation).value().getSimpleName());
+ } else if (annotation instanceof Default.String) {
+ return Optional.of(((Default.String) annotation).value());
+ } else if (annotation instanceof Default.Boolean) {
+ return Optional.of(Boolean.toString(((Default.Boolean) annotation).value()));
+ } else if (annotation instanceof Default.Character) {
+ return Optional.of(Character.toString(((Default.Character) annotation).value()));
+ } else if (annotation instanceof Default.Byte) {
+ return Optional.of(Byte.toString(((Default.Byte) annotation).value()));
+ } else if (annotation instanceof Default.Short) {
+ return Optional.of(Short.toString(((Default.Short) annotation).value()));
+ } else if (annotation instanceof Default.Integer) {
+ return Optional.of(Integer.toString(((Default.Integer) annotation).value()));
+ } else if (annotation instanceof Default.Long) {
+ return Optional.of(Long.toString(((Default.Long) annotation).value()));
+ } else if (annotation instanceof Default.Float) {
+ return Optional.of(Float.toString(((Default.Float) annotation).value()));
+ } else if (annotation instanceof Default.Double) {
+ return Optional.of(Double.toString(((Default.Double) annotation).value()));
+ } else if (annotation instanceof Default.Enum) {
+ return Optional.of(((Default.Enum) annotation).value());
+ } else if (annotation instanceof Default.InstanceFactory) {
+ return Optional.of(((Default.InstanceFactory) annotation).value().getSimpleName());
+ }
+ }
+ return Optional.empty();
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle
index 1a01351..d8ddffa 100644
--- a/runners/google-cloud-dataflow-java/build.gradle
+++ b/runners/google-cloud-dataflow-java/build.gradle
@@ -39,7 +39,7 @@
filter org.apache.tools.ant.filters.ReplaceTokens, tokens: [
'dataflow.legacy_environment_major_version' : '7',
'dataflow.fnapi_environment_major_version' : '7',
- 'dataflow.container_version' : 'beam-master-20190829'
+ 'dataflow.container_version' : 'beam-master-20191226'
]
}
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
index 390f0f0..4db0e3d 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
@@ -193,7 +193,7 @@
return ParDoTranslation.payloadForParDoLike(
new ParDoTranslation.ParDoLike() {
@Override
- public RunnerApi.SdkFunctionSpec translateDoFn(SdkComponents newComponents) {
+ public RunnerApi.FunctionSpec translateDoFn(SdkComponents newComponents) {
return ParDoTranslation.translateDoFn(
parDo.getFn(),
parDo.getMainOutputTag(),
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java
index e166a22..0e8eb91 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java
@@ -59,7 +59,7 @@
base,
SCHEMA,
StringUtils.byteArrayToJsonString(
- SchemaTranslation.schemaToProto(target.getSchema()).toByteArray()));
+ SchemaTranslation.schemaToProto(target.getSchema(), true).toByteArray()));
return base;
}
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
index 0d2907d..1f2a8cc 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
@@ -23,8 +23,8 @@
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.mock;
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index 4438e14..e965947 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -38,10 +38,10 @@
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.Assume.assumeFalse;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyListOf;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isA;
import static org.mockito.Mockito.mock;
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java
index 75d4166..f86136f 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java
@@ -24,8 +24,8 @@
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
index 4013d18..69e87ee 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
@@ -27,9 +27,9 @@
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyListOf;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle
index 4aba6eb..6b866d7 100644
--- a/runners/google-cloud-dataflow-java/worker/build.gradle
+++ b/runners/google-cloud-dataflow-java/worker/build.gradle
@@ -104,6 +104,7 @@
shadowTest project(path: ":runners:direct-java", configuration: "shadow")
shadowTest library.java.hamcrest_core
shadowTest library.java.hamcrest_library
+ shadowTest library.java.jsonassert
shadowTest library.java.junit
shadowTest library.java.mockito_core
}
diff --git a/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle b/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle
index 6dc99f2..5ee2d63 100644
--- a/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle
+++ b/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle
@@ -78,6 +78,7 @@
library.java.hamcrest_core, // Test only
library.java.hamcrest_library, // Test only
library.java.junit, // Test only
+ library.java.jsonassert // Test only
]
applyJavaNature(
@@ -224,6 +225,7 @@
shadowTest project(path: ":sdks:java:core", configuration: "shadowTest")
shadowTest library.java.hamcrest_core
shadowTest library.java.hamcrest_library
+ shadowTest library.java.jsonassert
shadowTest library.java.junit
shadowTest library.java.mockito_core
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java
index 71228c5..d184364 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleEntryReader.java
@@ -44,9 +44,9 @@
new ChunkingShuffleBatchReader(executionContext, operationContext, applianceShuffleReader);
if (cache) {
- // Limit the size of the cache.
- final int maxBatches = 32;
- batchReader = new CachingShuffleBatchReader(batchReader, maxBatches);
+ // Limit the size of the cache to ~32 full shuffle batches.
+ final long maxBytes = 128L * 1024 * 1024;
+ batchReader = new CachingShuffleBatchReader(batchReader, maxBytes);
}
entryReader = new BatchingShuffleEntryReader(batchReader);
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java
index 7f25c81..8939fa7 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ChunkingShuffleBatchReader.java
@@ -59,14 +59,18 @@
}
DataInputStream input = new DataInputStream(new ByteArrayInputStream(result.chunk));
ArrayList<ShuffleEntry> entries = new ArrayList<>();
+ long batchSize = 0;
while (input.available() > 0) {
- entries.add(getShuffleEntry(input));
+ ShuffleEntry entry = getShuffleEntry(input);
+ batchSize += entry.length();
+ entries.add(entry);
}
return new Batch(
entries,
result.nextStartPosition == null
? null
- : ByteArrayShufflePosition.of(result.nextStartPosition));
+ : ByteArrayShufflePosition.of(result.nextStartPosition),
+ batchSize);
}
/**
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
index a990c38..5449462 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.dataflow.worker;
+import com.google.common.base.Strings;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
@@ -41,6 +42,7 @@
private static final String CONTROL_API_SERVICE_DESCRIPTOR = "CONTROL_API_SERVICE_DESCRIPTOR";
private static final String LOGGING_API_SERVICE_DESCRIPTOR = "LOGGING_API_SERVICE_DESCRIPTOR";
+ private static final String STATUS_API_SERVICE_DESCRIPTOR = "STATUS_API_SERVICE_DESCRIPTOR";
private static final String ROOT_LOGGER_NAME = "";
private static final String PIPELINE_PATH = "PIPELINE_PATH";
@@ -113,6 +115,18 @@
return parseApiServiceDescriptorFromText(System.getenv().get(CONTROL_API_SERVICE_DESCRIPTOR));
}
+ @Nullable
+ public static Endpoints.ApiServiceDescriptor getStatusDescriptor()
+ throws TextFormat.ParseException {
+ String statusApiDescriptor = System.getenv().get(STATUS_API_SERVICE_DESCRIPTOR);
+ if (Strings.isNullOrEmpty(statusApiDescriptor)) {
+ // Missing STATUS_API_SERVICE_DESCRIPTOR env var is a signal that the worker status API
+ // is unsupported by the current runner.
+ return null;
+ }
+ return parseApiServiceDescriptorFromText(statusApiDescriptor);
+ }
+
// TODO: make env logic private to main() so it is never done outside of initializing the process
public static @Nullable RunnerApi.Pipeline getPipelineFromEnv() throws IOException {
String pipelinePath = System.getenv(PIPELINE_PATH);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
index 6c1ddde..4ba2487 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
@@ -75,7 +75,7 @@
Coder<WindowedValue<T>> inputCoder,
WindowingStrategy<?, W> windowingStrategy,
DataflowExecutionContext.DataflowStepContext stepContext,
- Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec> pCollectionViewToWindowMappingFns) {
+ Map<PCollectionView<?>, RunnerApi.FunctionSpec> pCollectionViewToWindowMappingFns) {
super(receivers, context);
this.sideInputFetcher =
@@ -167,9 +167,9 @@
FnDataService beamFnDataService,
ApiServiceDescriptor dataServiceApiServiceDescriptor,
Coder<BoundedWindow> mainInputWindowCoder,
- Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec> pCollectionViewsToWindowMappingFns) {
+ Map<PCollectionView<?>, RunnerApi.FunctionSpec> pCollectionViewsToWindowMappingFns) {
ImmutableList.Builder<PCollectionView<?>> wrappedViews = ImmutableList.builder();
- for (Map.Entry<PCollectionView<?>, RunnerApi.SdkFunctionSpec> entry :
+ for (Map.Entry<PCollectionView<?>, RunnerApi.FunctionSpec> entry :
pCollectionViewsToWindowMappingFns.entrySet()) {
WindowMappingFn windowMappingFn =
new FnApiWindowMappingFn(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
index dcf15f4..b49a9d6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
@@ -34,7 +34,6 @@
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.runners.core.construction.CoderTranslation;
import org.apache.beam.runners.core.construction.RehydratedComponents;
import org.apache.beam.runners.core.construction.SdkComponents;
@@ -79,11 +78,11 @@
@AutoValue
public abstract static class CacheKey {
- public static CacheKey create(SdkFunctionSpec windowMappingFn, BoundedWindow mainWindow) {
+ public static CacheKey create(FunctionSpec windowMappingFn, BoundedWindow mainWindow) {
return new AutoValue_FnApiWindowMappingFn_CacheKey(windowMappingFn, mainWindow);
}
- public abstract SdkFunctionSpec getWindowMappingFn();
+ public abstract FunctionSpec getWindowMappingFn();
public abstract BoundedWindow getMainWindow();
}
@@ -94,7 +93,7 @@
private final IdGenerator idGenerator;
private final FnDataService beamFnDataService;
private final InstructionRequestHandler instructionRequestHandler;
- private final SdkFunctionSpec windowMappingFn;
+ private final FunctionSpec windowMappingFn;
private final Coder<WindowedValue<KV<byte[], BoundedWindow>>> outboundCoder;
private final Coder<WindowedValue<KV<byte[], TargetWindowT>>> inboundCoder;
private final ProcessBundleDescriptor processBundleDescriptor;
@@ -104,7 +103,7 @@
InstructionRequestHandler instructionRequestHandler,
ApiServiceDescriptor dataServiceApiServiceDescriptor,
FnDataService beamFnDataService,
- SdkFunctionSpec windowMappingFn,
+ FunctionSpec windowMappingFn,
Coder<BoundedWindow> mainInputWindowCoder,
Coder<TargetWindowT> sideInputWindowCoder) {
this.idGenerator = idGenerator;
@@ -219,7 +218,7 @@
}
}
- private TargetWindowT loadIfNeeded(SdkFunctionSpec windowMappingFn, BoundedWindow mainWindow) {
+ private TargetWindowT loadIfNeeded(FunctionSpec windowMappingFn, BoundedWindow mainWindow) {
try {
String processRequestInstructionId = idGenerator.getId();
InstructionRequest processRequest =
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
index 4def350..d6d017a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
@@ -37,6 +37,7 @@
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.Deque;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -183,6 +184,7 @@
static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20;
static final int MAX_COMMIT_QUEUE_BYTES = 500 << 20; // 500MB
static final int NUM_COMMIT_STREAMS = 1;
+ static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3;
static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1);
private static final int DEFAULT_STATUS_PORT = 8081;
@@ -424,7 +426,7 @@
private final Counter<Long, Long> javaHarnessMaxMemory;
private final Counter<Integer, Integer> windmillMaxObservedWorkItemCommitBytes;
private final Counter<Integer, Integer> memoryThrashing;
- private Timer refreshActiveWorkTimer;
+ private Timer refreshWorkTimer;
private Timer statusPageTimer;
private final boolean publishCounters;
@@ -785,9 +787,9 @@
0,
options.getWindmillHarnessUpdateReportingPeriod().getMillis());
+ refreshWorkTimer = new Timer("RefreshWork");
if (options.getActiveWorkRefreshPeriodMillis() > 0) {
- refreshActiveWorkTimer = new Timer("RefreshActiveWork");
- refreshActiveWorkTimer.schedule(
+ refreshWorkTimer.schedule(
new TimerTask() {
@Override
public void run() {
@@ -797,6 +799,18 @@
options.getActiveWorkRefreshPeriodMillis(),
options.getActiveWorkRefreshPeriodMillis());
}
+ if (windmillServiceEnabled && options.getStuckCommitDurationMillis() > 0) {
+ int periodMillis = Math.max(options.getStuckCommitDurationMillis() / 10, 100);
+ refreshWorkTimer.schedule(
+ new TimerTask() {
+ @Override
+ public void run() {
+ invalidateStuckCommits();
+ }
+ },
+ periodMillis,
+ periodMillis);
+ }
if (options.getPeriodicStatusPageOutputDirectory() != null) {
statusPageTimer = new Timer("DumpStatusPages");
@@ -863,8 +877,8 @@
globalConfigRefreshTimer.cancel();
}
globalWorkerUpdatesTimer.cancel();
- if (refreshActiveWorkTimer != null) {
- refreshActiveWorkTimer.cancel();
+ if (refreshWorkTimer != null) {
+ refreshWorkTimer.cancel();
}
if (statusPageTimer != null) {
statusPageTimer.cancel();
@@ -925,7 +939,11 @@
computationMap.put(
computationId,
new ComputationState(
- computationId, mapTask, workUnitExecutor, transformUserNameToStateFamily));
+ computationId,
+ mapTask,
+ workUnitExecutor,
+ transformUserNameToStateFamily,
+ stateCache.forComputation(computationId)));
}
}
@@ -1014,7 +1032,9 @@
// Reconnect every now and again to enable better load balancing.
// If at any point the server closes the stream, we will reconnect immediately; otherwise
// we half-close the stream after some time and create a new one.
- stream.closeAfterDefaultTimeout();
+ if (!stream.awaitTermination(GET_WORK_STREAM_TIMEOUT_MINUTES, TimeUnit.MINUTES)) {
+ stream.close();
+ }
} catch (InterruptedException e) {
// Continue processing until !running.get()
}
@@ -1080,11 +1100,12 @@
private final Windmill.WorkItem workItem;
private final Instant startTime;
+ private Instant stateStartTime;
private State state;
public Work(Windmill.WorkItem workItem) {
this.workItem = workItem;
- this.startTime = Instant.now();
+ this.startTime = this.stateStartTime = Instant.now();
this.state = State.QUEUED;
}
@@ -1102,6 +1123,11 @@
public void setState(State state) {
this.state = state;
+ this.stateStartTime = Instant.now();
+ }
+
+ public Instant getStateStartTime() {
+ return stateStartTime;
}
}
@@ -1423,8 +1449,8 @@
sleep(retryLocallyDelayMs);
workUnitExecutor.forceExecute(work);
} else {
- // Consider the item invalid. It will eventually be retried by Windmill if it still needs
- // to be processed.
+ // Consider the item invalid. It will eventually be retried by Windmill if it still needs to
+ // be processed.
computationState.completeWork(key, workItem.getWorkToken());
}
} finally {
@@ -1494,7 +1520,7 @@
Windmill.CommitWorkRequest commitRequest = commitRequestBuilder.build();
LOG.trace("Commit: {}", commitRequest);
activeCommitBytes.set(commitBytes);
- commitWork(commitRequest);
+ windmillServer.commitWork(commitRequest);
activeCommitBytes.set(0);
for (Map.Entry<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> entry :
computationRequestMap.entrySet()) {
@@ -1506,71 +1532,82 @@
}
}
+ // Adds the commit to the commitStream if it fits, returning true iff it is consumed.
+ private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) {
+ Preconditions.checkNotNull(commit);
+ final ComputationState state = commit.getComputationState();
+ final Windmill.WorkItemCommitRequest request = commit.getRequest();
+ final int size = commit.getSize();
+ commit.getWork().setState(State.COMMITTING);
+ activeCommitBytes.addAndGet(size);
+ if (commitStream.commitWorkItem(
+ state.computationId,
+ request,
+ (Windmill.CommitStatus status) -> {
+ if (status != Windmill.CommitStatus.OK) {
+ stateCache.forComputation(state.computationId).invalidate(request.getKey());
+ }
+ activeCommitBytes.addAndGet(-size);
+ // This may throw an exception if the commit was not active, which is possible if it
+ // was deemed stuck.
+ state.completeWork(request.getKey(), request.getWorkToken());
+ })) {
+ return true;
+ } else {
+ // Back out the stats changes since the commit wasn't consumed.
+ commit.getWork().setState(State.COMMIT_QUEUED);
+ activeCommitBytes.addAndGet(-size);
+ return false;
+ }
+ }
+
+ // Helper to batch additional commits into the commit stream as long as they fit.
+ // Returns a commit that was removed from the queue but not consumed or null.
+ private Commit batchCommitsToStream(CommitWorkStream commitStream) {
+ int commits = 1;
+ while (running.get()) {
+ Commit commit;
+ try {
+ if (commits < 5) {
+ commit = commitQueue.poll(10 - 2 * commits, TimeUnit.MILLISECONDS);
+ } else {
+ commit = commitQueue.poll();
+ }
+ } catch (InterruptedException e) {
+ // Continue processing until !running.get()
+ continue;
+ }
+ if (commit == null || !addCommitToStream(commit, commitStream)) {
+ return commit;
+ }
+ commits++;
+ }
+ return null;
+ }
+
private void streamingCommitLoop() {
StreamPool<CommitWorkStream> streamPool =
new StreamPool<>(
NUM_COMMIT_STREAMS, COMMIT_STREAM_TIMEOUT, windmillServer::commitWorkStream);
- Commit commit = null;
+ Commit initialCommit = null;
while (running.get()) {
- // Batch commits as long as there are more and we can fit them in the current request.
- // We lazily initialize the commit stream to make sure that we only create one after
- // we have a commit.
- CommitWorkStream commitStream = null;
- int commits = 0;
- while (running.get()) {
- // There may be a commit left over from the previous iteration but if not, pull one.
- if (commit == null) {
- try {
- if (commits == 0) {
- commit = commitQueue.take();
- } else if (commits < 10) {
- commit = commitQueue.poll(20 - 2 * commits, TimeUnit.MILLISECONDS);
- } else {
- commit = commitQueue.poll();
- }
- } catch (InterruptedException e) {
- // Continue processing until !running.get()
- continue;
- }
- if (commit == null) {
- // No longer batching, break loop to trigger flush.
- break;
- }
- }
-
- commits++;
- final ComputationState state = commit.getComputationState();
- final Windmill.WorkItemCommitRequest request = commit.getRequest();
- final int size = commit.getSize();
- commit.getWork().setState(State.COMMITTING);
- if (commitStream == null) {
- commitStream = streamPool.getStream();
- }
- if (commitStream.commitWorkItem(
- state.computationId,
- request,
- (Windmill.CommitStatus status) -> {
- if (status != Windmill.CommitStatus.OK) {
- stateCache.forComputation(state.computationId).invalidate(request.getKey());
- }
- state.completeWork(request.getKey(), request.getWorkToken());
- activeCommitBytes.addAndGet(-size);
- })) {
- // The commit was consumed.
- commit = null;
- // It's possible we could decrement from the callback above before adding here but since
- // it's just for the status page we don't care.
- activeCommitBytes.addAndGet(size);
- } else {
- // The commit was not consumed, leave it set and it will be added to the subsequent stream
- // on the next iteration after this stream is flushed.
- break;
+ if (initialCommit == null) {
+ try {
+ initialCommit = commitQueue.take();
+ } catch (InterruptedException e) {
+ continue;
}
}
- if (commitStream != null) {
- commitStream.flush();
- streamPool.releaseStream(commitStream);
+ // We initialize the commit stream only after we have a commit to make sure it is fresh.
+ CommitWorkStream commitStream = streamPool.getStream();
+ if (!addCommitToStream(initialCommit, commitStream)) {
+ throw new AssertionError("Initial commit on flushed stream should always be accepted.");
}
+ // Batch additional commits to the stream and possibly make an un-batched commit the next
+ // initial commit.
+ initialCommit = batchCommitsToStream(commitStream);
+ commitStream.flush();
+ streamPool.releaseStream(commitStream);
}
}
@@ -1583,10 +1620,6 @@
.build());
}
- private void commitWork(Windmill.CommitWorkRequest request) {
- windmillServer.commitWork(request);
- }
-
private void getConfigFromWindmill(String computation) {
Windmill.GetConfigRequest request =
Windmill.GetConfigRequest.newBuilder().addComputations(computation).build();
@@ -2056,6 +2089,14 @@
metricTrackingWindmillServer.refreshActiveWork(active);
}
+ private void invalidateStuckCommits() {
+ Instant stuckCommitDeadline =
+ Instant.now().minus(Duration.millis(options.getStuckCommitDurationMillis()));
+ for (Map.Entry<String, ComputationState> entry : computationMap.entrySet()) {
+ entry.getValue().invalidateStuckCommits(stuckCommitDeadline);
+ }
+ }
+
/**
* Class representing the state of a computation.
*
@@ -2068,16 +2109,18 @@
private final ImmutableMap<String, String> transformUserNameToStateFamily;
// Map from key to work for the key. The first item in the queue is
// actively processing. Synchronized by itself.
- private final Map<ByteString, Queue<Work>> activeWork = new HashMap<>();
+ private final Map<ByteString, Deque<Work>> activeWork = new HashMap<>();
private final BoundedQueueExecutor executor;
private final ConcurrentMap<SdkWorkerHarness, ConcurrentLinkedQueue<ExecutionState>>
executionStateQueues = new ConcurrentHashMap<>();
+ private final WindmillStateCache.ForComputation computationStateCache;
public ComputationState(
String computationId,
MapTask mapTask,
BoundedQueueExecutor executor,
- Map<String, String> transformUserNameToStateFamily) {
+ Map<String, String> transformUserNameToStateFamily,
+ WindmillStateCache.ForComputation computationStateCache) {
this.computationId = computationId;
this.mapTask = mapTask;
this.executor = executor;
@@ -2085,6 +2128,7 @@
transformUserNameToStateFamily != null
? ImmutableMap.copyOf(transformUserNameToStateFamily)
: ImmutableMap.of();
+ this.computationStateCache = computationStateCache;
Preconditions.checkNotNull(mapTask.getStageName());
Preconditions.checkNotNull(mapTask.getSystemName());
}
@@ -2110,20 +2154,23 @@
/** Mark the given key and work as active. */
public boolean activateWork(ByteString key, Work work) {
synchronized (activeWork) {
- Queue<Work> queue = activeWork.get(key);
- if (queue == null) {
- queue = new ArrayDeque<>();
- activeWork.put(key, queue);
- queue.add(work);
- // Fall through to execute without the lock held.
- } else {
- if (queue.peek().getWorkItem().getWorkToken() != work.getWorkItem().getWorkToken()) {
- // Queue the work for later processing.
- queue.add(work);
- return true;
+ Deque<Work> queue = activeWork.get(key);
+ if (queue != null) {
+ Preconditions.checkState(!queue.isEmpty());
+ // Ensure we don't already have this work token queueud.
+ for (Work queuedWork : queue) {
+ if (queuedWork.getWorkItem().getWorkToken() == work.getWorkItem().getWorkToken()) {
+ return false;
+ }
}
- // Skip the work if duplicate
- return false;
+ // Queue the work for later processing.
+ queue.addLast(work);
+ return true;
+ } else {
+ queue = new ArrayDeque<>();
+ queue.addLast(work);
+ activeWork.put(key, queue);
+ // Fall through to execute without the lock held.
}
}
executor.execute(work);
@@ -2132,11 +2179,11 @@
/** Marks the work for a the given key as complete. Schedules queued work for the key if any. */
public void completeWork(ByteString key, long workToken) {
- Work work = null;
+ Work nextWork;
synchronized (activeWork) {
Queue<Work> queue = activeWork.get(key);
Preconditions.checkNotNull(queue);
- Work completedWork = queue.poll();
+ Work completedWork = queue.peek();
// avoid Preconditions.checkNotNull and checkState here to prevent eagerly evaluating the
// format string parameters for the error message.
if (completedWork == null) {
@@ -2153,20 +2200,45 @@
completedWork.getWorkItem().getWorkToken(),
workToken));
}
- if (queue.peek() == null) {
- activeWork.remove(key);
- return;
+ queue.remove(); // We consumed the matching work item.
+ nextWork = queue.peek();
+ if (nextWork == null) {
+ Preconditions.checkState(queue == activeWork.remove(key));
}
- work = queue.peek();
}
- executor.forceExecute(work);
+ if (nextWork != null) {
+ executor.forceExecute(nextWork);
+ }
+ }
+
+ public void invalidateStuckCommits(Instant stuckCommitDeadline) {
+ synchronized (activeWork) {
+ // Determine the stuck commit keys but complete them outside of iterating over
+ // activeWork as completeWork may delete the entry from activeWork.
+ Map<ByteString, Long> stuckCommits = new HashMap<>();
+ for (Map.Entry<ByteString, Deque<Work>> entry : activeWork.entrySet()) {
+ ByteString key = entry.getKey();
+ Work work = entry.getValue().peek();
+ if (work.getState() == State.COMMITTING
+ && work.getStateStartTime().isBefore(stuckCommitDeadline)) {
+ LOG.error(
+ "Detected key with sharding key {} stuck in COMMITTING state, completing it with error. Key ",
+ work.workItem.getShardingKey());
+ stuckCommits.put(key, work.getWorkItem().getWorkToken());
+ }
+ }
+ for (Map.Entry<ByteString, Long> stuckCommit : stuckCommits.entrySet()) {
+ computationStateCache.invalidate(stuckCommit.getKey());
+ completeWork(stuckCommit.getKey(), stuckCommit.getValue());
+ }
+ }
}
/** Adds any work started before the refreshDeadline to the GetDataRequest builder. */
public List<Windmill.KeyedGetDataRequest> getKeysToRefresh(Instant refreshDeadline) {
List<Windmill.KeyedGetDataRequest> result = new ArrayList<>();
synchronized (activeWork) {
- for (Map.Entry<ByteString, Queue<Work>> entry : activeWork.entrySet()) {
+ for (Map.Entry<ByteString, Deque<Work>> entry : activeWork.entrySet()) {
ByteString key = entry.getKey();
for (Work work : entry.getValue()) {
if (work.getStartTime().isBefore(refreshDeadline)) {
@@ -2183,6 +2255,12 @@
return result;
}
+ private String elapsedString(Instant start, Instant end) {
+ Duration activeFor = new Duration(start, end);
+ // Duration's toString always starts with "PT"; remove that here.
+ return activeFor.toString().substring(2);
+ }
+
public void printActiveWork(PrintWriter writer) {
final Instant now = Instant.now();
// The max number of keys in COMMITTING or COMMIT_QUEUED status to be shown.
@@ -2192,17 +2270,16 @@
"<table border=\"1\" "
+ "style=\"border-collapse:collapse;padding:5px;border-spacing:5px;border:1px\">");
writer.println(
- "<tr><th>Key</th><th>Token</th><th>Queued</th><th>Active For</th><th>State</th></tr>");
+ "<tr><th>Key</th><th>Token</th><th>Queued</th><th>Active For</th><th>State</th><th>State Active For</th></tr>");
// We use a StringBuilder in the synchronized section to buffer writes since the provided
// PrintWriter may block when flushing.
StringBuilder builder = new StringBuilder();
synchronized (activeWork) {
- for (Map.Entry<ByteString, Queue<Work>> entry : activeWork.entrySet()) {
+ for (Map.Entry<ByteString, Deque<Work>> entry : activeWork.entrySet()) {
Queue<Work> queue = entry.getValue();
+ Preconditions.checkNotNull(queue);
Work work = queue.peek();
- if (work == null) {
- continue;
- }
+ Preconditions.checkNotNull(work);
Windmill.WorkItem workItem = work.getWorkItem();
State state = work.getState();
if (state == State.COMMITTING || state == State.COMMIT_QUEUED) {
@@ -2218,12 +2295,12 @@
builder.append("</td><td>");
builder.append(queue.size() - 1);
builder.append("</td><td>");
- Duration activeFor = new Duration(work.getStartTime(), now);
- // Duration's toString always starts with "PT"; remove that here.
- builder.append(activeFor.toString().substring(2));
+ builder.append(elapsedString(work.getStartTime(), now));
builder.append("</td><td>");
builder.append(state);
builder.append("</td></tr>\n");
+ builder.append(elapsedString(work.getStateStartTime(), now));
+ builder.append("</td></tr>\n");
}
}
writer.print(builder.toString());
@@ -2231,7 +2308,7 @@
if (commitPendingCount >= maxCommitPending) {
writer.println("<br>");
writer.print("Skipped keys in COMMITTING/COMMIT_QUEUED: ");
- writer.println(maxCommitPending - commitPendingCount);
+ writer.println(commitPendingCount - maxCommitPending);
writer.println("<br>");
}
}
@@ -2326,20 +2403,13 @@
+ MAX_WORK_UNITS_QUEUED
+ "<br>");
writer.print("Commit Queue: ");
- writer.print(commitQueue.weight() >> 20);
- writer.print("MB, ");
+ appendHumanizedBytes(commitQueue.weight(), writer);
+ writer.print(", ");
writer.print(commitQueue.size());
writer.println(" elements<br>");
writer.print("Active commit: ");
- long commitBytes = activeCommitBytes.get();
- if (commitBytes == 0) {
- writer.print("none");
- } else {
- writer.print("~");
- writer.print((commitBytes >> 20) + 1);
- writer.print("MB");
- }
+ appendHumanizedBytes(activeCommitBytes.get(), writer);
writer.println("<br>");
metricTrackingWindmillServer.printHtml(writer);
@@ -2354,5 +2424,20 @@
writer.println("<br>");
}
}
+
+ private void appendHumanizedBytes(long bytes, PrintWriter writer) {
+ if (bytes < (4 << 10)) {
+ writer.print(bytes);
+ writer.print("B");
+ } else if (bytes < (4 << 20)) {
+ writer.print("~");
+ writer.print(bytes >> 10);
+ writer.print("KB");
+ } else {
+ writer.print("~");
+ writer.print(bytes >> 20);
+ writer.print("MB");
+ }
+ }
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
index a114b6f..76aa8b0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
@@ -602,6 +602,8 @@
.setTimer(
StateNamespaces.window(windowCoder, window),
timerId,
+ "",
+ cleanupTime,
cleanupTime,
TimeDomain.EVENT_TIME);
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java
index 7c5babf..2c00c99 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java
@@ -32,6 +32,7 @@
import org.apache.beam.runners.core.StateTags;
import org.apache.beam.runners.core.TimerInternals.TimerData;
import org.apache.beam.runners.core.TimerInternals.TimerDataCoder;
+import org.apache.beam.runners.core.TimerInternals.TimerDataCoderV2;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest;
import org.apache.beam.sdk.coders.AtomicCoder;
@@ -60,6 +61,7 @@
private final StateTag<BagState<WindowedValue<InputT>>> elementsAddr;
private final StateTag<BagState<TimerData>> timersAddr;
+ private final StateTag<BagState<TimerData>> oldTimersAddr;
private final StateTag<WatermarkHoldState> watermarkHoldingAddr;
private final StateTag<ValueState<Map<W, Set<Windmill.GlobalDataRequest>>>> blockedMapAddr;
@@ -85,8 +87,11 @@
this.elementsAddr =
StateTags.makeSystemTagInternal(
StateTags.bag("elem", WindowedValue.getFullCoder(inputCoder, mainWindowCoder)));
- this.timersAddr =
+ this.oldTimersAddr =
StateTags.makeSystemTagInternal(StateTags.bag("timer", TimerDataCoder.of(mainWindowCoder)));
+ this.timersAddr =
+ StateTags.makeSystemTagInternal(
+ StateTags.bag("timerV2", TimerDataCoderV2.of(mainWindowCoder)));
StateTag<WatermarkHoldState> watermarkTag =
StateTags.watermarkStateInternal(
"holdForSideinput", windowingStrategy.getTimestampCombiner());
@@ -169,6 +174,7 @@
List<BagState<TimerData>> timers = Lists.newArrayList();
for (W window : readyWindows) {
timers.add(timerBag(window).readLater());
+ timers.add(timerOldBag(window).readLater());
}
return timers;
}
@@ -275,6 +281,12 @@
.state(StateNamespaces.window(mainWindowCoder, window), timersAddr);
}
+ BagState<TimerData> timerOldBag(W window) {
+ return stepContext
+ .stateInternals()
+ .state(StateNamespaces.window(mainWindowCoder, window), oldTimersAddr);
+ }
+
private <SideWindowT extends BoundedWindow> Windmill.GlobalDataRequest buildGlobalDataRequest(
PCollectionView<?> view, BoundedWindow mainWindow) {
@SuppressWarnings("unchecked")
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java
index d74b0db..b419a38 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java
@@ -106,10 +106,9 @@
public void invalidate(ByteString processingKey) {
synchronized (this) {
ComputationKey key = new ComputationKey(computation, processingKey);
- for (StateId id : keyIndex.get(key)) {
+ for (StateId id : keyIndex.removeAll(key)) {
stateCache.invalidate(id);
}
- keyIndex.removeAll(key);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java
index c2deb2f..b2ba62e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java
@@ -94,8 +94,16 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant timestamp, TimeDomain timeDomain) {
- timers.put(timerId, namespace, TimerData.of(timerId, namespace, timestamp, timeDomain));
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant timestamp,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ timers.put(
+ timerId,
+ namespace,
+ TimerData.of(timerId, timerFamilyId, namespace, timestamp, outputTimestamp, timeDomain));
timerStillPresent.put(timerId, namespace, true);
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiver.java
index 286ea8a..e3d2277 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiver.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiver.java
@@ -119,7 +119,8 @@
String timerId = timerSpec.timerId();
TimerInternals timerInternals = stepContext.namespacedToUser().timerInternals();
- timerInternals.setTimer(namespace, timerId, timer.getTimestamp(), timeDomain);
+ timerInternals.setTimer(
+ namespace, timerId, "", timer.getTimestamp(), timer.getOutputTimestamp(), timeDomain);
timerIdToKey.put(timerId, windowedValue.getValue().getKey());
timerIdToPayload.put(timerId, timer.getPayload());
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java
index fc42534..9b75433 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.dataflow.worker.graph;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.apache.beam.runners.dataflow.util.Structs.getBytes;
import static org.apache.beam.runners.dataflow.util.Structs.getString;
import static org.apache.beam.runners.dataflow.worker.graph.LengthPrefixUnknownCoders.forSideInputInfos;
@@ -160,8 +161,6 @@
// For intermediate PCollections we fabricate, we make a bogus WindowingStrategy
// TODO: create a correct windowing strategy, including coders and environment
- // An SdkFunctionSpec is invalid without a working environment reference. We can revamp that
- // when we inline SdkFunctionSpec and FunctionSpec, both slated for inlining wherever they occur
// Default to use the Java environment if pipeline doesn't have environment specified.
if (pipeline.getComponents().getEnvironmentsMap().isEmpty()) {
@@ -389,7 +388,7 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(parDoPayload.toByteString());
} else {
- // legacy path - bytes are the SdkFunctionSpec's payload field, basically, and
+ // legacy path - bytes are the FunctionSpec's payload field, basically, and
// SDKs expect it in the PTransform's payload field
byte[] userFnBytes = getBytes(userFnSpec, PropertyNames.SERIALIZED_FN);
transformSpec
@@ -487,7 +486,8 @@
executableStageUserStateReference,
executableStageTimers,
executableStageTransforms,
- executableStageOutputs);
+ executableStageOutputs,
+ DEFAULT_WIRE_CODER_SETTING);
return ExecutableStageNode.create(
executableStage,
ptransformIdToNameContexts.build(),
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java
index 6ba81d1..0d96981 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java
@@ -132,7 +132,7 @@
}
// Gather all the side input window mapping fns which we need to request the SDK to map
- ImmutableMap.Builder<PCollectionView<?>, RunnerApi.SdkFunctionSpec>
+ ImmutableMap.Builder<PCollectionView<?>, RunnerApi.FunctionSpec>
pCollectionViewsToWindowMapingsFns = ImmutableMap.builder();
parDoPayload
.getSideInputsMap()
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java
index 853af69..327234c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java
@@ -349,7 +349,7 @@
public abstract static class FetchAndFilterStreamingSideInputsNode extends Node {
public static FetchAndFilterStreamingSideInputsNode create(
WindowingStrategy<?, ?> windowingStrategy,
- Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec> pCollectionViewsToWindowMappingFns,
+ Map<PCollectionView<?>, RunnerApi.FunctionSpec> pCollectionViewsToWindowMappingFns,
NameContext nameContext) {
return new AutoValue_Nodes_FetchAndFilterStreamingSideInputsNode(
windowingStrategy, pCollectionViewsToWindowMappingFns, nameContext);
@@ -357,7 +357,7 @@
public abstract WindowingStrategy<?, ?> getWindowingStrategy();
- public abstract Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec>
+ public abstract Map<PCollectionView<?>, RunnerApi.FunctionSpec>
getPCollectionViewsToWindowMappingFns();
public abstract NameContext getNameContext();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java
index 0c33ee8..b44e42c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java
@@ -190,8 +190,6 @@
// For intermediate PCollections we fabricate, we make a bogus WindowingStrategy
// TODO: create a correct windowing strategy, including coders and environment
- // An SdkFunctionSpec is invalid without a working environment reference. We can revamp that
- // when we inline SdkFunctionSpec and FunctionSpec, both slated for inlining wherever they occur
SdkComponents sdkComponents = SdkComponents.create(pipeline.getComponents());
// Default to use the Java environment if pipeline doesn't have environment specified.
@@ -351,7 +349,7 @@
.setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
.setPayload(parDoPayload.toByteString());
} else {
- // legacy path - bytes are the SdkFunctionSpec's payload field, basically, and
+ // legacy path - bytes are the FunctionSpec's payload field, basically, and
// SDKs expect it in the PTransform's payload field
byte[] userFnBytes = getBytes(userFnSpec, PropertyNames.SERIALIZED_FN);
transformSpec
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java
index dbad980..b5d3224 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java
@@ -69,6 +69,12 @@
void setActiveWorkRefreshPeriodMillis(int value);
+ @Description("Necessary duration for a commit to be considered stuck and invalidated.")
+ @Default.Integer(10 * 60 * 1000)
+ int getStuckCommitDurationMillis();
+
+ void setStuckCommitDurationMillis(int value);
+
@Description(
"Period for sending 'global get config' requests to the service. The duration is "
+ "specified as seconds in 'PTx.yS' format, e.g. 'PT5.125S'."
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java
index fc87bc4..c769881 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java
@@ -18,8 +18,8 @@
package org.apache.beam.runners.dataflow.worker.util.common.worker;
import java.io.IOException;
+import java.time.Duration;
import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Objects;
@@ -27,26 +27,41 @@
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheLoader;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LoadingCache;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Weigher;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Ints;
/** A {@link ShuffleBatchReader} that caches batches as they're read. */
public class CachingShuffleBatchReader implements ShuffleBatchReader {
private final ShuffleBatchReader reader;
@VisibleForTesting final LoadingCache<BatchRange, Batch> cache;
- /** Limit the size of the cache to 1000 batches. */
- private static final int MAXIMUM_BATCHES = 1000;
+ /**
+ * Limit the size of the cache to 1GiB of batches.
+ *
+ * <p>If this increases beyond Integer.MAX_VALUE then {@link BatchWeigher} must be updated.
+ * Because a batch may be larger than 1GiB, the actual in-memory batch size may exceed this value.
+ */
+ private static final int MAXIMUM_WEIGHT = 1024 * 1024 * 1024;
// Ensure that batches in the cache are expired quickly
// for improved GC performance.
- private static final long EXPIRE_AFTER_MS = 250;
+ private static final Duration EXPIRE_AFTER = Duration.ofMillis(250);
+ /**
+ * Creates the caching reader.
+ *
+ * @param shuffleReader wrapped reader.
+ * @param maximumWeightBytes maximum bytes for the cache.
+ * @param expireAfterAccess cache items may be evicted after the elapsed duration.
+ */
public CachingShuffleBatchReader(
- ShuffleBatchReader shuffleReader, int maximumBatches, long expireAfterAccessMillis) {
+ ShuffleBatchReader shuffleReader, long maximumWeightBytes, Duration expireAfterAccess) {
this.reader = shuffleReader;
this.cache =
CacheBuilder.newBuilder()
- .maximumSize(maximumBatches)
- .expireAfterAccess(expireAfterAccessMillis, TimeUnit.MILLISECONDS)
+ .maximumWeight(maximumWeightBytes)
+ .weigher(new BatchWeigher())
+ .expireAfterAccess(expireAfterAccess)
.<BatchRange, Batch>build(
new CacheLoader<BatchRange, Batch>() {
@Override
@@ -58,12 +73,24 @@
});
}
+ /**
+ * Creates the caching reader with a maximum size of {@link MAXIMUM_WEIGHT} and an element expiry
+ * duration of {@link EXPIRE_AFTER}.
+ *
+ * @param shuffleReader wrapped reader.
+ */
public CachingShuffleBatchReader(ShuffleBatchReader shuffleReader) {
- this(shuffleReader, MAXIMUM_BATCHES, EXPIRE_AFTER_MS);
+ this(shuffleReader, MAXIMUM_WEIGHT, EXPIRE_AFTER);
}
- public CachingShuffleBatchReader(ShuffleBatchReader shuffleReader, int maximumBatches) {
- this(shuffleReader, maximumBatches, EXPIRE_AFTER_MS);
+ /**
+ * Creates the caching reader with an element expiry duration of {@link EXPIRE_AFTER}.
+ *
+ * @param shuffleReader wrapped reader.
+ * @param maximumWeightBytes maximum bytes for the cache.
+ */
+ public CachingShuffleBatchReader(ShuffleBatchReader shuffleReader, long maximumWeightBytes) {
+ this(shuffleReader, maximumWeightBytes, EXPIRE_AFTER);
}
@Override
@@ -102,4 +129,18 @@
return Objects.hashCode(startPosition, endPosition);
}
}
+
+ /**
+ * Returns the weight of a Batch, in bytes, within the range [0, Integer.MAX_VALUE].
+ *
+ * <p>The cache holds {@link MAX_WEIGHT} bytes. If {@link MAX_WEIGHT} is increased beyond
+ * Integer.MAX_VALUE bytes, a new weighing heuristic will be required to avoid under representing
+ * the number of bytes in memory.
+ */
+ static final class BatchWeigher implements Weigher<BatchRange, Batch> {
+ @Override
+ public int weigh(BatchRange key, Batch value) {
+ return Ints.saturatedCast(value.bytes);
+ }
+ }
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleBatchReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleBatchReader.java
index d1676eb..29890c8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleBatchReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleBatchReader.java
@@ -30,10 +30,13 @@
public static class Batch {
public final List<ShuffleEntry> entries;
@Nullable public final ShufflePosition nextStartPosition;
+ public final long bytes;
- public Batch(List<ShuffleEntry> entries, @Nullable ShufflePosition nextStartPosition) {
+ public Batch(
+ List<ShuffleEntry> entries, @Nullable ShufflePosition nextStartPosition, long bytes) {
this.entries = entries;
this.nextStartPosition = nextStartPosition;
+ this.bytes = bytes;
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java
index 98f74c7..c64803d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java
@@ -117,8 +117,6 @@
// high.
private static final long DEFAULT_UNARY_RPC_DEADLINE_SECONDS = 300;
private static final long DEFAULT_STREAM_RPC_DEADLINE_SECONDS = 300;
- // Stream clean close seconds must be set lower than the stream deadline seconds.
- private static final long DEFAULT_STREAM_CLEAN_CLOSE_SECONDS = 180;
private static final Duration MIN_BACKOFF = Duration.millis(1);
private static final Duration MAX_BACKOFF = Duration.standardSeconds(30);
@@ -137,13 +135,14 @@
syncStubList = new ArrayList<>();
private WindmillApplianceGrpc.WindmillApplianceBlockingStub syncApplianceStub = null;
private long unaryDeadlineSeconds = DEFAULT_UNARY_RPC_DEADLINE_SECONDS;
+ private long streamDeadlineSeconds = DEFAULT_STREAM_RPC_DEADLINE_SECONDS;
private ImmutableSet<HostAndPort> endpoints;
private int logEveryNStreamFailures = 20;
private Duration maxBackoff = MAX_BACKOFF;
private final ThrottleTimer getWorkThrottleTimer = new ThrottleTimer();
private final ThrottleTimer getDataThrottleTimer = new ThrottleTimer();
private final ThrottleTimer commitWorkThrottleTimer = new ThrottleTimer();
- Random rand = new Random();
+ private final Random rand = new Random();
private final Set<AbstractWindmillStream<?, ?>> streamRegistry =
Collections.newSetFromMap(new ConcurrentHashMap<AbstractWindmillStream<?, ?>, Boolean>());
@@ -213,7 +212,7 @@
private synchronized void initializeLocalHost(int port) throws IOException {
this.logEveryNStreamFailures = 1;
this.maxBackoff = Duration.millis(500);
- this.unaryDeadlineSeconds = 10; // For local testing use a short deadline.
+ this.unaryDeadlineSeconds = 10; // For local testing use short deadlines.
Channel channel = localhostChannel(port);
if (streamingEngineEnabled()) {
this.stubList.add(CloudWindmillServiceV1Alpha1Grpc.newStub(channel));
@@ -599,7 +598,7 @@
this.clientFactory = clientFactory;
}
- /** Called on each response from the server */
+ /** Called on each response from the server. */
protected abstract void onResponse(ResponseT response);
/** Called when a new underlying stream to the server has been opened. */
protected abstract void onNewStream();
@@ -607,7 +606,7 @@
protected abstract boolean hasPendingRequests();
/**
* Called when the stream is throttled due to resource exhausted errors. Will be called for each
- * resource exhausted error not just the first. onResponse() must stop throttling on reciept of
+ * resource exhausted error not just the first. onResponse() must stop throttling on receipt of
* the first good message.
*/
protected abstract void startThrottleTimer();
@@ -746,15 +745,6 @@
}
@Override
- public final void closeAfterDefaultTimeout() throws InterruptedException {
- if (!finishLatch.await(DEFAULT_STREAM_CLEAN_CLOSE_SECONDS, TimeUnit.SECONDS)) {
- // If the stream did not close due to error in the specified amount of time, half-close
- // the stream cleanly.
- close();
- }
- }
-
- @Override
public final Instant startTime() {
return new Instant(startTimeMs.get());
}
@@ -773,7 +763,7 @@
super(
responseObserver ->
stub()
- .withDeadlineAfter(DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
+ .withDeadlineAfter(streamDeadlineSeconds, TimeUnit.SECONDS)
.getWorkStream(responseObserver));
this.request = request;
this.receiver = receiver;
@@ -946,7 +936,7 @@
super(
responseObserver ->
stub()
- .withDeadlineAfter(DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
+ .withDeadlineAfter(streamDeadlineSeconds, TimeUnit.SECONDS)
.getDataStream(responseObserver));
startStream();
}
@@ -1161,7 +1151,7 @@
private class Batcher {
long queuedBytes = 0;
- Map<Long, PendingRequest> queue = new HashMap<>();
+ final Map<Long, PendingRequest> queue = new HashMap<>();
boolean canAccept(PendingRequest request) {
return queue.isEmpty()
@@ -1178,6 +1168,7 @@
void flush() {
flushInternal(queue);
queuedBytes = 0;
+ queue.clear();
}
}
@@ -1187,7 +1178,7 @@
super(
responseObserver ->
stub()
- .withDeadlineAfter(DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
+ .withDeadlineAfter(streamDeadlineSeconds, TimeUnit.SECONDS)
.commitWorkStream(responseObserver));
startStream();
}
@@ -1219,16 +1210,27 @@
protected void onResponse(StreamingCommitResponse response) {
commitWorkThrottleTimer.stop();
+ RuntimeException finalException = null;
for (int i = 0; i < response.getRequestIdCount(); ++i) {
long requestId = response.getRequestId(i);
PendingRequest done = pending.remove(requestId);
if (done == null) {
LOG.error("Got unknown commit request ID: {}", requestId);
} else {
- done.onDone.accept(
- (i < response.getStatusCount()) ? response.getStatus(i) : CommitStatus.OK);
+ try {
+ done.onDone.accept(
+ (i < response.getStatusCount()) ? response.getStatus(i) : CommitStatus.OK);
+ } catch (RuntimeException e) {
+ // Catch possible exceptions to ensure that an exception for one commit does not prevent
+ // other commits from being processed.
+ LOG.warn("Exception while processing commit response {} ", e);
+ finalException = e;
+ }
}
}
+ if (finalException != null) {
+ throw finalException;
+ }
}
@Override
@@ -1252,7 +1254,7 @@
batcher.flush();
}
- private final void flushInternal(Map<Long, PendingRequest> requests) {
+ private void flushInternal(Map<Long, PendingRequest> requests) {
if (requests.isEmpty()) {
return;
}
@@ -1266,7 +1268,6 @@
} else {
issueBatchedRequest(requests);
}
- requests.clear();
}
private void issueSingleRequest(final long id, PendingRequest pendingRequest) {
@@ -1278,13 +1279,13 @@
.setShardingKey(pendingRequest.request.getShardingKey())
.setSerializedWorkItemCommit(pendingRequest.request.toByteString());
StreamingCommitWorkRequest chunk = requestBuilder.build();
- try {
- synchronized (this) {
- pending.put(id, pendingRequest);
+ synchronized (this) {
+ pending.put(id, pendingRequest);
+ try {
send(chunk);
+ } catch (IllegalStateException e) {
+ // Stream was broken, request will be retried when stream is reopened.
}
- } catch (IllegalStateException e) {
- // Stream was broken, request will be retried when stream is reopened.
}
}
@@ -1303,13 +1304,13 @@
chunkBuilder.setSerializedWorkItemCommit(request.request.toByteString());
}
StreamingCommitWorkRequest request = requestBuilder.build();
- try {
- synchronized (this) {
- pending.putAll(requests);
+ synchronized (this) {
+ pending.putAll(requests);
+ try {
send(request);
+ } catch (IllegalStateException e) {
+ // Stream was broken, request will be retried when stream is reopened.
}
- } catch (IllegalStateException e) {
- // Stream was broken, request will be retried when stream is reopened.
}
}
@@ -1492,7 +1493,7 @@
}
}
- /** Returns if the specified type is currently being throttled */
+ /** Returns if the specified type is currently being throttled. */
public synchronized boolean throttled() {
return startTime != -1;
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java
index 2b5453b..31c5114 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java
@@ -106,12 +106,6 @@
/** Waits for the server to close its end of the connection, with timeout. */
boolean awaitTermination(int time, TimeUnit unit) throws InterruptedException;
- /**
- * Cleanly closes the stream after implementation-speficied timeout, unless the stream is
- * aborted before the timeout is reached.
- */
- void closeAfterDefaultTimeout() throws InterruptedException;
-
/** Returns when the stream was opened. */
Instant startTime();
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java
index 3682545..3451743 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java
@@ -18,7 +18,7 @@
package org.apache.beam.runners.dataflow.worker;
import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doCallRealMethod;
import static org.mockito.Mockito.when;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java
index 947d290..b96ee2f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java
@@ -18,7 +18,7 @@
package org.apache.beam.runners.dataflow.worker;
import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doCallRealMethod;
import static org.mockito.Mockito.when;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java
index 72c894a..66be62d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java
@@ -18,6 +18,7 @@
package org.apache.beam.runners.dataflow.worker;
import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -82,4 +83,9 @@
assertThat(decoded, equalTo(descriptor));
assertThat(decoded.getUrl(), equalTo("some_test_url"));
}
+
+ @Test
+ public void testParseStatusApiDescriptor() throws TextFormat.ParseException {
+ assertNull(DataflowWorkerHarnessHelper.getStatusDescriptor());
+ }
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java
index 2c27606..e788578 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java
@@ -64,6 +64,8 @@
private final AtomicInteger expectedExceptionCount;
private final ErrorCollector errorCollector;
private boolean isReady = true;
+ private boolean dropStreamingCommits = false;
+ private final AtomicInteger droppedStreamingCommits;
public FakeWindmillServer(ErrorCollector errorCollector) {
workToOffer = new ConcurrentLinkedQueue<>();
@@ -73,6 +75,11 @@
expectedExceptionCount = new AtomicInteger();
this.errorCollector = errorCollector;
statsReceived = new ArrayList<>();
+ droppedStreamingCommits = new AtomicInteger();
+ }
+
+ public void setDropStreamingCommits(boolean dropStreamingCommits) {
+ this.dropStreamingCommits = dropStreamingCommits;
}
public void addWorkToOffer(Windmill.GetWorkResponse work) {
@@ -188,7 +195,12 @@
final CountDownLatch done = new CountDownLatch(1);
return new GetWorkStream() {
@Override
- public void closeAfterDefaultTimeout() {
+ public void close() {
+ done.countDown();
+ }
+
+ @Override
+ public boolean awaitTermination(int time, TimeUnit unit) throws InterruptedException {
while (done.getCount() > 0) {
Windmill.GetWorkResponse response = workToOffer.poll();
if (response == null) {
@@ -210,15 +222,6 @@
}
}
}
- }
-
- @Override
- public void close() {
- done.countDown();
- }
-
- @Override
- public boolean awaitTermination(int time, TimeUnit unit) throws InterruptedException {
return done.await(time, unit);
}
@@ -280,9 +283,6 @@
}
@Override
- public void closeAfterDefaultTimeout() {}
-
- @Override
public Instant startTime() {
return startTime;
}
@@ -303,9 +303,15 @@
errorCollector.checkThat(
request.getShardingKey(), allOf(greaterThan(0L), lessThan(Long.MAX_VALUE)));
errorCollector.checkThat(request.getCacheToken(), not(equalTo(0L)));
- commitsReceived.put(request.getWorkToken(), request);
- onDone.accept(Windmill.CommitStatus.OK);
- return true; // The request was accepted.
+ if (dropStreamingCommits) {
+ droppedStreamingCommits.incrementAndGet();
+ } else {
+ commitsReceived.put(request.getWorkToken(), request);
+ onDone.accept(Windmill.CommitStatus.OK);
+ }
+ // Return true to indicate the request was accepted even if we are dropping the commit
+ // to simulate a dropped commit.
+ return true;
}
@Override
@@ -320,9 +326,6 @@
}
@Override
- public void closeAfterDefaultTimeout() {}
-
- @Override
public Instant startTime() {
return startTime;
}
@@ -358,6 +361,15 @@
return commitsReceived;
}
+ public void waitForDroppedCommits(int droppedCommits) {
+ LOG.debug("waitForDroppedCommits: {}", droppedCommits);
+ int maxTries = 10;
+ while (maxTries-- > 0 && droppedStreamingCommits.get() < droppedCommits) {
+ Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+ }
+ assertEquals(droppedCommits, droppedStreamingCommits.get());
+ }
+
public void setExpectedExceptionCount(int i) {
expectedExceptionCount.getAndAdd(i);
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java
index 81d4a67..91380b8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java
@@ -30,7 +30,7 @@
import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.RegisterResponse;
import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.runners.core.construction.ParDoTranslation;
import org.apache.beam.runners.core.construction.SdkComponents;
import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
@@ -62,7 +62,7 @@
public class FnApiWindowMappingFnTest {
private static final ApiServiceDescriptor DATA_SERVICE =
ApiServiceDescriptor.newBuilder().setUrl("test://data").build();
- private static final SdkFunctionSpec WINDOW_MAPPING_SPEC =
+ private static final FunctionSpec WINDOW_MAPPING_SPEC =
ParDoTranslation.translateWindowMappingFn(
new GlobalWindows().getDefaultWindowMappingFn(),
SdkComponents.create(PipelineOptionsFactory.create()));
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java
index 467034a..2728e26 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java
@@ -20,8 +20,8 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.verify;
import static org.powermock.api.mockito.PowerMockito.mock;
import static org.powermock.api.mockito.PowerMockito.mockStatic;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
index 2cb5ada..c954bdb 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
@@ -2089,7 +2089,8 @@
"computation",
defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))),
mockExecutor,
- ImmutableMap.of());
+ ImmutableMap.of(),
+ null);
ByteString key1 = ByteString.copyFromUtf8("key1");
ByteString key2 = ByteString.copyFromUtf8("key2");
@@ -2600,4 +2601,39 @@
assertThat(commit.getSerializedSize(), isWithinBundleSizeLimits);
}
+
+ @Test
+ public void testStuckCommit() throws Exception {
+ if (!streamingEngine) {
+ // Stuck commits have only been observed with streaming engine and thus recovery from them is
+ // not implemented for non-streaming engine.
+ return;
+ }
+
+ List<ParallelInstruction> instructions =
+ Arrays.asList(
+ makeSourceInstruction(StringUtf8Coder.of()),
+ makeSinkInstruction(StringUtf8Coder.of(), 0));
+
+ FakeWindmillServer server = new FakeWindmillServer(errorCollector);
+ StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server);
+ options.setStuckCommitDurationMillis(2000);
+ StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */);
+ worker.start();
+ // Prevent commit callbacks from being called to simulate a stuck commit.
+ server.setDropStreamingCommits(true);
+
+ // Add some work for key 1.
+ server.addWorkToOffer(makeInput(10, TimeUnit.MILLISECONDS.toMicros(2), keyStringForIndex(1)));
+ server.waitForDroppedCommits(1);
+ server.setDropStreamingCommits(false);
+ // Enqueue another work item for key 1.
+ server.addWorkToOffer(makeInput(1, TimeUnit.MILLISECONDS.toMicros(1)));
+ // Ensure that the second work item processes.
+ Map<Long, Windmill.WorkItemCommitRequest> result = server.waitForAndGetCommits(1);
+ worker.stop();
+
+ assertTrue(result.containsKey(1L));
+ assertEquals(makeExpectedOutput(1, TimeUnit.MILLISECONDS.toMicros(1)).build(), result.get(1L));
+ }
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java
index 24b2491..3970724 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java
@@ -17,8 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.fn.control;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonAssert.assertEqualsAsJson;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.when;
@@ -56,7 +57,7 @@
mockSpecValidator, pcollectionNameMapping);
Optional<String> error = Optional.of("Error text");
when(mockSpecValidator.validate(any())).thenReturn(error);
- assertEquals(null, testObject.transform(null));
+ assertNull(testObject.transform(null));
}
@Test
@@ -85,7 +86,7 @@
new ElementCountMonitoringInfoToCounterUpdateTransformer(
mockSpecValidator, pcollectionNameMapping);
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
- assertEquals(null, testObject.transform(monitoringInfo));
+ assertNull(testObject.transform(monitoringInfo));
}
@Test
@@ -106,12 +107,12 @@
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
CounterUpdate result = testObject.transform(monitoringInfo);
- assertNotEquals(null, result);
+ assertNotNull(result);
- assertEquals(
- "{cumulative=true, integer={highBits=0, lowBits=0}, "
- + "nameAndKind={kind=SUM, "
- + "name=transformedValue-ElementCount}}",
- result.toString());
+ assertEqualsAsJson(
+ "{cumulative:true, integer:{highBits:0, lowBits:0}, "
+ + "nameAndKind:{kind:'SUM', "
+ + "name:'transformedValue-ElementCount'}}",
+ result);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java
index 3dec6e1..e89d059 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.dataflow.worker.fn.control;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.when;
@@ -86,6 +87,6 @@
CounterUpdate result = testObject.transform(monitoringInfo);
- assertSame(null, result);
+ assertNull(result);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MSecMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MSecMonitoringInfoToCounterUpdateTransformerTest.java
index 0dbedb7..95a65bc 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MSecMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MSecMonitoringInfoToCounterUpdateTransformerTest.java
@@ -17,8 +17,10 @@
*/
package org.apache.beam.runners.dataflow.worker.fn.control;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonAssert.assertEqualsAsJson;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -68,7 +70,7 @@
MonitoringInfo.newBuilder()
.setUrn("beam:metric:pardo_execution_time:start_bundle_msecs:v1:invalid")
.build();
- assertEquals(null, testObject.transform(monitoringInfo));
+ assertNull(testObject.transform(monitoringInfo));
}
@Test
@@ -142,14 +144,13 @@
CounterUpdate result = testObject.transform(monitoringInfo);
// Validate
- assertNotEquals(null, result);
-
- assertEquals(
- "{cumulative=true, integer={highBits=0, lowBits=0}, "
- + "structuredNameAndMetadata={metadata={kind=SUM}, "
- + "name={executionStepName=anyStageName, name=supportedCounter, origin=SYSTEM, "
- + "originalStepName=anyOriginalName}}}",
- result.toString());
+ assertNotNull(result);
+ assertEqualsAsJson(
+ "{cumulative:true, integer:{highBits:0, lowBits:0}, "
+ + "structuredNameAndMetadata:{metadata:{kind:'SUM'}, "
+ + "name:{executionStepName:'anyStageName', name:'supportedCounter', origin:'SYSTEM', "
+ + "originalStepName:'anyOriginalName'}}}",
+ result);
}
@Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java
index d554b07..e4efa1c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java
@@ -17,8 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.fn.control;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonAssert.assertEqualsAsJson;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.when;
@@ -56,7 +57,7 @@
mockSpecValidator, pcollectionNameMapping);
Optional<String> error = Optional.of("Error text");
when(mockSpecValidator.validate(any())).thenReturn(error);
- assertEquals(null, testObject.transform(null));
+ assertNull(testObject.transform(null));
}
@Test
@@ -85,7 +86,7 @@
new MeanByteCountMonitoringInfoToCounterUpdateTransformer(
mockSpecValidator, pcollectionNameMapping);
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
- assertEquals(null, testObject.transform(monitoringInfo));
+ assertNull(testObject.transform(monitoringInfo));
}
@Test
@@ -107,12 +108,12 @@
CounterUpdate result = testObject.transform(monitoringInfo);
- assertNotEquals(null, result);
- assertEquals(
- "{cumulative=true, integerMean={count={highBits=0, lowBits=0}, "
- + "sum={highBits=0, lowBits=0}}, "
- + "nameAndKind={kind=MEAN, "
- + "name=transformedValue-MeanByteCount}}",
- result.toString());
+ assertNotNull(result);
+ assertEqualsAsJson(
+ "{cumulative:true, integerMean:{count:{highBits:0, lowBits:0}, "
+ + "sum:{highBits:0, lowBits:0}}, "
+ + "nameAndKind:{kind:'MEAN', "
+ + "name:'transformedValue-MeanByteCount'}}",
+ result);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java
index 1d9c9f5..b0b4884 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java
@@ -17,8 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.fn.control;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonAssert.assertEqualsAsJson;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -58,7 +59,7 @@
mockSpecValidator, stepContextMapping);
Optional<String> error = Optional.of("Error text");
when(mockSpecValidator.validate(any())).thenReturn(error);
- assertEquals(null, testObject.transform(null));
+ assertNull(testObject.transform(null));
}
@Test
@@ -89,7 +90,7 @@
new UserDistributionMonitoringInfoToCounterUpdateTransformer(
mockSpecValidator, stepContextMapping);
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
- assertEquals(null, testObject.transform(monitoringInfo));
+ assertNull(testObject.transform(monitoringInfo));
}
@Test
@@ -114,15 +115,15 @@
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
CounterUpdate result = testObject.transform(monitoringInfo);
- assertNotEquals(null, result);
+ assertNotNull(result);
- assertEquals(
- "{cumulative=true, distribution={count={highBits=0, lowBits=0}, "
- + "max={highBits=0, lowBits=0}, min={highBits=0, lowBits=0}, "
- + "sum={highBits=0, lowBits=0}}, "
- + "structuredNameAndMetadata={metadata={kind=DISTRIBUTION}, "
- + "name={name=anyName, origin=USER, originNamespace=anyNamespace, "
- + "originalStepName=anyOriginalName}}}",
- result.toString());
+ assertEqualsAsJson(
+ "{cumulative:true, distribution:{count:{highBits:0, lowBits:0}, "
+ + "max:{highBits:0, lowBits:0}, min:{highBits:0, lowBits:0}, "
+ + "sum:{highBits:0, lowBits:0}}, "
+ + "structuredNameAndMetadata:{metadata:{kind:'DISTRIBUTION'}, "
+ + "name:{name:'anyName', origin:'USER', originNamespace:'anyNamespace', "
+ + "originalStepName:'anyOriginalName'}}}",
+ result);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java
index e2992f5..f24fba0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java
@@ -17,8 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.fn.control;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonAssert.assertEqualsAsJson;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -57,7 +58,7 @@
new UserMonitoringInfoToCounterUpdateTransformer(mockSpecValidator, stepContextMapping);
Optional<String> error = Optional.of("Error text");
when(mockSpecValidator.validate(any())).thenReturn(error);
- assertEquals(null, testObject.transform(null));
+ assertNull(testObject.transform(null));
}
@Test
@@ -86,7 +87,7 @@
UserMonitoringInfoToCounterUpdateTransformer testObject =
new UserMonitoringInfoToCounterUpdateTransformer(mockSpecValidator, stepContextMapping);
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
- assertEquals(null, testObject.transform(monitoringInfo));
+ assertNull(testObject.transform(monitoringInfo));
}
@Test
@@ -110,13 +111,13 @@
when(mockSpecValidator.validate(any())).thenReturn(Optional.empty());
CounterUpdate result = testObject.transform(monitoringInfo);
- assertNotEquals(null, result);
+ assertNotNull(result);
- assertEquals(
- "{cumulative=true, integer={highBits=0, lowBits=0}, "
- + "structuredNameAndMetadata={metadata={kind=SUM}, "
- + "name={name=anyName, origin=USER, originNamespace=anyNamespace, "
- + "originalStepName=anyOriginalName}}}",
- result.toString());
+ assertEqualsAsJson(
+ "{cumulative:true, integer:{highBits:0, lowBits:0}, "
+ + "structuredNameAndMetadata:{metadata:{kind:'SUM'}, "
+ + "name:{name:'anyName', origin:'USER', originNamespace:'anyNamespace', "
+ + "originalStepName:'anyOriginalName'}}}",
+ result);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java
index 5cf303c..d1115c9 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java
@@ -30,8 +30,8 @@
import java.util.Objects;
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.ParDoTranslation;
import org.apache.beam.runners.core.construction.PipelineTranslation;
@@ -194,9 +194,9 @@
&& b instanceof FetchAndFilterStreamingSideInputsNode) {
FetchAndFilterStreamingSideInputsNode nodeA = (FetchAndFilterStreamingSideInputsNode) a;
FetchAndFilterStreamingSideInputsNode nodeB = (FetchAndFilterStreamingSideInputsNode) b;
- Map.Entry<PCollectionView<?>, SdkFunctionSpec> nodeAEntry =
+ Map.Entry<PCollectionView<?>, FunctionSpec> nodeAEntry =
Iterables.getOnlyElement(nodeA.getPCollectionViewsToWindowMappingFns().entrySet());
- Map.Entry<PCollectionView<?>, SdkFunctionSpec> nodeBEntry =
+ Map.Entry<PCollectionView<?>, FunctionSpec> nodeBEntry =
Iterables.getOnlyElement(nodeB.getPCollectionViewsToWindowMappingFns().entrySet());
return Objects.equals(
nodeAEntry.getKey().getTagInternal(), nodeBEntry.getKey().getTagInternal())
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java
index ebe0d4e..a8b37b0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java
@@ -23,9 +23,14 @@
import static org.apache.beam.runners.dataflow.worker.graph.LengthPrefixUnknownCoders.forInstructionOutput;
import static org.apache.beam.runners.dataflow.worker.graph.LengthPrefixUnknownCoders.forInstructionOutputNode;
import static org.apache.beam.runners.dataflow.worker.graph.LengthPrefixUnknownCoders.forParallelInstruction;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonAssert.assertEqualsAsJson;
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonMatcher.jsonOf;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotSame;
+import com.google.api.client.json.GenericJson;
import com.google.api.client.json.jackson.JacksonFactory;
import com.google.api.services.dataflow.model.InstructionOutput;
import com.google.api.services.dataflow.model.ParDoInstruction;
@@ -36,10 +41,8 @@
import com.google.api.services.dataflow.model.Source;
import com.google.api.services.dataflow.model.WriteInstruction;
import java.util.ArrayList;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import org.apache.beam.runners.dataflow.util.CloudObject;
import org.apache.beam.runners.dataflow.util.CloudObjects;
import org.apache.beam.runners.dataflow.worker.graph.Edges.DefaultEdge;
@@ -105,7 +108,7 @@
public void testLengthPrefixUnknownCoders() throws Exception {
Map<String, Object> lengthPrefixedCoderCloudObject =
forCodec(CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null), false);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
lengthPrefixedCoderCloudObject);
}
@@ -126,7 +129,7 @@
KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())),
GlobalWindow.Coder.INSTANCE);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(expectedCoder, /*sdkComponents=*/ null),
lengthPrefixedCoderCloudObject);
}
@@ -141,7 +144,7 @@
Map<String, Object> lengthPrefixedCoderCloudObject =
forCodec(CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null), true);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedAndReplacedWindowedValueCoder, /*sdkComponents=*/ null),
lengthPrefixedCoderCloudObject);
}
@@ -153,11 +156,11 @@
output.setFactory(new JacksonFactory());
InstructionOutput prefixedOutput = forInstructionOutput(output, false);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
prefixedOutput.getCodec());
// Should not mutate the instruction.
- assertEquals(
+ assertEqualsAsJson(
output.getCodec(), CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null));
}
@@ -170,11 +173,11 @@
instruction.setRead(readInstruction);
ParallelInstruction prefixedInstruction = forParallelInstruction(instruction, false);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
prefixedInstruction.getRead().getSource().getCodec());
// Should not mutate the instruction.
- assertEquals(
+ assertEqualsAsJson(
readInstruction.getSource().getCodec(),
CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null));
}
@@ -188,11 +191,11 @@
instruction.setWrite(writeInstruction);
ParallelInstruction prefixedInstruction = forParallelInstruction(instruction, false);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
prefixedInstruction.getWrite().getSink().getCodec());
// Should not mutate the instruction.
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null),
writeInstruction.getSink().getCodec());
}
@@ -208,11 +211,11 @@
instruction.setParDo(parDo);
ParallelInstruction prefixedInstruction = forParallelInstruction(instruction, false);
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
prefixedInstruction.getParDo().getUserFn().get(WorkerPropertyNames.INPUT_CODER));
// Should not mutate the instruction.
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null),
parDo.getUserFn().get(WorkerPropertyNames.INPUT_CODER));
}
@@ -226,7 +229,7 @@
}
@Test
- public void testLengthPrefixAndReplaceForRunnerNetwork() {
+ public void testLengthPrefixAndReplaceForRunnerNetwork() throws Exception {
Node readNode = createReadNode("Read", "Source", windowedValueCoder);
Edge readNodeEdge = DefaultEdge.create();
Node readNodeOut = createInstructionOutputNode("Read.out", windowedValueCoder);
@@ -243,7 +246,7 @@
MutableNetwork<Node, Edge> prefixedNetwork = andReplaceForRunnerNetwork(network);
- Set prefixedInstructions = new HashSet<>();
+ ImmutableSet.Builder<GenericJson> prefixedInstructions = ImmutableSet.builder();
for (Node node : prefixedNetwork.nodes()) {
if (node instanceof ParallelInstructionNode) {
prefixedInstructions.add(((ParallelInstructionNode) node).getParallelInstruction());
@@ -252,11 +255,11 @@
}
}
- Set expectedInstructions =
- ImmutableSet.of(
- prefixedReadNode.getParallelInstruction(), prefixedReadNodeOut.getInstructionOutput());
-
- assertEquals(expectedInstructions, prefixedInstructions);
+ assertThat(
+ prefixedInstructions.build(),
+ containsInAnyOrder(
+ jsonOf(prefixedReadNodeOut.getInstructionOutput()),
+ jsonOf(prefixedReadNode.getParallelInstruction())));
}
@Test
@@ -265,7 +268,7 @@
network.addNode(instructionOutputNode);
network.addNode(grpcPortNode);
network.addEdge(grpcPortNode, instructionOutputNode, DefaultEdge.create());
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
((InstructionOutputNode) forInstructionOutputNode(network).apply(instructionOutputNode))
.getInstructionOutput()
@@ -278,7 +281,7 @@
network.addNode(instructionOutputNode);
network.addNode(grpcPortNode);
network.addEdge(instructionOutputNode, grpcPortNode, DefaultEdge.create());
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null),
((InstructionOutputNode) forInstructionOutputNode(network).apply(instructionOutputNode))
.getInstructionOutput()
@@ -292,7 +295,7 @@
network.addNode(instructionOutputNode);
network.addNode(readNode);
network.addEdge(readNode, instructionOutputNode, DefaultEdge.create());
- assertEquals(
+ assertEqualsAsJson(
CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null),
((InstructionOutputNode) forInstructionOutputNode(network).apply(instructionOutputNode))
.getInstructionOutput()
@@ -306,7 +309,7 @@
ImmutableList.of(
createSideInputInfosWithCoders(windowedValueCoder, prefixedWindowedValueCoder)),
false);
- assertEquals(
+ assertEqualsAsJson(
ImmutableList.of(
createSideInputInfosWithCoders(prefixedWindowedValueCoder, prefixedWindowedValueCoder)),
prefixedSideInputInfos);
@@ -316,7 +319,7 @@
ImmutableList.of(
createSideInputInfosWithCoders(windowedValueCoder, prefixedWindowedValueCoder)),
true);
- assertEquals(
+ assertEqualsAsJson(
ImmutableList.of(
createSideInputInfosWithCoders(
prefixedAndReplacedWindowedValueCoder, prefixedAndReplacedWindowedValueCoder)),
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java
index 3888103..d87ec98 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java
@@ -28,7 +28,6 @@
import org.apache.beam.model.fnexecution.v1.BeamFnApi;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.runners.dataflow.worker.DataflowPortabilityPCollectionView;
import org.apache.beam.runners.dataflow.worker.NameContextsForTests;
import org.apache.beam.runners.dataflow.worker.counters.NameContext;
@@ -146,12 +145,10 @@
@Test
public void testFetchReadySideInputsAndFilterBlockedStreamingSideInputsNode() {
WindowingStrategy windowingStrategy = WindowingStrategy.globalDefault();
- Map<PCollectionView<?>, RunnerApi.SdkFunctionSpec> pcollectionViewsToWindowMappingFns =
+ Map<PCollectionView<?>, RunnerApi.FunctionSpec> pcollectionViewsToWindowMappingFns =
ImmutableMap.of(
mock(PCollectionView.class),
- SdkFunctionSpec.newBuilder()
- .setSpec(FunctionSpec.newBuilder().setUrn("beam:test:urn:1.0"))
- .build());
+ FunctionSpec.newBuilder().setUrn("beam:test:urn:1.0").build());
NameContext nameContext = NameContextsForTests.nameContextForTest();
assertSame(
FetchAndFilterStreamingSideInputsNode.create(
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonAssert.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonAssert.java
new file mode 100644
index 0000000..f4e7268
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonAssert.java
@@ -0,0 +1,51 @@
+/*
+ * 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.runners.dataflow.worker.testing;
+
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import java.io.IOException;
+import org.json.JSONException;
+import org.skyscreamer.jsonassert.JSONAssert;
+
+/** Assertions on {@link GenericJson} class. */
+public class GenericJsonAssert {
+
+ private static final JacksonFactory jacksonFactory = JacksonFactory.getDefaultInstance();
+
+ /**
+ * Asserts that {@code actual} has the same JSON representation as {@code expected}.
+ *
+ * @param expected expected JSON string, {@link GenericJson}, {@link java.util.Map}, or {@link
+ * Iterable} of {@link GenericJson}.
+ * @param actual actual object to compare its JSON representation.
+ */
+ public static void assertEqualsAsJson(Object expected, Object actual) {
+
+ try {
+ String expectedJsonText =
+ expected instanceof String ? (String) expected : jacksonFactory.toString(expected);
+ String actualJsonText = jacksonFactory.toString(actual);
+ JSONAssert.assertEquals(expectedJsonText, actualJsonText, true);
+ } catch (JSONException ex) {
+ throw new IllegalArgumentException("Could not parse JSON", ex);
+ } catch (IOException ex) {
+ throw new IllegalArgumentException("Could not generate JSON text", ex);
+ }
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcher.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcher.java
new file mode 100644
index 0000000..e576a33
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcher.java
@@ -0,0 +1,68 @@
+/*
+ * 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.runners.dataflow.worker.testing;
+
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import java.io.IOException;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+import org.json.JSONException;
+import org.skyscreamer.jsonassert.JSONCompare;
+import org.skyscreamer.jsonassert.JSONCompareMode;
+import org.skyscreamer.jsonassert.JSONCompareResult;
+
+/**
+ * Matcher to compare {@link GenericJson}s using JSONassert's {@link JSONCompare}. This matcher does
+ * not rely on {@link GenericJson#equals(Object)}, which may use fields irrelevant to JSON values.
+ */
+public final class GenericJsonMatcher extends TypeSafeMatcher<GenericJson> {
+
+ private String expectedJsonText;
+
+ private static final JacksonFactory jacksonFactory = JacksonFactory.getDefaultInstance();
+
+ private GenericJsonMatcher(GenericJson expected) {
+ try {
+ expectedJsonText = jacksonFactory.toString(expected);
+ } catch (IOException ex) {
+ throw new IllegalArgumentException("Could not parse JSON", ex);
+ }
+ }
+
+ public static GenericJsonMatcher jsonOf(GenericJson genericJson) {
+ return new GenericJsonMatcher(genericJson);
+ }
+
+ @Override
+ protected boolean matchesSafely(GenericJson actual) {
+ try {
+ String actualJsonText = jacksonFactory.toString(actual);
+ JSONCompareResult result =
+ JSONCompare.compareJSON(expectedJsonText, actualJsonText, JSONCompareMode.STRICT);
+ return result.passed();
+ } catch (IOException | JSONException ex) {
+ throw new IllegalArgumentException("Could not parse JSON", ex);
+ }
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText(expectedJsonText);
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java
new file mode 100644
index 0000000..cac9fe3
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.runners.dataflow.worker.testing;
+
+import static org.apache.beam.runners.dataflow.worker.testing.GenericJsonMatcher.jsonOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import com.google.api.client.json.GenericJson;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link GenericJsonMatcher}. */
+@RunWith(JUnit4.class)
+public class GenericJsonMatcherTest {
+
+ @Test
+ public void testMatch() {
+ GenericJson expected = new GenericJson();
+ expected.set("foo", "bar");
+ GenericJson actual = new GenericJson();
+ actual.set("foo", "bar");
+
+ assertThat(expected, is(jsonOf(actual)));
+ }
+
+ @Test
+ public void testMatchFailure() {
+ GenericJson expected = new GenericJson();
+ expected.set("foo", "expected");
+ GenericJson actual = new GenericJson();
+ actual.set("foo", "actual");
+
+ try {
+ assertThat(actual, is(jsonOf(expected)));
+ } catch (AssertionError ex) {
+ assertEquals(
+ "\nExpected: is {\"foo\":\"expected\"}\n but: was <{foo=actual}>", ex.getMessage());
+
+ // pass
+ return;
+ }
+ fail("The difference in JSON should raise AssertionError");
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java
index 9ee670b..faa4b05 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java
@@ -18,9 +18,9 @@
package org.apache.beam.runners.dataflow.worker.util.common.worker;
import static com.google.api.client.util.Lists.newArrayList;
+import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
@@ -68,8 +68,9 @@
ArrayList<ShuffleEntry> entries = new ArrayList<>();
entries.add(e1);
entries.add(e2);
+ long batchSize = (long) e1.length() + e2.length();
when(batchReader.read(START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(entries, null));
+ .thenReturn(new ShuffleBatchReader.Batch(entries, null, batchSize));
List<ShuffleEntry> results = newArrayList(reader.read(START_POSITION, END_POSITION));
assertThat(results, contains(e1, e2));
}
@@ -81,8 +82,9 @@
ArrayList<ShuffleEntry> entries = new ArrayList<>();
entries.add(e1);
entries.add(e2);
+ long batchSize = (long) e1.length() + e2.length();
when(batchReader.read(START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(entries, null));
+ .thenReturn(new ShuffleBatchReader.Batch(entries, null, batchSize));
Reiterator<ShuffleEntry> it = reader.read(START_POSITION, END_POSITION);
assertThat(it.hasNext(), equalTo(Boolean.TRUE));
assertThat(it.next(), equalTo(e1));
@@ -102,9 +104,9 @@
ShuffleEntry e2 = new ShuffleEntry(KEY, SKEY, VALUE);
List<ShuffleEntry> e2s = Collections.singletonList(e2);
when(batchReader.read(START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(e1s, NEXT_START_POSITION));
+ .thenReturn(new ShuffleBatchReader.Batch(e1s, NEXT_START_POSITION, e1.length()));
when(batchReader.read(NEXT_START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(e2s, null));
+ .thenReturn(new ShuffleBatchReader.Batch(e2s, null, e2.length()));
List<ShuffleEntry> results = newArrayList(reader.read(START_POSITION, END_POSITION));
assertThat(results, contains(e1, e2));
@@ -120,11 +122,11 @@
ShuffleEntry e3 = new ShuffleEntry(KEY, SKEY, VALUE);
List<ShuffleEntry> e3s = Collections.singletonList(e3);
when(batchReader.read(START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(e1s, NEXT_START_POSITION));
+ .thenReturn(new ShuffleBatchReader.Batch(e1s, NEXT_START_POSITION, 0));
when(batchReader.read(NEXT_START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(e2s, SECOND_NEXT_START_POSITION));
+ .thenReturn(new ShuffleBatchReader.Batch(e2s, SECOND_NEXT_START_POSITION, 0));
when(batchReader.read(SECOND_NEXT_START_POSITION, END_POSITION))
- .thenReturn(new ShuffleBatchReader.Batch(e3s, null));
+ .thenReturn(new ShuffleBatchReader.Batch(e3s, null, e3.length()));
List<ShuffleEntry> results = newArrayList(reader.read(START_POSITION, END_POSITION));
assertThat(results, contains(e3));
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReaderTest.java
index 88db3b2..27dbc1d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReaderTest.java
@@ -17,9 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.util.common.worker;
+import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
-import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
@@ -37,7 +37,7 @@
@RunWith(JUnit4.class)
public final class CachingShuffleBatchReaderTest {
private final ShuffleBatchReader.Batch testBatch =
- new ShuffleBatchReader.Batch(new ArrayList<ShuffleEntry>(), null);
+ new ShuffleBatchReader.Batch(new ArrayList<ShuffleEntry>(), null, 0);
@Test
public void readerShouldCacheReads() throws IOException {
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleSplitHandler.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleSplitHandler.java
new file mode 100644
index 0000000..cb03238
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleSplitHandler.java
@@ -0,0 +1,46 @@
+/*
+ * 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.runners.fnexecution.control;
+
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitResponse;
+
+/**
+ * A handler which is invoked whenever an active bundle is split. The active bundle will continue to
+ * keep processing until it is complete.
+ *
+ * <p>The returned split response contains a description of work that has been performed containing
+ * a {@code primary} portion that the SDK is responsible for processing and a {@code residual} which
+ * the runner is responsible for scheduling for future processing. See <a
+ * href="https://s.apache.org/beam-breaking-fusion">breaking the fusion barrier</a> for further
+ * details.
+ */
+public interface BundleSplitHandler {
+ void split(ProcessBundleSplitResponse splitResponse);
+
+ /** Returns a bundle split handler that throws on any split response. */
+ static BundleSplitHandler unsupported() {
+ return new BundleSplitHandler() {
+ @Override
+ public void split(ProcessBundleSplitResponse splitResponse) {
+ throw new UnsupportedOperationException(
+ String.format(
+ "%s does not support splitting.", BundleSplitHandler.class.getSimpleName()));
+ }
+ };
+ };
+}
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
index a5f70c6..006790f 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
@@ -400,6 +400,11 @@
}
@Override
+ public void split(double fractionOfRemainder) {
+ bundle.split(fractionOfRemainder);
+ }
+
+ @Override
public void close() throws Exception {
bundle.close();
currentClient.wrappedClient.unref();
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
index d4932a9..cd81c0a 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
@@ -122,10 +122,12 @@
// The order of these does not matter.
inputDestinationsBuilder.put(
stage.getInputPCollection().getId(),
- addStageInput(dataEndpoint, stage.getInputPCollection(), components));
+ addStageInput(
+ dataEndpoint, stage.getInputPCollection(), components, stage.getWireCoderSetting()));
remoteOutputCodersBuilder.putAll(
- addStageOutputs(dataEndpoint, stage.getOutputPCollections(), components));
+ addStageOutputs(
+ dataEndpoint, stage.getOutputPCollections(), components, stage.getWireCoderSetting()));
Map<String, Map<String, SideInputSpec>> sideInputSpecs = addSideInputs(stage, components);
@@ -189,11 +191,13 @@
private static Map<String, Coder<WindowedValue<?>>> addStageOutputs(
ApiServiceDescriptor dataEndpoint,
Collection<PCollectionNode> outputPCollections,
- Components.Builder components)
+ Components.Builder components,
+ RunnerApi.WireCoderSetting wireCoderSetting)
throws IOException {
Map<String, Coder<WindowedValue<?>>> remoteOutputCoders = new LinkedHashMap<>();
for (PCollectionNode outputPCollection : outputPCollections) {
- OutputEncoding outputEncoding = addStageOutput(dataEndpoint, components, outputPCollection);
+ OutputEncoding outputEncoding =
+ addStageOutput(dataEndpoint, components, outputPCollection, wireCoderSetting);
remoteOutputCoders.put(outputEncoding.getPTransformId(), outputEncoding.getCoder());
}
return remoteOutputCoders;
@@ -202,12 +206,16 @@
private static RemoteInputDestination<WindowedValue<?>> addStageInput(
ApiServiceDescriptor dataEndpoint,
PCollectionNode inputPCollection,
- Components.Builder components)
+ Components.Builder components,
+ RunnerApi.WireCoderSetting wireCoderSetting)
throws IOException {
- String inputWireCoderId = WireCoders.addSdkWireCoder(inputPCollection, components);
+ String inputWireCoderId =
+ WireCoders.addSdkWireCoder(inputPCollection, components, wireCoderSetting);
@SuppressWarnings("unchecked")
Coder<WindowedValue<?>> wireCoder =
- (Coder) WireCoders.instantiateRunnerWireCoder(inputPCollection, components.build());
+ (Coder)
+ WireCoders.instantiateRunnerWireCoder(
+ inputPCollection, components.build(), wireCoderSetting);
RemoteGrpcPort inputPort =
RemoteGrpcPort.newBuilder()
@@ -226,12 +234,16 @@
private static OutputEncoding addStageOutput(
ApiServiceDescriptor dataEndpoint,
Components.Builder components,
- PCollectionNode outputPCollection)
+ PCollectionNode outputPCollection,
+ RunnerApi.WireCoderSetting wireCoderSetting)
throws IOException {
- String outputWireCoderId = WireCoders.addSdkWireCoder(outputPCollection, components);
+ String outputWireCoderId =
+ WireCoders.addSdkWireCoder(outputPCollection, components, wireCoderSetting);
@SuppressWarnings("unchecked")
Coder<WindowedValue<?>> wireCoder =
- (Coder) WireCoders.instantiateRunnerWireCoder(outputPCollection, components.build());
+ (Coder)
+ WireCoders.instantiateRunnerWireCoder(
+ outputPCollection, components.build(), wireCoderSetting);
RemoteGrpcPort outputPort =
RemoteGrpcPort.newBuilder()
.setApiServiceDescriptor(dataEndpoint)
@@ -385,7 +397,8 @@
addStageInput(
dataEndpoint,
PipelineNode.pCollection(inputTimerPCollectionId, timerCollectionSpec),
- components));
+ components,
+ stage.getWireCoderSetting()));
String outputTimerPCollectionId =
SyntheticComponents.uniqueId(
String.format(
@@ -397,7 +410,8 @@
addStageOutput(
dataEndpoint,
components,
- PipelineNode.pCollection(outputTimerPCollectionId, timerCollectionSpec));
+ PipelineNode.pCollection(outputTimerPCollectionId, timerCollectionSpec),
+ stage.getWireCoderSetting());
outputTransformCodersBuilder.put(outputEncoding.getPTransformId(), outputEncoding.getCoder());
components.putTransforms(
timerReference.transform().getId(),
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java
index 30fea85..cd7ac11 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java
@@ -39,6 +39,15 @@
Map<String, FnDataReceiver> getInputReceivers();
/**
+ * Ask the remote bundle to split its current processing based upon its knowledge of remaining
+ * work. A fraction of 0, is equivalent to asking the SDK to checkpoint.
+ *
+ * <p>This method will return after the request has been issued. Any splits will be forwarded to
+ * the {@link BundleSplitHandler}.
+ */
+ void split(double fractionOfRemainder);
+
+ /**
* Closes this bundle. This causes the input {@link FnDataReceiver} to be closed (future calls to
* that {@link FnDataReceiver} will throw an exception), and causes the {@link RemoteBundle} to
* produce any buffered outputs. The call to {@link #close()} will block until all of the outputs
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
index 2799e58..08d50b0 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
@@ -25,10 +25,13 @@
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleRequest;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleSplitRequest.DesiredSplit;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.RegisterResponse;
import org.apache.beam.model.pipeline.v1.Endpoints;
import org.apache.beam.runners.fnexecution.data.FnDataService;
@@ -140,6 +143,7 @@
outputReceivers,
stateRequestHandler,
progressHandler,
+ BundleSplitHandler.unsupported(),
request -> {
throw new UnsupportedOperationException(
String.format(
@@ -174,6 +178,7 @@
Map<String, RemoteOutputReceiver<?>> outputReceivers,
StateRequestHandler stateRequestHandler,
BundleProgressHandler progressHandler,
+ BundleSplitHandler splitHandler,
BundleCheckpointHandler checkpointHandler,
BundleFinalizationHandler finalizationHandler) {
String bundleId = idGenerator.getId();
@@ -205,14 +210,15 @@
outputClients.put(receiver.getKey(), outputClient);
}
- ImmutableMap.Builder<String, CloseableFnDataReceiver> dataReceiversBuilder =
+ ImmutableMap.Builder<String, CountingFnDataReceiver> dataReceiversBuilder =
ImmutableMap.builder();
for (Map.Entry<String, RemoteInputDestination> remoteInput : remoteInputs.entrySet()) {
dataReceiversBuilder.put(
remoteInput.getKey(),
- fnApiDataService.send(
- LogicalEndpoint.of(bundleId, remoteInput.getValue().getPTransformId()),
- (Coder) remoteInput.getValue().getCoder()));
+ new CountingFnDataReceiver(
+ fnApiDataService.send(
+ LogicalEndpoint.of(bundleId, remoteInput.getValue().getPTransformId()),
+ (Coder) remoteInput.getValue().getCoder())));
}
return new ActiveBundle(
@@ -222,6 +228,7 @@
outputClients,
stateDelegator.registerForProcessBundleInstructionId(bundleId, stateRequestHandler),
progressHandler,
+ splitHandler,
checkpointHandler,
finalizationHandler);
}
@@ -231,122 +238,122 @@
return fnApiDataService.receive(
LogicalEndpoint.of(bundleId, ptransformId), receiver.getCoder(), receiver.getReceiver());
}
- }
- /** An active bundle for a particular {@link BeamFnApi.ProcessBundleDescriptor}. */
- public static class ActiveBundle implements RemoteBundle {
- private final String bundleId;
- private final CompletionStage<BeamFnApi.ProcessBundleResponse> response;
- private final Map<String, CloseableFnDataReceiver> inputReceivers;
- private final Map<String, InboundDataClient> outputClients;
- private final StateDelegator.Registration stateRegistration;
- private final BundleProgressHandler progressHandler;
- private final BundleCheckpointHandler checkpointHandler;
- private final BundleFinalizationHandler finalizationHandler;
+ /** An active bundle for a particular {@link BeamFnApi.ProcessBundleDescriptor}. */
+ public class ActiveBundle implements RemoteBundle {
+ private final String bundleId;
+ private final CompletionStage<BeamFnApi.ProcessBundleResponse> response;
+ private final Map<String, CountingFnDataReceiver> inputReceivers;
+ private final Map<String, InboundDataClient> outputClients;
+ private final StateDelegator.Registration stateRegistration;
+ private final BundleProgressHandler progressHandler;
+ private final BundleSplitHandler splitHandler;
+ private final BundleCheckpointHandler checkpointHandler;
+ private final BundleFinalizationHandler finalizationHandler;
- private ActiveBundle(
- String bundleId,
- CompletionStage<ProcessBundleResponse> response,
- Map<String, CloseableFnDataReceiver> inputReceivers,
- Map<String, InboundDataClient> outputClients,
- StateDelegator.Registration stateRegistration,
- BundleProgressHandler progressHandler,
- BundleCheckpointHandler checkpointHandler,
- BundleFinalizationHandler finalizationHandler) {
- this.bundleId = bundleId;
- this.response = response;
- this.inputReceivers = inputReceivers;
- this.outputClients = outputClients;
- this.stateRegistration = stateRegistration;
- this.progressHandler = progressHandler;
- this.checkpointHandler = checkpointHandler;
- this.finalizationHandler = finalizationHandler;
- }
+ private ActiveBundle(
+ String bundleId,
+ CompletionStage<ProcessBundleResponse> response,
+ Map<String, CountingFnDataReceiver> inputReceivers,
+ Map<String, InboundDataClient> outputClients,
+ StateDelegator.Registration stateRegistration,
+ BundleProgressHandler progressHandler,
+ BundleSplitHandler splitHandler,
+ BundleCheckpointHandler checkpointHandler,
+ BundleFinalizationHandler finalizationHandler) {
+ this.bundleId = bundleId;
+ this.response = response;
+ this.inputReceivers = inputReceivers;
+ this.outputClients = outputClients;
+ this.stateRegistration = stateRegistration;
+ this.progressHandler = progressHandler;
+ this.splitHandler = splitHandler;
+ this.checkpointHandler = checkpointHandler;
+ this.finalizationHandler = finalizationHandler;
+ }
- /** Returns an id used to represent this bundle. */
- @Override
- public String getId() {
- return bundleId;
- }
+ /** Returns an id used to represent this bundle. */
+ @Override
+ public String getId() {
+ return bundleId;
+ }
- /**
- * Get a map of PCollection ids to {@link FnDataReceiver receiver}s which consume input
- * elements, forwarding them to the remote environment.
- */
- @Override
- public Map<String, FnDataReceiver> getInputReceivers() {
- return (Map) inputReceivers;
- }
+ /**
+ * Get a map of PCollection ids to {@link FnDataReceiver receiver}s which consume input
+ * elements, forwarding them to the remote environment.
+ */
+ @Override
+ public Map<String, FnDataReceiver> getInputReceivers() {
+ return (Map) inputReceivers;
+ }
- /**
- * Blocks until bundle processing is finished. This is comprised of:
- *
- * <ul>
- * <li>closing each {@link #getInputReceivers() input receiver}.
- * <li>waiting for the SDK to say that processing the bundle is finished.
- * <li>waiting for all inbound data clients to complete
- * </ul>
- *
- * <p>This method will throw an exception if bundle processing has failed. {@link
- * Throwable#getSuppressed()} will return all the reasons as to why processing has failed.
- */
- @Override
- public void close() throws Exception {
- Exception exception = null;
- for (CloseableFnDataReceiver<?> inputReceiver : inputReceivers.values()) {
+ @Override
+ public void split(double fractionOfRemainder) {
+ Map<String, DesiredSplit> splits = new HashMap<>();
+ for (Map.Entry<String, CountingFnDataReceiver> ptransformToInput :
+ inputReceivers.entrySet()) {
+ splits.put(
+ ptransformToInput.getKey(),
+ DesiredSplit.newBuilder()
+ .setFractionOfRemainder(fractionOfRemainder)
+ .setEstimatedInputElements(ptransformToInput.getValue().getCount())
+ .build());
+ }
+ InstructionRequest request =
+ InstructionRequest.newBuilder()
+ .setInstructionId(idGenerator.getId())
+ .setProcessBundleSplit(
+ ProcessBundleSplitRequest.newBuilder()
+ .setInstructionId(bundleId)
+ .putAllDesiredSplits(splits)
+ .build())
+ .build();
+ CompletionStage<InstructionResponse> response = fnApiControlClient.handle(request);
+ response.thenAccept(
+ instructionResponse -> splitHandler.split(instructionResponse.getProcessBundleSplit()));
+ }
+
+ /**
+ * Blocks until bundle processing is finished. This is comprised of:
+ *
+ * <ul>
+ * <li>closing each {@link #getInputReceivers() input receiver}.
+ * <li>waiting for the SDK to say that processing the bundle is finished.
+ * <li>waiting for all inbound data clients to complete
+ * </ul>
+ *
+ * <p>This method will throw an exception if bundle processing has failed. {@link
+ * Throwable#getSuppressed()} will return all the reasons as to why processing has failed.
+ */
+ @Override
+ public void close() throws Exception {
+ Exception exception = null;
+ for (CloseableFnDataReceiver<?> inputReceiver : inputReceivers.values()) {
+ try {
+ inputReceiver.close();
+ } catch (Exception e) {
+ if (exception == null) {
+ exception = e;
+ } else {
+ exception.addSuppressed(e);
+ }
+ }
+ }
try {
- inputReceiver.close();
- } catch (Exception e) {
+ // We don't have to worry about the completion stage.
if (exception == null) {
- exception = e;
+ BeamFnApi.ProcessBundleResponse completedResponse = MoreFutures.get(response);
+ progressHandler.onCompleted(completedResponse);
+ if (completedResponse.getResidualRootsCount() > 0) {
+ checkpointHandler.onCheckpoint(completedResponse);
+ }
+ if (completedResponse.getRequiresFinalization()) {
+ finalizationHandler.requestsFinalization(bundleId);
+ }
} else {
- exception.addSuppressed(e);
- }
- }
- }
- try {
- // We don't have to worry about the completion stage.
- if (exception == null) {
- BeamFnApi.ProcessBundleResponse completedResponse = MoreFutures.get(response);
- progressHandler.onCompleted(completedResponse);
- if (completedResponse.getResidualRootsCount() > 0) {
- checkpointHandler.onCheckpoint(completedResponse);
- }
- if (completedResponse.getRequiresFinalization()) {
- finalizationHandler.requestsFinalization(bundleId);
- }
- } else {
- // TODO: [BEAM-3962] Handle aborting the bundle being processed.
- throw new IllegalStateException(
- "Processing bundle failed, TODO: [BEAM-3962] abort bundle.");
- }
- } catch (Exception e) {
- if (exception == null) {
- exception = e;
- } else {
- exception.addSuppressed(e);
- }
- }
- try {
- if (exception == null) {
- stateRegistration.deregister();
- } else {
- stateRegistration.abort();
- }
- } catch (Exception e) {
- if (exception == null) {
- exception = e;
- } else {
- exception.addSuppressed(e);
- }
- }
- for (InboundDataClient outputClient : outputClients.values()) {
- try {
- // If we failed processing this bundle, we should cancel all inbound data.
- if (exception == null) {
- outputClient.awaitCompletion();
- } else {
- outputClient.cancel();
+ // TODO: [BEAM-3962] Handle aborting the bundle being processed.
+ throw new IllegalStateException(
+ "Processing bundle failed, TODO: [BEAM-3962] abort bundle.");
}
} catch (Exception e) {
if (exception == null) {
@@ -355,9 +362,37 @@
exception.addSuppressed(e);
}
}
- }
- if (exception != null) {
- throw exception;
+ try {
+ if (exception == null) {
+ stateRegistration.deregister();
+ } else {
+ stateRegistration.abort();
+ }
+ } catch (Exception e) {
+ if (exception == null) {
+ exception = e;
+ } else {
+ exception.addSuppressed(e);
+ }
+ }
+ for (InboundDataClient outputClient : outputClients.values()) {
+ try {
+ if (exception == null) {
+ outputClient.awaitCompletion();
+ } else {
+ outputClient.cancel();
+ }
+ } catch (Exception e) {
+ if (exception == null) {
+ exception = e;
+ } else {
+ exception.addSuppressed(e);
+ }
+ }
+ }
+ if (exception != null) {
+ throw exception;
+ }
}
}
}
@@ -476,6 +511,38 @@
}
}
+ /**
+ * A {@link CloseableFnDataReceiver} which counts the number of elements that have been accepted.
+ */
+ private static class CountingFnDataReceiver<T> implements CloseableFnDataReceiver<T> {
+ private final CloseableFnDataReceiver delegate;
+ private long count;
+
+ private CountingFnDataReceiver(CloseableFnDataReceiver delegate) {
+ this.delegate = delegate;
+ }
+
+ public long getCount() {
+ return count;
+ }
+
+ @Override
+ public void accept(T input) throws Exception {
+ count += 1;
+ delegate.accept(input);
+ }
+
+ @Override
+ public void flush() throws Exception {
+ delegate.flush();
+ }
+
+ @Override
+ public void close() throws Exception {
+ delegate.close();
+ }
+ }
+
/** Registers a {@link BeamFnApi.ProcessBundleDescriptor} for future processing. */
private BundleProcessor create(
BeamFnApi.ProcessBundleDescriptor processBundleDescriptor,
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java
index 154aaec..5ec4a39 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java
@@ -159,25 +159,31 @@
containerId = docker.runImage(containerImage, dockerOptsBuilder.build(), argsBuilder.build());
LOG.debug("Created Docker Container with Container ID {}", containerId);
// Wait on a client from the gRPC server.
- try {
- instructionHandler = clientSource.take(workerId, Duration.ofMinutes(1));
- } catch (TimeoutException timeoutEx) {
- RuntimeException runtimeException =
- new RuntimeException(
- String.format(
- "Docker container %s failed to start up successfully within 1 minute.",
- containerImage),
- timeoutEx);
+ while (instructionHandler == null) {
try {
- String containerLogs = docker.getContainerLogs(containerId);
- LOG.error("Docker container {} logs:\n{}", containerId, containerLogs);
- } catch (Exception getLogsException) {
- runtimeException.addSuppressed(getLogsException);
+ // If the docker is not alive anymore, we abort.
+ if (!docker.isContainerRunning(containerId)) {
+ IllegalStateException illegalStateException =
+ new IllegalStateException(
+ String.format("No container running for id %s", containerId));
+ try {
+ String containerLogs = docker.getContainerLogs(containerId);
+ LOG.error("Docker container {} logs:\n{}", containerId, containerLogs);
+ } catch (Exception getLogsException) {
+ illegalStateException.addSuppressed(getLogsException);
+ }
+ throw illegalStateException;
+ }
+ instructionHandler = clientSource.take(workerId, Duration.ofSeconds(5));
+ } catch (TimeoutException timeoutEx) {
+ LOG.info(
+ "Still waiting for startup of environment {} for worker id {}",
+ dockerPayload.getContainerImage(),
+ workerId);
+ } catch (InterruptedException interruptEx) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(interruptEx);
}
- throw runtimeException;
- } catch (InterruptedException interruptEx) {
- Thread.currentThread().interrupt();
- throw new RuntimeException(interruptEx);
}
} catch (Exception e) {
if (containerId != null) {
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java
index a696770..a06a5d7 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java
@@ -23,6 +23,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
import org.apache.beam.fn.harness.FnHarness;
import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
@@ -123,8 +124,23 @@
}
});
- // TODO: find some way to populate the actual ID in FnHarness.main()
- InstructionRequestHandler handler = clientSource.take("", Duration.ofMinutes(1L));
+ InstructionRequestHandler handler = null;
+ // Wait on a client from the gRPC server.
+ while (handler == null) {
+ try {
+ // If the thread is not alive anymore, we abort.
+ if (executor.isShutdown()) {
+ throw new IllegalStateException("FnHarness startup failed");
+ }
+ // TODO: find some way to populate the actual ID in FnHarness.main()
+ handler = clientSource.take("", Duration.ofSeconds(5L));
+ } catch (TimeoutException timeoutEx) {
+ LOG.info("Still waiting for startup of FnHarness");
+ } catch (InterruptedException interruptEx) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(interruptEx);
+ }
+ }
return RemoteEnvironment.forHandler(environment, handler);
}
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java
index a90a245..539d76a 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java
@@ -135,7 +135,7 @@
try {
// If the process is not alive anymore, we abort.
process.isAliveOrThrow();
- instructionHandler = clientSource.take(workerId, Duration.ofMinutes(2));
+ instructionHandler = clientSource.take(workerId, Duration.ofSeconds(5));
} catch (TimeoutException timeoutEx) {
LOG.info(
"Still waiting for startup of environment '{}' for worker id {}",
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java
index 65fcdf2..f1d88d0 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java
@@ -47,19 +47,12 @@
/** A list of all managers to ensure all processes shutdown on JVM exit . */
private static final List<ProcessManager> ALL_PROCESS_MANAGERS = new ArrayList<>();
- static {
- // Install a shutdown hook to ensure processes are stopped/killed.
- Runtime.getRuntime().addShutdownHook(ShutdownHook.create());
- }
+ @VisibleForTesting static Thread shutdownHook = null;
private final Map<String, Process> processes;
public static ProcessManager create() {
- synchronized (ALL_PROCESS_MANAGERS) {
- ProcessManager processManager = new ProcessManager();
- ALL_PROCESS_MANAGERS.add(processManager);
- return processManager;
- }
+ return new ProcessManager();
}
private ProcessManager() {
@@ -126,6 +119,7 @@
return startProcess(id, command, args, env, outputFile);
}
+ @SuppressFBWarnings("ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD")
public RunningProcess startProcess(
String id, String command, List<String> args, Map<String, String> env, File outputFile)
throws IOException {
@@ -149,6 +143,15 @@
LOG.debug("Attempting to start process with command: {}", pb.command());
Process newProcess = pb.start();
Process oldProcess = processes.put(id, newProcess);
+ synchronized (ALL_PROCESS_MANAGERS) {
+ if (!ALL_PROCESS_MANAGERS.contains(this)) {
+ ALL_PROCESS_MANAGERS.add(this);
+ }
+ if (shutdownHook == null) {
+ shutdownHook = ShutdownHook.create();
+ Runtime.getRuntime().addShutdownHook(shutdownHook);
+ }
+ }
if (oldProcess != null) {
stopProcess(id, oldProcess);
stopProcess(id, newProcess);
@@ -159,10 +162,23 @@
}
/** Stops a previously started process identified by its unique id. */
+ @SuppressFBWarnings("ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD")
public void stopProcess(String id) {
checkNotNull(id, "Process id must not be null");
- Process process = checkNotNull(processes.remove(id), "Process for id does not exist: " + id);
- stopProcess(id, process);
+ try {
+ Process process = checkNotNull(processes.remove(id), "Process for id does not exist: " + id);
+ stopProcess(id, process);
+ } finally {
+ synchronized (ALL_PROCESS_MANAGERS) {
+ if (processes.isEmpty()) {
+ ALL_PROCESS_MANAGERS.remove(this);
+ }
+ if (ALL_PROCESS_MANAGERS.isEmpty() && shutdownHook != null) {
+ Runtime.getRuntime().removeShutdownHook(shutdownHook);
+ shutdownHook = null;
+ }
+ }
+ }
}
private void stopProcess(String id, Process process) {
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java
index 38a885f..920dae6 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java
@@ -44,7 +44,7 @@
import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Timestamps;
+import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Durations;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.joda.time.Instant;
@@ -152,7 +152,8 @@
holdState.add(watermarkHold);
Instant requestedWakeupTime =
- new Instant(Timestamps.toMillis(residual.getRequestedExecutionTime()));
+ new Instant(
+ System.currentTimeMillis() + Durations.toMillis(residual.getRequestedTimeDelay()));
Instant wakeupTime =
timerInternals.currentProcessingTime().isBefore(requestedWakeupTime)
? requestedWakeupTime
@@ -160,7 +161,12 @@
// Set a timer to continue processing this element.
timerInternals.setTimer(
- stateNamespace, "sdfContinuation", wakeupTime, TimeDomain.PROCESSING_TIME);
+ stateNamespace,
+ "sdfContinuation",
+ "sdfContinuation",
+ wakeupTime,
+ wakeupTime,
+ TimeDomain.PROCESSING_TIME);
}
/** Signals that a split happened. */
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java
index 46d894a..f37901b 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java
@@ -17,6 +17,8 @@
*/
package org.apache.beam.runners.fnexecution.wire;
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import java.io.IOException;
@@ -41,8 +43,10 @@
* @return id of a windowed value coder containing the PCollection's element coder
*/
public static String addSdkWireCoder(
- PCollectionNode pCollectionNode, RunnerApi.Components.Builder components) {
- return addWireCoder(pCollectionNode, components, false);
+ PCollectionNode pCollectionNode,
+ RunnerApi.Components.Builder components,
+ RunnerApi.WireCoderSetting wireCoderSetting) {
+ return addWireCoder(pCollectionNode, components, false, wireCoderSetting);
}
/**
@@ -54,22 +58,38 @@
* @return id of a windowed value coder containing the PCollection's element coder
*/
public static String addRunnerWireCoder(
- PCollectionNode pCollectionNode, RunnerApi.Components.Builder components) {
- return addWireCoder(pCollectionNode, components, true);
+ PCollectionNode pCollectionNode,
+ RunnerApi.Components.Builder components,
+ RunnerApi.WireCoderSetting wireCoderSetting) {
+ return addWireCoder(pCollectionNode, components, true, wireCoderSetting);
}
/**
* Instantiates a runner-side wire coder for the given PCollection. Any component coders that are
* unknown by the runner are replaced with length-prefixed byte arrays.
*
- * @return a windowed value coder containing the PCollection's element coder
+ * @return a full windowed value coder containing the PCollection's element coder
*/
public static <T> Coder<WindowedValue<T>> instantiateRunnerWireCoder(
PCollectionNode pCollectionNode, RunnerApi.Components components) throws IOException {
+ return instantiateRunnerWireCoder(pCollectionNode, components, DEFAULT_WIRE_CODER_SETTING);
+ }
+
+ /**
+ * Instantiates a runner-side wire coder for the given PCollection. Any component coders that are
+ * unknown by the runner are replaced with length-prefixed byte arrays.
+ *
+ * @return a full or parameterized windowed value coder containing the PCollection's element coder
+ */
+ public static <T> Coder<WindowedValue<T>> instantiateRunnerWireCoder(
+ PCollectionNode pCollectionNode,
+ RunnerApi.Components components,
+ RunnerApi.WireCoderSetting wireCoderSetting)
+ throws IOException {
// NOTE: We discard the new set of components so we don't bother to ensure it's consistent with
// the caller's view.
RunnerApi.Components.Builder builder = components.toBuilder();
- String protoCoderId = addRunnerWireCoder(pCollectionNode, builder);
+ String protoCoderId = addRunnerWireCoder(pCollectionNode, builder, wireCoderSetting);
Coder<?> javaCoder = RehydratedComponents.forComponents(builder.build()).getCoder(protoCoderId);
checkArgument(
javaCoder instanceof WindowedValue.FullWindowedValueCoder,
@@ -83,13 +103,31 @@
private static String addWireCoder(
PCollectionNode pCollectionNode,
RunnerApi.Components.Builder components,
- boolean useByteArrayCoder) {
+ boolean useByteArrayCoder,
+ RunnerApi.WireCoderSetting wireCoderSetting) {
String elementCoderId = pCollectionNode.getPCollection().getCoderId();
String windowingStrategyId = pCollectionNode.getPCollection().getWindowingStrategyId();
String windowCoderId =
components.getWindowingStrategiesOrThrow(windowingStrategyId).getWindowCoderId();
- RunnerApi.Coder windowedValueCoder =
- ModelCoders.windowedValueCoder(elementCoderId, windowCoderId);
+
+ // decide type of windowedValueCoder according to the wire coder setting.
+ RunnerApi.Coder windowedValueCoder;
+ String wireCoderUrn = wireCoderSetting.getUrn();
+ if (wireCoderUrn.equals(getUrn(RunnerApi.StandardCoders.Enum.WINDOWED_VALUE))
+ || wireCoderUrn.isEmpty()) {
+ windowedValueCoder = ModelCoders.windowedValueCoder(elementCoderId, windowCoderId);
+ } else {
+ checkArgument(
+ wireCoderUrn.equals(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE)),
+ "Unexpected wire coder urn %s, currently only %s or %s are supported!",
+ wireCoderUrn,
+ getUrn(RunnerApi.StandardCoders.Enum.WINDOWED_VALUE),
+ getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE));
+ windowedValueCoder =
+ ModelCoders.paramWindowedValueCoder(
+ elementCoderId, windowCoderId, wireCoderSetting.getPayload().toByteArray());
+ }
+
// Add the original WindowedValue<T, W> coder to the components;
String windowedValueId =
SyntheticComponents.uniqueId(
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
index dc73203..d9d51d4 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
@@ -60,7 +60,6 @@
import org.apache.beam.runners.fnexecution.GrpcFnServer;
import org.apache.beam.runners.fnexecution.InProcessServerFactory;
import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.ExecutableProcessBundleDescriptor;
-import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.ActiveBundle;
import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor;
import org.apache.beam.runners.fnexecution.data.GrpcDataService;
import org.apache.beam.runners.fnexecution.logging.GrpcLoggingService;
@@ -286,7 +285,7 @@
}
// The impulse example
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(outputReceivers, BundleProgressHandler.ignored())) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
.accept(WindowedValue.valueInGlobalWindow(new byte[0]));
@@ -350,7 +349,7 @@
(FnDataReceiver<? super WindowedValue<?>>) outputContents::add));
}
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(outputReceivers, BundleProgressHandler.ignored())) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
.accept(
@@ -359,7 +358,7 @@
}
try {
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(outputReceivers, BundleProgressHandler.ignored())) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
.accept(
@@ -372,7 +371,7 @@
assertTrue(e.getMessage().contains("testBundleExecutionFailure"));
}
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(outputReceivers, BundleProgressHandler.ignored())) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
.accept(
@@ -504,7 +503,7 @@
});
BundleProgressHandler progressHandler = BundleProgressHandler.ignored();
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(outputReceivers, stateRequestHandler, progressHandler)) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
.accept(WindowedValue.valueInGlobalWindow("X"));
@@ -818,7 +817,7 @@
}
};
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(outputReceivers, stateRequestHandler, progressHandler)) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
.accept(
@@ -959,7 +958,7 @@
}
});
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(
outputReceivers, stateRequestHandler, BundleProgressHandler.ignored())) {
Iterables.getOnlyElement(bundle.getInputReceivers().values())
@@ -1102,7 +1101,7 @@
// output.
DateTimeUtils.setCurrentMillisFixed(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis());
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(
outputReceivers, StateRequestHandler.unsupported(), BundleProgressHandler.ignored())) {
bundle
@@ -1222,7 +1221,7 @@
(Coder<WindowedValue<?>>) remoteOutputCoder.getValue(), outputValues::add));
}
- try (ActiveBundle bundle =
+ try (RemoteBundle bundle =
processor.newBundle(
outputReceivers,
StateRequestHandler.unsupported(),
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
index ef0b2ac..57b5cbb 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
@@ -53,8 +53,8 @@
import org.apache.beam.runners.core.construction.CoderTranslation;
import org.apache.beam.runners.core.construction.PipelineTranslation;
import org.apache.beam.runners.fnexecution.EmbeddedSdkHarness;
-import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.ActiveBundle;
import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor;
+import org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor.ActiveBundle;
import org.apache.beam.runners.fnexecution.data.FnDataService;
import org.apache.beam.runners.fnexecution.data.RemoteInputDestination;
import org.apache.beam.runners.fnexecution.state.StateDelegator;
@@ -228,7 +228,7 @@
(FullWindowedValueCoder) coder, SDK_GRPC_READ_TRANSFORM)));
when(dataService.send(any(), eq(coder))).thenReturn(mock(CloseableFnDataReceiver.class));
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(Collections.emptyMap(), BundleProgressHandler.ignored())) {
// Correlating the ProcessBundleRequest and ProcessBundleResponse is owned by the underlying
// FnApiControlClient. The SdkHarnessClient owns just wrapping the request and unwrapping
@@ -256,7 +256,7 @@
SDK_GRPC_READ_TRANSFORM)));
Collection<WindowedValue<String>> outputs = new ArrayList<>();
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
Collections.singletonMap(
SDK_GRPC_WRITE_TRANSFORM,
@@ -311,7 +311,7 @@
BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
try {
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
mockProgressHandler)) {
@@ -363,7 +363,7 @@
RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
try {
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
mockStateHandler,
@@ -408,7 +408,7 @@
BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
try {
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
mockProgressHandler)) {
@@ -457,7 +457,7 @@
RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
try {
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
mockStateHandler,
@@ -503,7 +503,7 @@
BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
try {
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
mockProgressHandler)) {
@@ -559,7 +559,7 @@
RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
try {
- try (ActiveBundle activeBundle =
+ try (RemoteBundle activeBundle =
processor.newBundle(
ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
mockStateHandler,
@@ -649,6 +649,7 @@
RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
+ BundleSplitHandler mockSplitHandler = mock(BundleSplitHandler.class);
BundleCheckpointHandler mockCheckpointHandler = mock(BundleCheckpointHandler.class);
BundleFinalizationHandler mockFinalizationHandler = mock(BundleFinalizationHandler.class);
@@ -663,6 +664,7 @@
throw new UnsupportedOperationException();
},
mockProgressHandler,
+ mockSplitHandler,
mockCheckpointHandler,
mockFinalizationHandler)) {
processBundleResponseFuture.complete(
@@ -671,7 +673,7 @@
verify(mockProgressHandler).onCompleted(response);
verify(mockCheckpointHandler).onCheckpoint(response);
- verifyZeroInteractions(mockFinalizationHandler);
+ verifyZeroInteractions(mockFinalizationHandler, mockSplitHandler);
}
@Test
@@ -698,6 +700,7 @@
RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
+ BundleSplitHandler mockSplitHandler = mock(BundleSplitHandler.class);
BundleCheckpointHandler mockCheckpointHandler = mock(BundleCheckpointHandler.class);
BundleFinalizationHandler mockFinalizationHandler = mock(BundleFinalizationHandler.class);
@@ -711,6 +714,7 @@
throw new UnsupportedOperationException();
},
mockProgressHandler,
+ mockSplitHandler,
mockCheckpointHandler,
mockFinalizationHandler)) {
bundleId = activeBundle.getId();
@@ -720,7 +724,7 @@
verify(mockProgressHandler).onCompleted(response);
verify(mockFinalizationHandler).requestsFinalization(bundleId);
- verifyZeroInteractions(mockCheckpointHandler);
+ verifyZeroInteractions(mockCheckpointHandler, mockSplitHandler);
}
private CompletableFuture<InstructionResponse> createRegisterResponse() {
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
index a199cb2..f163640 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
@@ -20,14 +20,13 @@
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.any;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.util.Arrays;
import java.util.Collection;
-import java.util.concurrent.TimeoutException;
import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
import org.apache.beam.runners.core.construction.Environments;
@@ -158,15 +157,11 @@
}
@Test(expected = RuntimeException.class)
- public void logsDockerOutputOnTimeoutException() throws Exception {
+ public void logsDockerOutputOnStartupFailed() throws Exception {
when(docker.runImage(Mockito.eq(IMAGE_NAME), Mockito.any(), Mockito.any()))
.thenReturn(CONTAINER_ID);
- when(docker.isContainerRunning(Mockito.eq(CONTAINER_ID))).thenReturn(true);
- DockerEnvironmentFactory factory =
- getFactory(
- (workerId, timeout) -> {
- throw new TimeoutException();
- });
+ when(docker.isContainerRunning(Mockito.eq(CONTAINER_ID))).thenReturn(false);
+ DockerEnvironmentFactory factory = getFactory((workerId, timeout) -> client);
factory.createEnvironment(ENVIRONMENT);
@@ -188,7 +183,7 @@
@Test
public void createsMultipleEnvironments() throws Exception {
- when(docker.isContainerRunning(anyString())).thenReturn(true);
+ when(docker.isContainerRunning(any())).thenReturn(true);
DockerEnvironmentFactory factory = getFactory((workerId, timeout) -> client);
Environment fooEnv = Environments.createDockerEnvironment("foo");
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
index a38b1b5..519face 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
@@ -20,9 +20,9 @@
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.anyList;
import static org.mockito.Matchers.anyMap;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java
index d0c02c6..a1377f3 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java
@@ -21,7 +21,9 @@
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.ByteArrayOutputStream;
@@ -157,4 +159,32 @@
// assertThat(outputStr, containsString("testing123"));
assertFalse(ProcessManager.INHERIT_IO_FILE.exists());
}
+
+ @Test
+ public void testShutdownHook() throws IOException {
+ ProcessManager processManager = ProcessManager.create();
+
+ // no process alive, no shutdown hook
+ assertNull(ProcessManager.shutdownHook);
+
+ processManager.startProcess(
+ "1", "bash", Arrays.asList("-c", "echo 'testing123'"), Collections.emptyMap());
+ // the shutdown hook will be created when process is started
+ assertNotNull(ProcessManager.shutdownHook);
+ // check the shutdown hook is registered
+ assertTrue(Runtime.getRuntime().removeShutdownHook(ProcessManager.shutdownHook));
+ // add back the shutdown hook
+ Runtime.getRuntime().addShutdownHook(ProcessManager.shutdownHook);
+
+ processManager.startProcess(
+ "2", "bash", Arrays.asList("-c", "echo 'testing123'"), Collections.emptyMap());
+
+ processManager.stopProcess("1");
+ // the shutdown hook will be not removed if there are still processes alive
+ assertNotNull(ProcessManager.shutdownHook);
+
+ processManager.stopProcess("2");
+ // the shutdown hook will be removed when there is no process alive
+ assertNull(ProcessManager.shutdownHook);
+ }
}
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
index 1afb93f..e1ccad6 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.fnexecution.translation;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -234,6 +235,7 @@
Collections.emptyList(),
Collections.emptyList(),
Collections.emptyList(),
- Collections.emptyList());
+ Collections.emptyList(),
+ DEFAULT_WIRE_CODER_SETTING);
}
}
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java
index d504929..330fb24 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java
@@ -118,8 +118,14 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- getInternals().setTimer(namespace, timerId, target, timeDomain);
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ getInternals()
+ .setTimer(namespace, timerId, timerFamilyId, target, outputTimestamp, timeDomain);
}
@Override
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
index 2f3b809..a6214be 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
@@ -120,9 +120,9 @@
}
/**
- * Coder for {@link KeyedTimerData}. Note we don't use the {@link
- * org.apache.beam.runners.core.TimerInternals.TimerDataCoder} here directly since we want to
- * en/decode timestamp first so the timers will be sorted in the state.
+ * Coder for {@link KeyedTimerData}. Note we don't use the {@link TimerInternals.TimerDataCoderV2}
+ * here directly since we want to en/decode timestamp first so the timers will be sorted in the
+ * state.
*/
public static class KeyedTimerDataCoder<K> extends StructuredCoder<KeyedTimerData<K>> {
private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of();
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java
index 676129d..9ac082b 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java
@@ -184,8 +184,14 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- setTimer(TimerData.of(timerId, namespace, target, timeDomain));
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
+ setTimer(
+ TimerData.of(timerId, timerFamilyId, namespace, target, outputTimestamp, timeDomain));
}
@Override
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java
index 58b6645..0f536c3 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java
@@ -19,7 +19,7 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
diff --git a/runners/spark/job-server/build.gradle b/runners/spark/job-server/build.gradle
index 63ab04f..30dc24d 100644
--- a/runners/spark/job-server/build.gradle
+++ b/runners/spark/job-server/build.gradle
@@ -38,7 +38,7 @@
def sparkRunnerProject = project.parent.path
-description = project(sparkRunnerProject).description + " :: Job Server"
+description = "Apache Beam :: Runners :: Spark :: Job Server"
configurations {
validatesPortableRunner
diff --git a/runners/spark/job-server/container/Dockerfile b/runners/spark/job-server/container/Dockerfile
new file mode 100644
index 0000000..11a5f0e
--- /dev/null
+++ b/runners/spark/job-server/container/Dockerfile
@@ -0,0 +1,28 @@
+###############################################################################
+# 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.
+###############################################################################
+
+FROM openjdk:8
+MAINTAINER "Apache Beam <dev@beam.apache.org>"
+
+RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y libltdl7
+
+ADD beam-runners-spark-job-server.jar /opt/apache/beam/jars/
+ADD spark-job-server.sh /opt/apache/beam/
+
+WORKDIR /opt/apache/beam
+ENTRYPOINT ["./spark-job-server.sh"]
diff --git a/runners/spark/job-server/container/build.gradle b/runners/spark/job-server/container/build.gradle
new file mode 100644
index 0000000..c7244a3
--- /dev/null
+++ b/runners/spark/job-server/container/build.gradle
@@ -0,0 +1,64 @@
+import org.apache.beam.gradle.BeamModulePlugin
+
+/*
+ * 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.
+ */
+
+/**
+ * Build a Docker image to bootstrap SparkJobServerDriver which requires a Java environment.
+ * Alternatively, it can also be bootstrapped through the runShadow goal
+ * or by directly running the generated JAR file.
+ */
+
+apply plugin: 'org.apache.beam.module'
+apply plugin: 'base'
+applyDockerNature()
+
+def sparkJobServerProject = project.parent.path
+
+description = "Apache Beam :: Runners :: Spark :: Job Server :: Container"
+
+configurations {
+ dockerDependency
+}
+
+dependencies {
+ dockerDependency project(path: sparkJobServerProject, configuration: "shadow")
+}
+
+task copyDockerfileDependencies(type: Copy) {
+ // Required Jars
+ from configurations.dockerDependency
+ rename 'beam-runners-spark-job-server.*.jar', 'beam-runners-spark-job-server.jar'
+ into "build"
+ // Entry script
+ from "spark-job-server.sh"
+ into "build"
+ // Dockerfile
+ from "Dockerfile"
+ into "build"
+}
+
+docker {
+ name containerImageName(name: 'spark-job-server',
+ root: project.rootProject.hasProperty(["docker-repository-root"]) ?
+ project.rootProject["docker-repository-root"] : "apachebeam")
+ files "./build/"
+}
+
+// Ensure that we build the required resources and copy and file dependencies from related projects
+dockerPrepare.dependsOn copyDockerfileDependencies
diff --git a/runners/spark/job-server/container/spark-job-server.sh b/runners/spark/job-server/container/spark-job-server.sh
new file mode 100755
index 0000000..e39c5ae
--- /dev/null
+++ b/runners/spark/job-server/container/spark-job-server.sh
@@ -0,0 +1,29 @@
+#!/bin/sh
+###############################################################################
+# 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.
+###############################################################################
+
+### Just a simple script to bootstrap the SparkJobServerDriver
+### For the environment, see the Dockerfile
+
+# The following (forking to the background, then waiting) enables to use CTRL+C to kill the container.
+# We're PID 1 which doesn't handle signals. By forking the Java process to the background,
+# a PID > 1 is created which handles signals. After the command shuts down, the script and
+# thus the container will also exit.
+
+java -cp "jars/*" org.apache.beam.runners.spark.SparkJobServerDriver "$@" &
+wait
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
index 6df3bf3..d0c1c0f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
@@ -169,6 +169,10 @@
SparkPipelineOptions sparkOptions = portablePipelineOptions.as(SparkPipelineOptions.class);
String invocationId =
String.format("%s_%s", sparkOptions.getJobName(), UUID.randomUUID().toString());
+ if (sparkOptions.getAppName() == null) {
+ LOG.debug("App name was null. Using invocationId {}", invocationId);
+ sparkOptions.setAppName(invocationId);
+ }
SparkPipelineRunner runner = new SparkPipelineRunner(sparkOptions);
JobInfo jobInfo =
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java
index f8ff5e6..b741050 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java
@@ -368,7 +368,7 @@
private final FullWindowedValueCoder<InputT> wvCoder;
private final Coder<K> keyCoder;
private final List<Integer> sourceIds;
- private final TimerInternals.TimerDataCoder timerDataCoder;
+ private final TimerInternals.TimerDataCoderV2 timerDataCoder;
private final WindowingStrategy<?, W> windowingStrategy;
private final SerializablePipelineOptions options;
private final IterableCoder<WindowedValue<InputT>> itrWvCoder;
@@ -461,9 +461,9 @@
return FullWindowedValueCoder.of(KvCoder.of(keyCoder, IterableCoder.of(iCoder)), wCoder);
}
- private static <W extends BoundedWindow> TimerInternals.TimerDataCoder timerDataCoderOf(
+ private static <W extends BoundedWindow> TimerInternals.TimerDataCoderV2 timerDataCoderOf(
final WindowingStrategy<?, W> windowingStrategy) {
- return TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder());
+ return TimerInternals.TimerDataCoderV2.of(windowingStrategy.getWindowFn().windowCoder());
}
private static void checkpointIfNeeded(
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java
index 6cdcef4..1d5b36b 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java
@@ -155,7 +155,12 @@
@Override
public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
+ StateNamespace namespace,
+ String timerId,
+ String timerFamilyId,
+ Instant target,
+ Instant outputTimestamp,
+ TimeDomain timeDomain) {
throw new UnsupportedOperationException("Setting a timer by ID not yet supported.");
}
@@ -165,12 +170,12 @@
}
public static Collection<byte[]> serializeTimers(
- Collection<TimerData> timers, TimerDataCoder timerDataCoder) {
+ Collection<TimerData> timers, TimerDataCoderV2 timerDataCoder) {
return CoderHelpers.toByteArrays(timers, timerDataCoder);
}
public static Iterator<TimerData> deserializeTimers(
- Collection<byte[]> serTimers, TimerDataCoder timerDataCoder) {
+ Collection<byte[]> serTimers, TimerDataCoderV2 timerDataCoder) {
return CoderHelpers.fromByteArrays(serTimers, timerDataCoder).iterator();
}
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java
index 3310546..eca87bd 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java
@@ -19,7 +19,6 @@
import static org.apache.spark.sql.types.DataTypes.BinaryType;
-import java.io.ByteArrayInputStream;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
@@ -61,19 +60,54 @@
List<Expression> serialiserList = new ArrayList<>();
Class<? super T> claz = beamCoder.getEncodedTypeDescriptor().getRawType();
+ BeamCoderWrapper<T> beamCoderWrapper = new BeamCoderWrapper<>(beamCoder);
serialiserList.add(
- new EncodeUsingBeamCoder<>(new BoundReference(0, new ObjectType(claz), true), beamCoder));
+ new EncodeUsingBeamCoder<>(
+ new BoundReference(0, new ObjectType(claz), true), beamCoderWrapper));
ClassTag<T> classTag = ClassTag$.MODULE$.apply(claz);
return new ExpressionEncoder<>(
SchemaHelpers.binarySchema(),
false,
JavaConversions.collectionAsScalaIterable(serialiserList).toSeq(),
new DecodeUsingBeamCoder<>(
- new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType), classTag, beamCoder),
+ new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType),
+ classTag,
+ beamCoderWrapper),
classTag);
}
+ public static class BeamCoderWrapper<T> implements Serializable {
+
+ private Coder<T> beamCoder;
+
+ public BeamCoderWrapper(Coder<T> beamCoder) {
+ this.beamCoder = beamCoder;
+ }
+
+ public byte[] encode(boolean isInputNull, T inputValue) {
+ if (isInputNull) {
+ return null;
+ } else {
+ java.io.ByteArrayOutputStream baos = new java.io.ByteArrayOutputStream();
+ try {
+ beamCoder.encode(inputValue, baos);
+ } catch (Exception e) {
+ throw org.apache.beam.sdk.util.UserCodeException.wrap(e);
+ }
+ return baos.toByteArray();
+ }
+ }
+
+ public T decode(boolean isInputNull, byte[] inputValue) {
+ try {
+ return isInputNull ? null : beamCoder.decode(new java.io.ByteArrayInputStream(inputValue));
+ } catch (Exception e) {
+ throw org.apache.beam.sdk.util.UserCodeException.wrap(e);
+ }
+ }
+ }
+
/**
* Catalyst Expression that serializes elements using Beam {@link Coder}.
*
@@ -83,11 +117,11 @@
implements NonSQLExpression, Serializable {
private Expression child;
- private Coder<T> beamCoder;
+ private BeamCoderWrapper<T> beamCoderWrapper;
- public EncodeUsingBeamCoder(Expression child, Coder<T> beamCoder) {
+ public EncodeUsingBeamCoder(Expression child, BeamCoderWrapper<T> beamCoderWrapper) {
this.child = child;
- this.beamCoder = beamCoder;
+ this.beamCoderWrapper = beamCoderWrapper;
}
@Override
@@ -97,48 +131,32 @@
@Override
public ExprCode doGenCode(CodegenContext ctx, ExprCode ev) {
- // Code to serialize.
String accessCode =
- ctx.addReferenceObj("beamCoder", beamCoder, beamCoder.getClass().getName());
+ ctx.addReferenceObj(
+ "beamCoderWrapper", beamCoderWrapper, BeamCoderWrapper.class.getName());
ExprCode input = child.genCode(ctx);
+ String javaType = CodeGenerator.javaType(dataType());
- /*
- CODE GENERATED
- byte[] ${ev.value};
- try {
- java.io.ByteArrayOutputStream baos = new java.io.ByteArrayOutputStream();
- if ({input.isNull})
- ${ev.value} = null;
- else{
- $beamCoder.encode(${input.value}, baos);
- ${ev.value} = baos.toByteArray();
- }
- } catch (Exception e) {
- throw org.apache.beam.sdk.util.UserCodeException.wrap(e);
- }
- */
List<String> parts = new ArrayList<>();
- parts.add("byte[] ");
- parts.add(
- ";try { java.io.ByteArrayOutputStream baos = new java.io.ByteArrayOutputStream(); if (");
- parts.add(") ");
- parts.add(" = null; else{");
+ List<Object> args = new ArrayList<>();
+ /*
+ CODE GENERATED
+ final $javaType ${ev.value} = $beamCoderWrapper.encode(${input.isNull}, ${input.value});
+ */
+ parts.add("final ");
+ args.add(javaType);
+ parts.add(" ");
+ args.add(ev.value());
+ parts.add(" = ");
+ args.add(accessCode);
parts.add(".encode(");
- parts.add(", baos); ");
- parts.add(
- " = baos.toByteArray();}} catch (Exception e) {throw org.apache.beam.sdk.util.UserCodeException.wrap(e);}");
+ args.add(input.isNull());
+ parts.add(", ");
+ args.add(input.value());
+ parts.add(");");
StringContext sc =
new StringContext(JavaConversions.collectionAsScalaIterable(parts).toSeq());
-
- List<Object> args = new ArrayList<>();
-
- args.add(ev.value());
- args.add(input.isNull());
- args.add(ev.value());
- args.add(accessCode);
- args.add(input.value());
- args.add(ev.value());
Block code =
(new Block.BlockHelper(sc)).code(JavaConversions.collectionAsScalaIterable(args).toSeq());
@@ -156,7 +174,7 @@
case 0:
return child;
case 1:
- return beamCoder;
+ return beamCoderWrapper;
default:
throw new ArrayIndexOutOfBoundsException("productElement out of bounds");
}
@@ -181,12 +199,12 @@
return false;
}
EncodeUsingBeamCoder<?> that = (EncodeUsingBeamCoder<?>) o;
- return beamCoder.equals(that.beamCoder) && child.equals(that.child);
+ return beamCoderWrapper.equals(that.beamCoderWrapper) && child.equals(that.child);
}
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), child, beamCoder);
+ return Objects.hash(super.hashCode(), child, beamCoderWrapper);
}
}
@@ -200,12 +218,13 @@
private Expression child;
private ClassTag<T> classTag;
- private Coder<T> beamCoder;
+ private BeamCoderWrapper<T> beamCoderWrapper;
- public DecodeUsingBeamCoder(Expression child, ClassTag<T> classTag, Coder<T> beamCoder) {
+ public DecodeUsingBeamCoder(
+ Expression child, ClassTag<T> classTag, BeamCoderWrapper<T> beamCoderWrapper) {
this.child = child;
this.classTag = classTag;
- this.beamCoder = beamCoder;
+ this.beamCoderWrapper = beamCoderWrapper;
}
@Override
@@ -215,65 +234,38 @@
@Override
public ExprCode doGenCode(CodegenContext ctx, ExprCode ev) {
- // Code to deserialize.
String accessCode =
- ctx.addReferenceObj("beamCoder", beamCoder, beamCoder.getClass().getName());
+ ctx.addReferenceObj(
+ "beamCoderWrapper", beamCoderWrapper, BeamCoderWrapper.class.getName());
ExprCode input = child.genCode(ctx);
String javaType = CodeGenerator.javaType(dataType());
-
- /*
- CODE GENERATED:
- final $javaType ${ev.value}
- try {
- ${ev.value} =
- ${input.isNull} ?
- ${CodeGenerator.defaultValue(dataType)} :
- ($javaType) $beamCoder.decode(new java.io.ByteArrayInputStream(${input.value}));
- } catch (Exception e) {
- throw org.apache.beam.sdk.util.UserCodeException.wrap(e);
- }
- */
-
List<String> parts = new ArrayList<>();
+ List<Object> args = new ArrayList<>();
+ /*
+ CODE GENERATED:
+ final $javaType ${ev.value} = ($javaType) $beamCoderWrapper.decode(${input.isNull}, ${input.value});
+ */
parts.add("final ");
+ args.add(javaType);
parts.add(" ");
- parts.add(";try { ");
- parts.add(" = ");
- parts.add("? ");
- parts.add(": (");
+ args.add(ev.value());
+ parts.add(" = (");
+ args.add(javaType);
parts.add(") ");
- parts.add(".decode(new java.io.ByteArrayInputStream(");
- parts.add(
- ")); } catch (Exception e) {throw org.apache.beam.sdk.util.UserCodeException.wrap(e);}");
-
+ args.add(accessCode);
+ parts.add(".decode(");
+ args.add(input.isNull());
+ parts.add(", ");
+ args.add(input.value());
+ parts.add(");");
StringContext sc =
new StringContext(JavaConversions.collectionAsScalaIterable(parts).toSeq());
-
- List<Object> args = new ArrayList<>();
- args.add(javaType);
- args.add(ev.value());
- args.add(ev.value());
- args.add(input.isNull());
- args.add(CodeGenerator.defaultValue(dataType(), false));
- args.add(javaType);
- args.add(accessCode);
- args.add(input.value());
Block code =
(new Block.BlockHelper(sc)).code(JavaConversions.collectionAsScalaIterable(args).toSeq());
-
return ev.copy(input.code().$plus(code), input.isNull(), ev.value());
}
@Override
- public Object nullSafeEval(Object input) {
- try {
- return beamCoder.decode(new ByteArrayInputStream((byte[]) input));
- } catch (Exception e) {
- throw new IllegalStateException("Error decoding bytes for coder: " + beamCoder, e);
- }
- }
-
- @Override
public DataType dataType() {
return new ObjectType(classTag.runtimeClass());
}
@@ -286,7 +278,7 @@
case 1:
return classTag;
case 2:
- return beamCoder;
+ return beamCoderWrapper;
default:
throw new ArrayIndexOutOfBoundsException("productElement out of bounds");
}
@@ -313,12 +305,12 @@
DecodeUsingBeamCoder<?> that = (DecodeUsingBeamCoder<?>) o;
return child.equals(that.child)
&& classTag.equals(that.classTag)
- && beamCoder.equals(that.beamCoder);
+ && beamCoderWrapper.equals(that.beamCoderWrapper);
}
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), child, classTag, beamCoder);
+ return Objects.hash(super.hashCode(), child, classTag, beamCoderWrapper);
}
}
}
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
index 5a59fdd..f89f115 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
@@ -161,6 +161,11 @@
}
@Override
+ public void split(double fractionOfRemainder) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public void close() throws Exception {
if (once) {
return;
diff --git a/sdks/go/container/boot.go b/sdks/go/container/boot.go
index bc2f9ab..7b82cd3 100644
--- a/sdks/go/container/boot.go
+++ b/sdks/go/container/boot.go
@@ -122,6 +122,10 @@
"--semi_persist_dir=" + *semiPersistDir,
"--options=" + options,
}
+ if info.GetStatusEndpoint() != nil {
+ args = append(args, "--status_endpoint=" + info.GetStatusEndpoint().GetUrl())
+ }
+
log.Fatalf("User program exited: %v", execx.Execute(prog, args...))
}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go
index 60d2a8d..06131b7 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/datasource.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/datasource.go
@@ -262,7 +262,7 @@
// Split takes a sorted set of potential split indices, selects and actuates
// split on an appropriate split index, and returns the selected split index
// if successful. Returns an error when unable to split.
-func (n *DataSource) Split(splits []int64, frac float32) (int64, error) {
+func (n *DataSource) Split(splits []int64, frac float64) (int64, error) {
if splits == nil {
return 0, fmt.Errorf("failed to split: requested splits were empty")
}
@@ -275,7 +275,7 @@
// the promised split index to this value.
for _, s := range splits {
// // Never split on the first element, or the current element.
- if s > 0 && s > c && s < n.splitIdx {
+ if s > 0 && s > c && s <= n.splitIdx {
n.splitIdx = s
fs := n.splitIdx
n.mu.Unlock()
diff --git a/sdks/go/pkg/beam/core/runtime/exec/plan.go b/sdks/go/pkg/beam/core/runtime/exec/plan.go
index d221c7e..d87ff54 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/plan.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/plan.go
@@ -201,7 +201,7 @@
type SplitPoints struct {
// Splits is a list of desired split indices.
Splits []int64
- Frac float32
+ Frac float64
}
// Split takes a set of potential split indexes, and if successful returns
diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go
index 139db34..6134506 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go
@@ -168,7 +168,7 @@
if !ok {
return nil, errors.Errorf("windowing strategy %v not found", id)
}
- wfn, err := unmarshalWindowFn(ws.GetWindowFn().GetSpec())
+ wfn, err := unmarshalWindowFn(ws.GetWindowFn())
if err != nil {
return nil, err
}
@@ -345,13 +345,13 @@
if err := proto.Unmarshal(payload, &pardo); err != nil {
return nil, errors.Wrapf(err, "invalid ParDo payload for %v", transform)
}
- data = string(pardo.GetDoFn().GetSpec().GetPayload())
+ data = string(pardo.GetDoFn().GetPayload())
case urnPerKeyCombinePre, urnPerKeyCombineMerge, urnPerKeyCombineExtract:
var cmb pb.CombinePayload
if err := proto.Unmarshal(payload, &cmb); err != nil {
return nil, errors.Wrapf(err, "invalid CombinePayload payload for %v", transform)
}
- data = string(cmb.GetCombineFn().GetSpec().GetPayload())
+ data = string(cmb.GetCombineFn().GetPayload())
default:
// TODO(herohde) 12/4/2017: we see DoFns directly with Dataflow. Handle that
// case here, for now, so that the harness can use this logic.
@@ -479,7 +479,7 @@
if err := proto.Unmarshal(payload, &wp); err != nil {
return nil, errors.Wrapf(err, "invalid WindowInto payload for %v", transform)
}
- wfn, err := unmarshalWindowFn(wp.GetWindowFn().GetSpec())
+ wfn, err := unmarshalWindowFn(wp.GetWindowFn())
if err != nil {
return nil, err
}
diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
index 17dc253..2f915c5 100644
--- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
@@ -161,16 +161,14 @@
edge := s.Edges[1].Edge
acID := m.coders.Add(edge.AccumCoder)
payload := &pb.CombinePayload{
- CombineFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
- Urn: URNJavaDoFn,
- Payload: []byte(mustEncodeMultiEdgeBase64(edge)),
- },
- EnvironmentId: m.addDefaultEnv(),
+ CombineFn: &pb.FunctionSpec{
+ Urn: URNJavaDoFn,
+ Payload: []byte(mustEncodeMultiEdgeBase64(edge)),
},
AccumulatorCoderId: acID,
}
transform.Spec = &pb.FunctionSpec{Urn: URNCombinePerKey, Payload: protox.MustEncode(payload)}
+ transform.EnvironmentId = m.addDefaultEnv()
}
func (m *marshaller) addMultiEdge(edge NamedEdge) []string {
@@ -197,6 +195,7 @@
// allPIds tracks additional PTransformIDs generated for the pipeline
var allPIds []string
var spec *pb.FunctionSpec
+ var transformEnvID = ""
switch edge.Edge.Op {
case graph.Impulse:
// TODO(herohde) 7/18/2018: Encode data?
@@ -218,14 +217,11 @@
m.makeNode(out, m.coders.Add(makeBytesKeyedCoder(in.From.Coder)), in.From)
payload := &pb.ParDoPayload{
- DoFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
+ DoFn: &pb.FunctionSpec{
+ Urn: URNIterableSideInputKey,
+ Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
Urn: URNIterableSideInputKey,
- Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
- Urn: URNIterableSideInputKey,
- })),
- },
- EnvironmentId: m.addDefaultEnv(),
+ })),
},
}
@@ -236,8 +232,9 @@
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
- Inputs: map[string]string{"i0": nodeID(in.From)},
- Outputs: map[string]string{"i0": out},
+ Inputs: map[string]string{"i0": nodeID(in.From)},
+ Outputs: map[string]string{"i0": out},
+ EnvironmentId: m.addDefaultEnv(),
}
m.transforms[keyedID] = keyed
allPIds = append(allPIds, keyedID)
@@ -249,17 +246,11 @@
AccessPattern: &pb.FunctionSpec{
Urn: URNMultimapSideInput,
},
- ViewFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
- Urn: "foo",
- },
- EnvironmentId: m.addDefaultEnv(),
+ ViewFn: &pb.FunctionSpec{
+ Urn: "foo",
},
- WindowMappingFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
- Urn: "bar",
- },
- EnvironmentId: m.addDefaultEnv(),
+ WindowMappingFn: &pb.FunctionSpec{
+ Urn: "bar",
},
}
@@ -272,27 +263,23 @@
}
payload := &pb.ParDoPayload{
- DoFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
- Urn: URNJavaDoFn,
- Payload: []byte(mustEncodeMultiEdgeBase64(edge.Edge)),
- },
- EnvironmentId: m.addDefaultEnv(),
+ DoFn: &pb.FunctionSpec{
+ Urn: URNJavaDoFn,
+ Payload: []byte(mustEncodeMultiEdgeBase64(edge.Edge)),
},
SideInputs: si,
}
+ transformEnvID = m.addDefaultEnv()
spec = &pb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)}
case graph.Combine:
payload := &pb.ParDoPayload{
- DoFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
- Urn: URNJavaDoFn,
- Payload: []byte(mustEncodeMultiEdgeBase64(edge.Edge)),
- },
- EnvironmentId: m.addDefaultEnv(),
+ DoFn: &pb.FunctionSpec{
+ Urn: URNJavaDoFn,
+ Payload: []byte(mustEncodeMultiEdgeBase64(edge.Edge)),
},
}
+ transformEnvID = m.addDefaultEnv()
spec = &pb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)}
case graph.Flatten:
@@ -303,9 +290,7 @@
case graph.WindowInto:
payload := &pb.WindowIntoPayload{
- WindowFn: &pb.SdkFunctionSpec{
- Spec: makeWindowFn(edge.Edge.WindowFn),
- },
+ WindowFn: makeWindowFn(edge.Edge.WindowFn),
}
spec = &pb.FunctionSpec{Urn: URNWindow, Payload: protox.MustEncode(payload)}
@@ -317,10 +302,11 @@
}
transform := &pb.PTransform{
- UniqueName: edge.Name,
- Spec: spec,
- Inputs: inputs,
- Outputs: outputs,
+ UniqueName: edge.Name,
+ Spec: spec,
+ Inputs: inputs,
+ Outputs: outputs,
+ EnvironmentId: transformEnvID,
}
m.transforms[id] = transform
allPIds = append(allPIds, id)
@@ -348,15 +334,12 @@
injectID := fmt.Sprintf("%v_inject%v", id, i)
payload := &pb.ParDoPayload{
- DoFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
- Urn: URNInject,
- Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
- Urn: URNInject,
- Inject: &v1.InjectPayload{N: (int32)(i)},
- })),
- },
- EnvironmentId: m.addDefaultEnv(),
+ DoFn: &pb.FunctionSpec{
+ Urn: URNInject,
+ Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
+ Urn: URNInject,
+ Inject: &v1.InjectPayload{N: (int32)(i)},
+ })),
},
}
inject := &pb.PTransform{
@@ -365,8 +348,9 @@
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
- Inputs: map[string]string{"i0": nodeID(in.From)},
- Outputs: map[string]string{"i0": out},
+ Inputs: map[string]string{"i0": nodeID(in.From)},
+ Outputs: map[string]string{"i0": out},
+ EnvironmentId: m.addDefaultEnv(),
}
m.transforms[injectID] = inject
subtransforms = append(subtransforms, injectID)
@@ -412,14 +396,11 @@
expandID := fmt.Sprintf("%v_expand", id)
payload := &pb.ParDoPayload{
- DoFn: &pb.SdkFunctionSpec{
- Spec: &pb.FunctionSpec{
+ DoFn: &pb.FunctionSpec{
+ Urn: URNExpand,
+ Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
Urn: URNExpand,
- Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
- Urn: URNExpand,
- })),
- },
- EnvironmentId: m.addDefaultEnv(),
+ })),
},
}
expand := &pb.PTransform{
@@ -428,8 +409,9 @@
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
- Inputs: map[string]string{"i0": gbkOut},
- Outputs: map[string]string{"i0": nodeID(outNode)},
+ Inputs: map[string]string{"i0": gbkOut},
+ Outputs: map[string]string{"i0": nodeID(outNode)},
+ EnvironmentId: m.addDefaultEnv(),
}
m.transforms[id] = expand
subtransforms = append(subtransforms, id)
@@ -481,6 +463,7 @@
func (m *marshaller) addWindowingStrategy(w *window.WindowingStrategy) string {
ws := marshalWindowingStrategy(m.coders, w)
+ ws.EnvironmentId = m.addDefaultEnv()
return m.internWindowingStrategy(ws)
}
@@ -500,9 +483,7 @@
// the given coder context.
func marshalWindowingStrategy(c *CoderMarshaller, w *window.WindowingStrategy) *pb.WindowingStrategy {
ws := &pb.WindowingStrategy{
- WindowFn: &pb.SdkFunctionSpec{
- Spec: makeWindowFn(w.Fn),
- },
+ WindowFn: makeWindowFn(w.Fn),
MergeStatus: pb.MergeStatus_NON_MERGING,
AccumulationMode: pb.AccumulationMode_DISCARDING,
WindowCoderId: c.AddWindowCoder(makeWindowCoder(w.Fn)),
diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go
index 7e9f7b1..7615356 100644
--- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go
+++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go
@@ -150,14 +150,14 @@
writers map[clientID]*dataWriter
readers map[clientID]*dataReader
-
// readErr indicates a client.Recv error and is used to prevent new readers.
readErr error
+
// a closure that forces the data manager to recreate this stream.
forceRecreate func(id string, err error)
cancelFn context.CancelFunc // Allows writers to stop the grpc reading goroutine.
- mu sync.Mutex // guards both the readers and writers maps.
+ mu sync.Mutex // guards mutable internal data, notably the maps and readErr.
}
func newDataChannel(ctx context.Context, port exec.Port) (*DataChannel, error) {
@@ -198,6 +198,8 @@
// OpenRead returns an io.ReadCloser of the data elements for the given instruction and ptransform.
func (c *DataChannel) OpenRead(ctx context.Context, ptransformID string, instID instructionID) io.ReadCloser {
+ c.mu.Lock()
+ defer c.mu.Unlock()
cid := clientID{ptransformID: ptransformID, instID: instID}
if c.readErr != nil {
log.Errorf(ctx, "opening a reader %v on a closed channel", cid)
@@ -256,7 +258,9 @@
if local, ok := cache[id]; ok {
r = local
} else {
+ c.mu.Lock()
r = c.makeReader(ctx, id)
+ c.mu.Unlock()
cache[id] = r
}
@@ -306,10 +310,8 @@
return r.err
}
+// makeReader creates a dataReader. It expects to be called while c.mu is held.
func (c *DataChannel) makeReader(ctx context.Context, id clientID) *dataReader {
- c.mu.Lock()
- defer c.mu.Unlock()
-
if r, ok := c.readers[id]; ok {
return r
}
diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go
index 28bdf4c..58d4d2d 100644
--- a/sdks/go/pkg/beam/core/runtime/harness/harness.go
+++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go
@@ -287,8 +287,8 @@
ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{
ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{
&fnpb.ProcessBundleSplitResponse_ChannelSplit{
- LastPrimaryElement: int32(split - 1),
- FirstResidualElement: int32(split),
+ LastPrimaryElement: split - 1,
+ FirstResidualElement: split,
},
},
},
diff --git a/sdks/go/pkg/beam/core/runtime/harness/init/init.go b/sdks/go/pkg/beam/core/runtime/harness/init/init.go
index 889c33b..4043ada 100644
--- a/sdks/go/pkg/beam/core/runtime/harness/init/init.go
+++ b/sdks/go/pkg/beam/core/runtime/harness/init/init.go
@@ -42,6 +42,7 @@
id = flag.String("id", "", "Local identifier (required in worker mode).")
loggingEndpoint = flag.String("logging_endpoint", "", "Local logging gRPC endpoint (required in worker mode).")
controlEndpoint = flag.String("control_endpoint", "", "Local control gRPC endpoint (required in worker mode).")
+ statusEndpoint = flag.String("status_endpoint", "", "Local status gRPC endpoint (optional in worker mode).")
semiPersistDir = flag.String("semi_persist_dir", "/tmp", "Local semi-persistent directory (optional in worker mode).")
options = flag.String("options", "", "JSON-encoded pipeline options (required in worker mode).")
)
diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go b/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go
index 444af56..12465a8 100644
--- a/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go
+++ b/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go
@@ -52,6 +52,7 @@
ret.Subtransforms, _ = reflectx.ShallowClone(t.Subtransforms).([]string)
ret.Inputs, _ = reflectx.ShallowClone(t.Inputs).(map[string]string)
ret.Outputs, _ = reflectx.ShallowClone(t.Outputs).(map[string]string)
+ ret.EnvironmentId = t.EnvironmentId
return ret
}
diff --git a/sdks/go/pkg/beam/model/PROTOBUF.md b/sdks/go/pkg/beam/model/PROTOBUF.md
index 0dc63e7..5c9b162 100644
--- a/sdks/go/pkg/beam/model/PROTOBUF.md
+++ b/sdks/go/pkg/beam/model/PROTOBUF.md
@@ -20,14 +20,24 @@
# Rebuilding generated protobuf code
If you make changes to .proto files, you will need to rebuild the generated Go code.
-To do that, you will need:
-* [The protobuf compiler](https://github.com/google/protobuf/releases)
-* A proper Go development setup per `BUILD.md` (variables GOPATH and GOBIN set properly)
-* `go get -u github.com/golang/protobuf/protoc-gen-go`
+First, follow this one-time setup:
-If all this setup is complete, simply run `go generate` in the current directory
-(`pkg/beam/model`).
+1. Download [the protobuf compiler](https://github.com/google/protobuf/releases).
+ The simplest approach is to download one of the prebuilt binaries and extract
+ it somewhere in your machine's `$PATH`.
+1. A properly installed Go development environment per [the official
+ instructions](https://golang.org/doc/install). `$GOPATH` must be set properly.
+ If it's not set, follow
+ [these instructions](https://github.com/golang/go/wiki/SettingGOPATH).
+1. Add `$GOBIN` to your `$PATH`. (Note: If `$GOBIN` is not set, add `$GOPATH/bin`
+ instead.)
+
+To generate the code:
+
+1. Navigate to this directory (`pkg/beam/model`).
+1. `go get -u github.com/golang/protobuf/protoc-gen-go`
+1. `go generate`
## Generated Go code fails to build
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
index 37e7829..19bdfa4 100644
--- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
@@ -708,16 +708,12 @@
// Either an absolute timestamp or a relative timestamp can represent a
// scheduled execution time.
type DelayedBundleApplication struct {
- // Recommended time at which the application should be scheduled to execute
- // by the runner. Times in the past may be scheduled to execute immediately.
- // TODO(BEAM-8536): Migrate usage of absolute time to requested_time_delay.
- RequestedExecutionTime *timestamp.Timestamp `protobuf:"bytes,1,opt,name=requested_execution_time,json=requestedExecutionTime,proto3" json:"requested_execution_time,omitempty"`
// (Required) The application that should be scheduled.
- Application *BundleApplication `protobuf:"bytes,2,opt,name=application,proto3" json:"application,omitempty"`
+ Application *BundleApplication `protobuf:"bytes,1,opt,name=application,proto3" json:"application,omitempty"`
// Recommended time delay at which the application should be scheduled to
// execute by the runner. Time delay that equals 0 may be scheduled to execute
// immediately. The unit of time delay should be microsecond.
- RequestedTimeDelay *duration.Duration `protobuf:"bytes,3,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"`
+ RequestedTimeDelay *duration.Duration `protobuf:"bytes,2,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@@ -748,13 +744,6 @@
var xxx_messageInfo_DelayedBundleApplication proto.InternalMessageInfo
-func (m *DelayedBundleApplication) GetRequestedExecutionTime() *timestamp.Timestamp {
- if m != nil {
- return m.RequestedExecutionTime
- }
- return nil
-}
-
func (m *DelayedBundleApplication) GetApplication() *BundleApplication {
if m != nil {
return m.Application
@@ -1832,7 +1821,7 @@
//
// Set to 0 to "checkpoint" as soon as possible (keeping as little work as
// possible and returning the remainder).
- FractionOfRemainder float32 `protobuf:"fixed32,1,opt,name=fraction_of_remainder,json=fractionOfRemainder,proto3" json:"fraction_of_remainder,omitempty"`
+ FractionOfRemainder float64 `protobuf:"fixed64,1,opt,name=fraction_of_remainder,json=fractionOfRemainder,proto3" json:"fraction_of_remainder,omitempty"`
// A set of allowed element indices where the SDK may split. When this is
// empty, there are no constraints on where to split.
AllowedSplitPoints []int64 `protobuf:"varint,3,rep,packed,name=allowed_split_points,json=allowedSplitPoints,proto3" json:"allowed_split_points,omitempty"`
@@ -1872,7 +1861,7 @@
var xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit proto.InternalMessageInfo
-func (m *ProcessBundleSplitRequest_DesiredSplit) GetFractionOfRemainder() float32 {
+func (m *ProcessBundleSplitRequest_DesiredSplit) GetFractionOfRemainder() float64 {
if m != nil {
return m.FractionOfRemainder
}
@@ -1978,11 +1967,11 @@
// The last element of the input channel that should be entirely considered
// part of the primary, identified by its absolute index in the (ordered)
// channel.
- LastPrimaryElement int32 `protobuf:"varint,2,opt,name=last_primary_element,json=lastPrimaryElement,proto3" json:"last_primary_element,omitempty"`
+ LastPrimaryElement int64 `protobuf:"varint,2,opt,name=last_primary_element,json=lastPrimaryElement,proto3" json:"last_primary_element,omitempty"`
// The first element of the input channel that should be entirely considered
// part of the residual, identified by its absolute index in the (ordered)
// channel.
- FirstResidualElement int32 `protobuf:"varint,3,opt,name=first_residual_element,json=firstResidualElement,proto3" json:"first_residual_element,omitempty"`
+ FirstResidualElement int64 `protobuf:"varint,3,opt,name=first_residual_element,json=firstResidualElement,proto3" json:"first_residual_element,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@@ -2022,14 +2011,14 @@
return ""
}
-func (m *ProcessBundleSplitResponse_ChannelSplit) GetLastPrimaryElement() int32 {
+func (m *ProcessBundleSplitResponse_ChannelSplit) GetLastPrimaryElement() int64 {
if m != nil {
return m.LastPrimaryElement
}
return 0
}
-func (m *ProcessBundleSplitResponse_ChannelSplit) GetFirstResidualElement() int32 {
+func (m *ProcessBundleSplitResponse_ChannelSplit) GetFirstResidualElement() int64 {
if m != nil {
return m.FirstResidualElement
}
@@ -3671,210 +3660,208 @@
func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_6d954c03a4758710) }
var fileDescriptor_6d954c03a4758710 = []byte{
- // 3241 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xdd, 0x6f, 0x1b, 0xc7,
- 0xb5, 0xf7, 0xf2, 0x43, 0x22, 0x0f, 0x29, 0x89, 0x1c, 0x49, 0x36, 0xbd, 0xd7, 0xb9, 0xd7, 0xe1,
- 0xbd, 0x01, 0x84, 0x5c, 0x84, 0xb6, 0x65, 0x23, 0xb1, 0x73, 0x13, 0x27, 0x12, 0x45, 0xdb, 0x8c,
- 0x65, 0x9b, 0x77, 0x25, 0x5f, 0xdf, 0x9b, 0xdc, 0x64, 0xb1, 0xe2, 0x0e, 0xe9, 0x85, 0xc9, 0xdd,
- 0xcd, 0xcc, 0x52, 0xb6, 0xdc, 0xa0, 0xe9, 0x17, 0x5a, 0xb4, 0x68, 0x9b, 0x97, 0x3e, 0x24, 0x7d,
- 0x6b, 0x0b, 0x14, 0xe8, 0x4b, 0xff, 0x80, 0xfc, 0x03, 0x45, 0x9f, 0xfa, 0x0f, 0xe4, 0xa5, 0x40,
- 0x0b, 0xb4, 0x4d, 0x9f, 0x0b, 0xf4, 0xad, 0x98, 0x8f, 0xfd, 0xe0, 0x72, 0xe9, 0x2c, 0x29, 0xb9,
- 0x6f, 0x3b, 0x73, 0xf6, 0xfc, 0x7e, 0x33, 0x67, 0xcf, 0x9c, 0x39, 0x67, 0x66, 0xa1, 0x7a, 0x80,
- 0x8d, 0xa1, 0xde, 0xb3, 0x75, 0xc3, 0xb5, 0x1a, 0x2e, 0x71, 0x3c, 0x07, 0xbd, 0xe4, 0x90, 0x7e,
- 0xc3, 0x70, 0x8d, 0xee, 0x43, 0xdc, 0x60, 0xd2, 0xc6, 0xd0, 0x31, 0xf1, 0xa0, 0xd1, 0xb3, 0x75,
- 0xfc, 0x04, 0x77, 0x47, 0x9e, 0xe5, 0xd8, 0x8d, 0xc3, 0x4b, 0xea, 0x3a, 0xd7, 0x24, 0x23, 0xdb,
- 0xc6, 0x24, 0xd4, 0x56, 0x57, 0xb0, 0x6d, 0xba, 0x8e, 0x65, 0x7b, 0x54, 0x76, 0x9c, 0xef, 0x3b,
- 0x4e, 0x7f, 0x80, 0x2f, 0xf0, 0xd6, 0xc1, 0xa8, 0x77, 0xc1, 0xc4, 0xb4, 0x4b, 0x2c, 0xd7, 0x73,
- 0x88, 0x7c, 0xe3, 0xdf, 0xe2, 0x6f, 0x78, 0xd6, 0x10, 0x53, 0xcf, 0x18, 0xba, 0xf2, 0x85, 0x7f,
- 0x9d, 0x80, 0x18, 0x11, 0x83, 0x8f, 0x63, 0x8a, 0xfc, 0x31, 0x31, 0x5c, 0x17, 0x13, 0x7f, 0x08,
- 0x4b, 0x43, 0xec, 0x11, 0xab, 0x2b, 0x9b, 0xf5, 0x9f, 0x2a, 0xb0, 0xac, 0xe1, 0xa1, 0xe3, 0xe1,
- 0x9b, 0xc4, 0xed, 0x76, 0x1c, 0xe2, 0xa1, 0x21, 0x9c, 0x36, 0x5c, 0x4b, 0xa7, 0x98, 0x1c, 0x5a,
- 0x5d, 0xac, 0x87, 0x43, 0xac, 0x29, 0xe7, 0x95, 0x8d, 0xd2, 0xe6, 0x6b, 0x8d, 0x64, 0xa3, 0xb8,
- 0x96, 0x8b, 0x07, 0x96, 0x8d, 0x1b, 0x87, 0x97, 0x1a, 0x5b, 0xae, 0xb5, 0x27, 0xf4, 0x77, 0x02,
- 0x75, 0x6d, 0xcd, 0x48, 0xe8, 0x45, 0x67, 0xa1, 0xd0, 0x75, 0x4c, 0x4c, 0x74, 0xcb, 0xac, 0x65,
- 0xce, 0x2b, 0x1b, 0x45, 0x6d, 0x91, 0xb7, 0xdb, 0x66, 0xfd, 0x8f, 0x39, 0x40, 0x6d, 0x9b, 0x7a,
- 0x64, 0xd4, 0x65, 0x33, 0xd4, 0xf0, 0x87, 0x23, 0x4c, 0x3d, 0xf4, 0x12, 0x2c, 0x5b, 0x61, 0x2f,
- 0xd3, 0x53, 0xb8, 0xde, 0x52, 0xa4, 0xb7, 0x6d, 0xa2, 0xfb, 0x50, 0x20, 0xb8, 0x6f, 0x51, 0x0f,
- 0x93, 0xda, 0x9f, 0x16, 0xf9, 0xd0, 0x5f, 0x6d, 0xa4, 0xfa, 0x9e, 0x0d, 0x4d, 0xea, 0x49, 0xc6,
- 0x5b, 0xa7, 0xb4, 0x00, 0x0a, 0x61, 0x58, 0x76, 0x89, 0xd3, 0xc5, 0x94, 0xea, 0x07, 0x23, 0xdb,
- 0x1c, 0xe0, 0xda, 0x9f, 0x05, 0xf8, 0x7f, 0xa5, 0x04, 0xef, 0x08, 0xed, 0x6d, 0xae, 0x1c, 0x32,
- 0x2c, 0xb9, 0xd1, 0x7e, 0xf4, 0x75, 0x38, 0x33, 0x4e, 0xa3, 0xbb, 0xc4, 0xe9, 0x13, 0x4c, 0x69,
- 0xed, 0x2f, 0x82, 0xaf, 0x39, 0x0f, 0x5f, 0x47, 0x82, 0x84, 0xbc, 0xeb, 0x6e, 0x92, 0x1c, 0x8d,
- 0x60, 0x2d, 0xc6, 0x4f, 0xdd, 0x81, 0xe5, 0xd5, 0xbe, 0x14, 0xe4, 0x6f, 0xcf, 0x43, 0xbe, 0xc7,
- 0x10, 0x42, 0x66, 0xe4, 0x4e, 0x08, 0xd1, 0x43, 0x58, 0xe9, 0x59, 0xb6, 0x31, 0xb0, 0x9e, 0x62,
- 0xdf, 0xbc, 0x7f, 0x15, 0x8c, 0x6f, 0xa4, 0x64, 0xbc, 0x21, 0xd5, 0xe3, 0xf6, 0x5d, 0xee, 0x8d,
- 0x09, 0xb6, 0x8b, 0xb0, 0x48, 0x84, 0xb0, 0xfe, 0xad, 0x3c, 0xac, 0x8e, 0xf9, 0x19, 0x75, 0x1d,
- 0x9b, 0xe2, 0xb4, 0x8e, 0xb6, 0x06, 0x79, 0x4c, 0x88, 0x43, 0xa4, 0xfb, 0x8a, 0x06, 0xfa, 0x9f,
- 0x49, 0xf7, 0x7b, 0x6d, 0x66, 0xf7, 0x13, 0x03, 0x19, 0xf3, 0xbf, 0xde, 0x34, 0xff, 0x7b, 0x63,
- 0x3e, 0xff, 0x0b, 0x28, 0x62, 0x0e, 0xf8, 0xf1, 0x57, 0x3a, 0xe0, 0xce, 0xf1, 0x1c, 0x30, 0x20,
- 0x9e, 0xe2, 0x81, 0x87, 0xcf, 0xf6, 0xc0, 0xad, 0x63, 0x78, 0x60, 0x40, 0x9d, 0xe4, 0x82, 0xd6,
- 0x54, 0x17, 0x7c, 0x73, 0x4e, 0x17, 0x0c, 0xe8, 0xe2, 0x3e, 0x08, 0xcc, 0x47, 0x84, 0xb4, 0xfe,
- 0x23, 0x05, 0x56, 0x62, 0x71, 0x07, 0x3d, 0x85, 0xb3, 0x31, 0x13, 0x8c, 0x45, 0xe3, 0xec, 0x46,
- 0x69, 0xf3, 0xfa, 0x3c, 0x66, 0x88, 0x04, 0xe5, 0x33, 0x6e, 0xb2, 0xa0, 0x8e, 0xa0, 0x12, 0xf7,
- 0xc3, 0xfa, 0x2f, 0x00, 0xce, 0x4c, 0x01, 0x42, 0xcb, 0x90, 0x09, 0x16, 0x48, 0xc6, 0x32, 0x91,
- 0x0d, 0xe0, 0x11, 0xc3, 0xa6, 0x3d, 0x87, 0x0c, 0x69, 0x2d, 0xc3, 0x07, 0x7b, 0xf7, 0x78, 0x83,
- 0x6d, 0xec, 0x07, 0x80, 0x2d, 0xdb, 0x23, 0x47, 0x5a, 0x84, 0x01, 0x79, 0x50, 0x76, 0xbb, 0xce,
- 0x60, 0x80, 0xf9, 0xb2, 0xa4, 0xb5, 0x2c, 0x67, 0xec, 0x1c, 0x93, 0xb1, 0x13, 0x81, 0x14, 0x9c,
- 0x63, 0x2c, 0xe8, 0x07, 0x0a, 0xac, 0x3d, 0xb6, 0x6c, 0xd3, 0x79, 0x6c, 0xd9, 0x7d, 0x9d, 0x7a,
- 0xc4, 0xf0, 0x70, 0xdf, 0xc2, 0xb4, 0x96, 0xe3, 0xf4, 0x0f, 0x8e, 0x49, 0xff, 0xc0, 0x87, 0xde,
- 0x0b, 0x90, 0xc5, 0x28, 0x56, 0x1f, 0x4f, 0x4a, 0xd0, 0x01, 0x2c, 0xf0, 0xad, 0x93, 0xd6, 0xf2,
- 0x9c, 0xfd, 0x9d, 0x63, 0xb2, 0x37, 0x39, 0x98, 0x20, 0x94, 0xc8, 0xcc, 0xcc, 0xd8, 0x3e, 0xb4,
- 0x88, 0x63, 0x0f, 0xb1, 0xed, 0xd1, 0xda, 0xc2, 0x89, 0x98, 0xb9, 0x15, 0x81, 0x94, 0x66, 0x8e,
- 0xb2, 0xa0, 0x27, 0x70, 0x8e, 0x7a, 0x86, 0x87, 0xf5, 0x29, 0x99, 0xc9, 0xe2, 0xf1, 0x32, 0x93,
- 0xb3, 0x1c, 0x3c, 0x49, 0xa4, 0x0e, 0x60, 0x25, 0xe6, 0x75, 0xa8, 0x02, 0xd9, 0x47, 0xf8, 0x48,
- 0xba, 0x3a, 0x7b, 0x44, 0x4d, 0xc8, 0x1f, 0x1a, 0x83, 0x11, 0xe6, 0x3b, 0x40, 0x69, 0xf3, 0x95,
- 0x14, 0xe3, 0xe8, 0x04, 0xa8, 0x9a, 0xd0, 0x7d, 0x3d, 0x73, 0x55, 0x51, 0x1d, 0xa8, 0x4e, 0x78,
- 0x5c, 0x02, 0xdf, 0xce, 0x38, 0x5f, 0x23, 0x0d, 0x5f, 0x33, 0x80, 0x8d, 0x12, 0x7e, 0x04, 0xb5,
- 0x69, 0x3e, 0x96, 0xc0, 0xfb, 0xce, 0x38, 0xef, 0x95, 0x14, 0xbc, 0x71, 0xf4, 0xa3, 0x28, 0x7b,
- 0x17, 0x4a, 0x11, 0x1f, 0x4b, 0x20, 0xbc, 0x3e, 0x4e, 0xb8, 0x91, 0x82, 0x90, 0x03, 0xc6, 0x6c,
- 0x3a, 0xe1, 0x5e, 0x27, 0x63, 0xd3, 0x08, 0x6c, 0x84, 0xb0, 0xfe, 0xf7, 0x2c, 0x54, 0x85, 0x87,
- 0x6f, 0xb9, 0xee, 0xc0, 0xea, 0xf2, 0xf4, 0x1c, 0xbd, 0x08, 0xe5, 0x20, 0x5a, 0x85, 0xa9, 0x44,
- 0x29, 0xe8, 0x6b, 0x9b, 0x2c, 0x15, 0xb6, 0x6c, 0x77, 0xe4, 0x45, 0x52, 0x61, 0xde, 0x6e, 0x9b,
- 0xa8, 0x06, 0x8b, 0x78, 0x80, 0x19, 0x53, 0x2d, 0x7b, 0x5e, 0xd9, 0x28, 0x6b, 0x7e, 0x13, 0x7d,
- 0x0d, 0xaa, 0xce, 0xc8, 0x63, 0x5a, 0x8f, 0x0d, 0x0f, 0x93, 0xa1, 0x41, 0x1e, 0xf9, 0xd1, 0x27,
- 0x6d, 0xb8, 0x9d, 0x18, 0x6c, 0xe3, 0x1e, 0x47, 0x7c, 0x10, 0x00, 0x8a, 0x35, 0x59, 0x71, 0x62,
- 0xdd, 0xa8, 0x03, 0x60, 0x51, 0xfd, 0xc0, 0x19, 0xd9, 0x26, 0x36, 0x6b, 0xf9, 0xf3, 0xca, 0xc6,
- 0xf2, 0xe6, 0xa5, 0x14, 0x96, 0x6b, 0xd3, 0x6d, 0xa1, 0xd3, 0x68, 0xd9, 0xa3, 0xa1, 0x56, 0xb4,
- 0xfc, 0x36, 0xfa, 0x7f, 0xa8, 0x0c, 0x1d, 0xdb, 0xf2, 0x1c, 0xc2, 0x02, 0xaa, 0x65, 0xf7, 0x1c,
- 0x3f, 0xc6, 0xa4, 0xc1, 0xbd, 0x13, 0xa8, 0xb6, 0xed, 0x9e, 0xa3, 0xad, 0x0c, 0xc7, 0xda, 0x54,
- 0xd5, 0x61, 0x3d, 0x71, 0x6a, 0x09, 0xfe, 0x70, 0x71, 0xdc, 0x1f, 0xd4, 0x86, 0x28, 0xac, 0x1a,
- 0x7e, 0x61, 0xd5, 0xd8, 0xf7, 0x2b, 0xb3, 0xe8, 0xb7, 0xff, 0x2c, 0x03, 0xb5, 0x1d, 0x3c, 0x30,
- 0x8e, 0xb0, 0x39, 0xe9, 0x02, 0xfb, 0x50, 0x93, 0x29, 0x27, 0x36, 0xc3, 0x2f, 0xa0, 0xb3, 0x12,
- 0x4f, 0xd6, 0x56, 0xcf, 0x62, 0x39, 0x1d, 0xe8, 0xb6, 0x7c, 0x55, 0x26, 0x44, 0xef, 0x42, 0xc9,
- 0x08, 0x49, 0xe4, 0x70, 0xaf, 0xce, 0xfb, 0xe9, 0xb5, 0x28, 0x18, 0xba, 0x0d, 0x6b, 0xe1, 0x88,
- 0xd9, 0x38, 0x75, 0x93, 0x4d, 0x8e, 0xfb, 0x60, 0x69, 0xf3, 0xec, 0xc4, 0x68, 0x77, 0x64, 0x31,
- 0xaa, 0xa1, 0x40, 0x8d, 0x8d, 0x91, 0x5b, 0xa4, 0xfe, 0xe3, 0x1c, 0xac, 0x25, 0x15, 0x3f, 0xe8,
- 0x2d, 0x38, 0x37, 0x35, 0xcd, 0x09, 0x97, 0xca, 0xd9, 0x29, 0x99, 0x4a, 0xdb, 0x44, 0x16, 0x94,
- 0xbb, 0x6c, 0xa6, 0xba, 0xe7, 0x3c, 0xc2, 0xb6, 0x9f, 0x6d, 0xdc, 0x38, 0x46, 0x41, 0xd6, 0x68,
- 0x32, 0xad, 0x7d, 0x06, 0xa7, 0x95, 0xba, 0xc1, 0x33, 0x55, 0x7f, 0x9b, 0x01, 0x08, 0x65, 0xe8,
- 0x43, 0x80, 0x11, 0xc5, 0x44, 0xe7, 0x1b, 0x88, 0xfc, 0x88, 0x9d, 0x93, 0xe1, 0x6d, 0xdc, 0xa7,
- 0x98, 0xec, 0x31, 0xdc, 0x5b, 0xa7, 0xb4, 0xe2, 0xc8, 0x6f, 0x30, 0x4a, 0x6a, 0x99, 0x58, 0xe7,
- 0xa1, 0x41, 0x7e, 0xee, 0x93, 0xa2, 0xdc, 0xb3, 0x4c, 0xdc, 0x66, 0xb8, 0x8c, 0x92, 0xfa, 0x0d,
- 0x56, 0xe1, 0x70, 0xcb, 0xd6, 0x80, 0xc7, 0x1e, 0xd1, 0x50, 0x4b, 0x50, 0x0c, 0x86, 0xa8, 0xbe,
- 0x0c, 0xc5, 0x40, 0x19, 0xbd, 0x30, 0x36, 0x44, 0xf1, 0xf9, 0x42, 0xb8, 0xed, 0x05, 0xc8, 0x79,
- 0x47, 0x2e, 0xae, 0x7f, 0x91, 0x81, 0xf5, 0xc4, 0x6a, 0x04, 0xdd, 0x82, 0x45, 0x79, 0x4e, 0x21,
- 0x6d, 0xda, 0x48, 0x39, 0xc1, 0x3b, 0x42, 0x4b, 0xf3, 0xd5, 0x59, 0xb9, 0x44, 0x30, 0xb5, 0xcc,
- 0x91, 0x31, 0xd0, 0x89, 0xe3, 0x78, 0xbe, 0x73, 0xbc, 0x95, 0x12, 0x70, 0xda, 0x62, 0xd6, 0x96,
- 0x7c, 0x58, 0x8d, 0xa1, 0x26, 0xc6, 0xad, 0xec, 0x49, 0xc5, 0x2d, 0x74, 0x19, 0xd6, 0xd9, 0x82,
- 0xb2, 0x08, 0xa6, 0xba, 0xac, 0x21, 0xc4, 0x6a, 0xcf, 0x9d, 0x57, 0x36, 0x0a, 0xda, 0x9a, 0x2f,
- 0xbc, 0x11, 0x91, 0xd5, 0x5b, 0x70, 0xee, 0x59, 0xb5, 0x7f, 0xca, 0xf2, 0xb6, 0xfe, 0xe9, 0x2a,
- 0x2c, 0x4a, 0xb3, 0x22, 0x03, 0x4a, 0x6e, 0x24, 0xab, 0x57, 0x66, 0x32, 0xa5, 0x04, 0x69, 0x74,
- 0xbc, 0x58, 0x1a, 0x1f, 0xc5, 0x54, 0xbf, 0x28, 0x01, 0x84, 0xc9, 0x11, 0x7a, 0x0a, 0x7e, 0x8d,
- 0xc6, 0x62, 0xa6, 0xd8, 0xf3, 0x7c, 0xa7, 0xb8, 0x3d, 0x2b, 0x71, 0x00, 0xeb, 0x2f, 0x04, 0x6c,
- 0xb6, 0x24, 0xa4, 0x56, 0x75, 0xe3, 0x5d, 0xe8, 0x43, 0x58, 0x31, 0xba, 0x9e, 0x75, 0x88, 0x43,
- 0x62, 0xb1, 0xdc, 0x6e, 0xcd, 0x4f, 0xbc, 0xc5, 0x01, 0x03, 0xd6, 0x65, 0x63, 0xac, 0x8d, 0x2c,
- 0x80, 0xc8, 0x36, 0x2e, 0x1c, 0xa8, 0x3d, 0x3f, 0x5b, 0x7c, 0x07, 0x8f, 0x80, 0xa3, 0x9b, 0x90,
- 0x63, 0x41, 0x45, 0xe6, 0x0a, 0x97, 0x67, 0x24, 0x61, 0x2b, 0x5f, 0xe3, 0x00, 0xea, 0x1f, 0xb2,
- 0x50, 0xb8, 0x83, 0x0d, 0x3a, 0x22, 0xd8, 0x44, 0x3f, 0x54, 0x60, 0x4d, 0x24, 0x31, 0xd2, 0x66,
- 0x7a, 0xd7, 0x19, 0x89, 0x4f, 0xc6, 0x68, 0xde, 0x9d, 0x7f, 0x2e, 0x3e, 0x45, 0x83, 0x07, 0x11,
- 0x69, 0xb1, 0x26, 0x07, 0x17, 0x93, 0x43, 0xd6, 0x84, 0x00, 0x7d, 0xa2, 0xc0, 0xba, 0x4c, 0x8f,
- 0x62, 0xe3, 0x11, 0x61, 0xe0, 0xbd, 0x13, 0x18, 0x8f, 0xc8, 0x28, 0x12, 0x06, 0xb4, 0xea, 0x4c,
- 0x4a, 0xd0, 0x06, 0x54, 0x3c, 0xc7, 0x33, 0x06, 0x62, 0x3b, 0xa5, 0xae, 0x9f, 0xd2, 0x29, 0xda,
- 0x32, 0xef, 0x67, 0xfb, 0xe5, 0x1e, 0xeb, 0x55, 0x5b, 0x70, 0x66, 0xca, 0x54, 0x13, 0xd2, 0x95,
- 0xb5, 0x68, 0xba, 0x92, 0x8d, 0xe6, 0xbf, 0x37, 0xa0, 0x36, 0x6d, 0x84, 0x33, 0xe1, 0x50, 0xa8,
- 0x4e, 0xac, 0x1a, 0xf4, 0x01, 0x14, 0x86, 0xd2, 0x0e, 0x72, 0x51, 0x6e, 0x1f, 0xdf, 0xa2, 0x5a,
- 0x80, 0xa9, 0x7e, 0x92, 0x85, 0xe5, 0xf1, 0x25, 0xf3, 0xbc, 0x29, 0xd1, 0x2b, 0x80, 0x7a, 0xc4,
- 0x10, 0x31, 0x91, 0xe0, 0xa1, 0x61, 0xd9, 0x96, 0xdd, 0xe7, 0xe6, 0x50, 0xb4, 0xaa, 0x2f, 0xd1,
- 0x7c, 0x01, 0xfa, 0x99, 0x02, 0x67, 0xc7, 0x3d, 0x8c, 0x46, 0xd4, 0xc4, 0x0a, 0xc6, 0x27, 0x15,
- 0x2f, 0xc6, 0x7d, 0x8d, 0x06, 0xa3, 0x10, 0xfe, 0x76, 0xc6, 0x49, 0x96, 0xaa, 0xef, 0xc0, 0xb9,
- 0x67, 0x29, 0xce, 0xe4, 0x06, 0x6f, 0xc2, 0xca, 0x57, 0x27, 0xcf, 0xd3, 0xd5, 0x7f, 0x97, 0x87,
- 0x1c, 0x8b, 0x1d, 0x48, 0x87, 0x92, 0xd8, 0xa3, 0x75, 0xdb, 0x08, 0xf2, 0xdf, 0xeb, 0x73, 0x44,
- 0x21, 0xd9, 0xb8, 0x6b, 0x0c, 0xb1, 0x06, 0xc3, 0xe0, 0x19, 0x61, 0x28, 0xf3, 0xa5, 0x8e, 0x89,
- 0x6e, 0x1a, 0x9e, 0xe1, 0x1f, 0x93, 0xbe, 0x35, 0x0f, 0x45, 0x53, 0x00, 0xed, 0x18, 0x9e, 0x71,
- 0xeb, 0x94, 0x56, 0xea, 0x86, 0x4d, 0xe4, 0x41, 0xd5, 0xb4, 0xa8, 0x47, 0xac, 0x03, 0x91, 0xcd,
- 0x73, 0xae, 0x19, 0x4f, 0x48, 0xc7, 0xb8, 0x76, 0x22, 0x68, 0x92, 0xb0, 0x62, 0xc6, 0xfa, 0x90,
- 0x0e, 0xd0, 0x37, 0x46, 0x7d, 0x2c, 0xe8, 0xbe, 0x9c, 0xed, 0x7c, 0x72, 0x8c, 0xee, 0x26, 0x83,
- 0x91, 0x3c, 0xc5, 0xbe, 0xdf, 0x50, 0xaf, 0x03, 0x84, 0x76, 0x45, 0xe7, 0xa0, 0xc8, 0xbe, 0x12,
- 0x75, 0x8d, 0x2e, 0x96, 0xa5, 0x69, 0xd8, 0x81, 0x10, 0xe4, 0xf8, 0x37, 0xcc, 0x72, 0x01, 0x7f,
- 0x56, 0xff, 0x9d, 0x95, 0xf6, 0xa1, 0x95, 0x02, 0x87, 0x50, 0x22, 0x0e, 0xa1, 0x7e, 0x00, 0x95,
- 0xf8, 0x6c, 0xd9, 0x9b, 0xdc, 0xbc, 0xfe, 0x9b, 0xbc, 0xc1, 0x5c, 0x8c, 0x8e, 0x86, 0xd2, 0x9d,
- 0xd8, 0x23, 0xeb, 0x19, 0x5a, 0x36, 0xe7, 0xcc, 0x6a, 0xec, 0x91, 0xf7, 0x18, 0x4f, 0x78, 0x4a,
- 0xc4, 0x7a, 0x8c, 0x27, 0xea, 0x7b, 0x50, 0x0c, 0xa6, 0x97, 0x3c, 0x04, 0x74, 0x15, 0x8a, 0xc1,
- 0x15, 0x5b, 0x8a, 0x52, 0x2f, 0x7c, 0x99, 0x65, 0xb1, 0xcc, 0xf8, 0xea, 0x11, 0x54, 0xe2, 0x19,
- 0x4d, 0xc2, 0x8a, 0xb8, 0x37, 0x5e, 0x4e, 0x5e, 0x9b, 0x3b, 0x22, 0x44, 0xab, 0xcd, 0x5f, 0x66,
- 0xe0, 0x85, 0x67, 0x9e, 0xae, 0x9f, 0x60, 0x22, 0xfd, 0x7c, 0x13, 0xdc, 0xf7, 0x61, 0xc9, 0x25,
- 0xd6, 0xd0, 0x20, 0x47, 0x32, 0x4b, 0x17, 0x59, 0xc9, 0xfc, 0x65, 0x6c, 0x59, 0xc2, 0xf1, 0xec,
- 0xbc, 0xfe, 0xcd, 0x1c, 0x9c, 0x9d, 0x7a, 0x15, 0x95, 0xf6, 0x9e, 0xe7, 0x29, 0x2c, 0x9b, 0x98,
- 0x5a, 0x04, 0x9b, 0xe2, 0x26, 0xc2, 0x9f, 0xff, 0xde, 0x71, 0xef, 0xc2, 0x1a, 0x3b, 0x02, 0x96,
- 0xf7, 0xc9, 0xdc, 0x61, 0xc9, 0x8c, 0xf6, 0xa9, 0xbf, 0x56, 0xa0, 0x1c, 0x7d, 0x0b, 0x6d, 0xc2,
- 0x7a, 0xb0, 0x4b, 0x39, 0x3d, 0xb9, 0xe3, 0x98, 0x58, 0x5c, 0xd2, 0x66, 0xb4, 0x55, 0x5f, 0x78,
- 0xaf, 0xa7, 0xf9, 0x22, 0x74, 0x11, 0xd6, 0x8c, 0xc1, 0xc0, 0x79, 0xec, 0x4f, 0x40, 0x17, 0x97,
- 0xd3, 0x7c, 0x1a, 0x59, 0x0d, 0x49, 0x19, 0xc7, 0xef, 0x70, 0x09, 0xba, 0x0a, 0x35, 0x4c, 0x3d,
- 0x6b, 0x68, 0xb0, 0xfa, 0x7f, 0x2c, 0xad, 0xa3, 0x72, 0x2d, 0x9e, 0x0e, 0xe4, 0xd1, 0x5c, 0x85,
- 0xaa, 0x9f, 0x28, 0x80, 0x26, 0xa7, 0x95, 0xb0, 0x30, 0xba, 0xe3, 0x0b, 0xe3, 0xce, 0x89, 0x1a,
- 0x33, 0xba, 0x58, 0xfe, 0x96, 0x05, 0x75, 0xfa, 0x65, 0xd0, 0xa4, 0x07, 0x2a, 0x27, 0xe9, 0x81,
- 0xff, 0xb4, 0x3a, 0x74, 0x04, 0xcb, 0xdd, 0x87, 0x86, 0x6d, 0xe3, 0xc1, 0xb8, 0x93, 0xde, 0x3d,
- 0xf6, 0x75, 0x59, 0xa3, 0x29, 0x70, 0x45, 0xe7, 0x52, 0x37, 0xd2, 0xa2, 0xea, 0x67, 0x0a, 0x94,
- 0xa3, 0xf2, 0x34, 0xc7, 0x9d, 0x17, 0x61, 0x6d, 0x60, 0x50, 0x4f, 0xf7, 0xcd, 0xee, 0x1f, 0x70,
- 0x32, 0x47, 0xc8, 0x6b, 0x88, 0xc9, 0x3a, 0x42, 0x24, 0xbd, 0x0a, 0x5d, 0x81, 0xd3, 0x3d, 0x8b,
- 0x50, 0x4f, 0x0f, 0x4c, 0x19, 0x3d, 0x14, 0xcd, 0x6b, 0x6b, 0x5c, 0xaa, 0x49, 0xa1, 0xd4, 0xaa,
- 0x5f, 0x87, 0xf5, 0xc4, 0x4b, 0xe1, 0xb4, 0x05, 0x70, 0x0d, 0x4e, 0x27, 0xdf, 0xe8, 0xd5, 0x3f,
- 0x57, 0xa0, 0x10, 0xe4, 0xa5, 0xb7, 0xc4, 0x7e, 0x20, 0xfd, 0xe6, 0x4a, 0x4a, 0x7b, 0x07, 0x99,
- 0x1d, 0xdb, 0xa3, 0x34, 0xb1, 0xa3, 0x98, 0x90, 0xe3, 0x3b, 0x56, 0xca, 0xb8, 0x14, 0x37, 0x75,
- 0x66, 0xd2, 0xd4, 0x48, 0x8e, 0x4d, 0x9c, 0x1d, 0xf3, 0xe7, 0xfa, 0x4f, 0xb2, 0x50, 0xe6, 0x67,
- 0x37, 0xbe, 0x39, 0xe2, 0x37, 0x78, 0x93, 0xf4, 0x99, 0x24, 0xfa, 0x5d, 0x28, 0x8a, 0xbb, 0x19,
- 0xb6, 0xb0, 0xc5, 0xc1, 0xe0, 0x85, 0x94, 0x93, 0xe7, 0xf4, 0xb7, 0xf1, 0x91, 0x56, 0xa0, 0xf2,
- 0x09, 0xdd, 0x86, 0x6c, 0x1f, 0x7b, 0xb3, 0xfe, 0xb0, 0xc1, 0x81, 0x6e, 0xe2, 0xc8, 0xcf, 0x05,
- 0x0c, 0x05, 0xed, 0xc3, 0x82, 0xe1, 0xba, 0xd8, 0x36, 0xfd, 0xe4, 0xef, 0xda, 0x2c, 0x78, 0x5b,
- 0x5c, 0x35, 0x84, 0x94, 0x58, 0xe8, 0xbf, 0x21, 0xdf, 0x1d, 0x60, 0x83, 0xf8, 0x59, 0xde, 0xd5,
- 0x59, 0x40, 0x9b, 0x4c, 0x33, 0xc4, 0x14, 0x48, 0xd1, 0x9f, 0x11, 0x3e, 0xcf, 0xc0, 0x92, 0xfc,
- 0x2c, 0x32, 0x32, 0xc5, 0xbf, 0x4b, 0xf2, 0xff, 0x06, 0xbb, 0x63, 0x86, 0x7b, 0x6d, 0x66, 0xc3,
- 0x05, 0x97, 0xd4, 0xdc, 0x72, 0xf7, 0xe3, 0x96, 0x7b, 0x7d, 0x1e, 0xcb, 0x05, 0x98, 0xbe, 0xe9,
- 0xb4, 0x98, 0xe9, 0xae, 0xcd, 0x61, 0xba, 0x00, 0x54, 0xda, 0x2e, 0x7a, 0x89, 0xfe, 0x9b, 0x02,
- 0x14, 0x7c, 0xa7, 0x42, 0x1d, 0x58, 0x10, 0xbf, 0x64, 0xc9, 0xd4, 0xe7, 0xd5, 0x19, 0xbd, 0xb2,
- 0xa1, 0x71, 0x6d, 0x36, 0x7c, 0x81, 0x83, 0x28, 0xac, 0x0e, 0x47, 0x03, 0xb6, 0xdf, 0xb9, 0xfa,
- 0xc4, 0x19, 0xec, 0xd6, 0xac, 0xf0, 0x77, 0x24, 0x54, 0xf4, 0xd0, 0xb5, 0x3a, 0x8c, 0x77, 0x22,
- 0x13, 0x96, 0x0f, 0x8c, 0xbe, 0x1e, 0x39, 0x66, 0xce, 0xce, 0xf4, 0xbf, 0x47, 0xc0, 0xb7, 0x6d,
- 0xf4, 0xa3, 0x47, 0xca, 0xe5, 0x83, 0x48, 0x9b, 0x4d, 0xcd, 0xf2, 0x30, 0x31, 0x0e, 0x06, 0x38,
- 0x3a, 0xb5, 0xdc, 0x7c, 0x53, 0x6b, 0x4b, 0xa8, 0xb1, 0xa9, 0x59, 0xf1, 0x4e, 0xf4, 0x0d, 0x05,
- 0x6a, 0x81, 0x41, 0x1f, 0xe1, 0x23, 0x1a, 0xa5, 0xce, 0x73, 0xea, 0xd6, 0xbc, 0x56, 0xbd, 0x8d,
- 0x8f, 0x68, 0x94, 0x7e, 0x7d, 0x98, 0x24, 0x50, 0x55, 0x58, 0x10, 0x9f, 0x39, 0x9a, 0x9a, 0x94,
- 0x79, 0x6a, 0xa2, 0x12, 0xa8, 0x4e, 0x4c, 0x24, 0xcd, 0xc6, 0x56, 0x87, 0xa5, 0x70, 0x1e, 0x91,
- 0x88, 0x1c, 0x9c, 0x80, 0xb7, 0x4d, 0x74, 0x1a, 0x16, 0xc4, 0x15, 0xbe, 0x8c, 0xc9, 0xb2, 0xa5,
- 0x7e, 0x47, 0x81, 0xea, 0x84, 0x63, 0x3c, 0x67, 0x52, 0x7f, 0xea, 0xb9, 0x70, 0xea, 0x87, 0xb0,
- 0x9e, 0x68, 0xc8, 0xe7, 0x3d, 0xfd, 0x8f, 0xa1, 0x1c, 0x75, 0xd3, 0x94, 0x74, 0xe1, 0xda, 0x88,
- 0xd0, 0x05, 0x37, 0x26, 0xb3, 0x4c, 0x3c, 0xb8, 0x9b, 0x78, 0x1b, 0x56, 0x62, 0x9b, 0x0a, 0x7a,
- 0x05, 0x50, 0xd7, 0xb1, 0x3d, 0xcb, 0x1e, 0x19, 0xe2, 0xde, 0x8e, 0x5f, 0x89, 0x08, 0x7f, 0xa9,
- 0x46, 0x25, 0xfc, 0x2e, 0xa5, 0x7e, 0x1f, 0x2a, 0xf1, 0xe8, 0x3a, 0x23, 0x44, 0xb0, 0x6d, 0x67,
- 0x22, 0xdb, 0xf6, 0x06, 0xa0, 0xc9, 0xdd, 0x29, 0x78, 0x53, 0x89, 0xbc, 0xb9, 0x0e, 0xab, 0x09,
- 0xd1, 0xb8, 0xbe, 0x0a, 0xd5, 0x89, 0x9d, 0xa8, 0xbe, 0x26, 0x51, 0xc7, 0x62, 0x6c, 0xfd, 0xe7,
- 0x39, 0x28, 0xec, 0x3a, 0xf2, 0x90, 0xe8, 0xff, 0xa0, 0x40, 0xf1, 0x21, 0x26, 0x96, 0x27, 0x16,
- 0xc9, 0x72, 0xea, 0xf3, 0x06, 0x1f, 0xa2, 0xb1, 0x27, 0xf5, 0xc5, 0xad, 0x6f, 0x00, 0x37, 0x7f,
- 0x11, 0x8e, 0x6a, 0xac, 0xbe, 0xa5, 0xd4, 0xe8, 0xfb, 0xa7, 0x0f, 0x7e, 0x93, 0xdf, 0x59, 0x11,
- 0xa3, 0x8b, 0xf9, 0xc7, 0x2d, 0x6a, 0xa2, 0x91, 0x90, 0xd3, 0xe4, 0xd3, 0xa4, 0x54, 0x0b, 0x93,
- 0x6e, 0xf7, 0x22, 0x94, 0x07, 0x4e, 0x5f, 0x1f, 0x38, 0xf2, 0xde, 0x75, 0x51, 0xbc, 0x32, 0x70,
- 0xfa, 0xbb, 0xb2, 0x8b, 0x79, 0x9d, 0xf7, 0x90, 0x60, 0xc3, 0xac, 0x15, 0xb8, 0x50, 0xb6, 0xd4,
- 0xff, 0x85, 0xdc, 0xae, 0x45, 0x3d, 0xd4, 0x01, 0xf6, 0xba, 0x8e, 0x6d, 0x8f, 0x58, 0xd8, 0x2f,
- 0x38, 0x2e, 0xcc, 0x68, 0x54, 0x0d, 0x06, 0xe2, 0xc9, 0xc2, 0x54, 0x25, 0x50, 0xf0, 0x6d, 0x5c,
- 0xef, 0x41, 0x8e, 0x99, 0x19, 0xad, 0x40, 0xe9, 0xfe, 0xdd, 0xbd, 0x4e, 0xab, 0xd9, 0xbe, 0xd1,
- 0x6e, 0xed, 0x54, 0x4e, 0xa1, 0x22, 0xe4, 0xf7, 0xb5, 0xad, 0x66, 0xab, 0xa2, 0xb0, 0xc7, 0x9d,
- 0xd6, 0xf6, 0xfd, 0x9b, 0x95, 0x0c, 0x2a, 0x40, 0xae, 0x7d, 0xf7, 0xc6, 0xbd, 0x4a, 0x16, 0x01,
- 0x2c, 0xdc, 0xbd, 0xb7, 0xdf, 0x6e, 0xb6, 0x2a, 0x39, 0xd6, 0xfb, 0x60, 0x4b, 0xbb, 0x5b, 0xc9,
- 0xb3, 0x57, 0x5b, 0x9a, 0x76, 0x4f, 0xab, 0x2c, 0xa0, 0x32, 0x14, 0x9a, 0x5a, 0x7b, 0xbf, 0xdd,
- 0xdc, 0xda, 0xad, 0x2c, 0xd6, 0xcb, 0x00, 0xbb, 0x4e, 0xbf, 0xe9, 0xd8, 0x1e, 0x71, 0x06, 0xf5,
- 0xdf, 0xe7, 0xb8, 0x27, 0x11, 0xef, 0x81, 0x43, 0x1e, 0x85, 0x7f, 0xb2, 0xfd, 0x0b, 0x14, 0x1f,
- 0xf3, 0x8e, 0x70, 0x11, 0x17, 0x44, 0x47, 0xdb, 0x44, 0x07, 0x50, 0xe9, 0x0a, 0x75, 0xdd, 0xff,
- 0x63, 0x5a, 0x7a, 0xc1, 0xdc, 0x7f, 0xf4, 0xac, 0x48, 0xc0, 0x96, 0xc4, 0x63, 0x1c, 0x03, 0xa7,
- 0xdf, 0xb7, 0xec, 0x7e, 0xc8, 0x91, 0x3d, 0x26, 0x87, 0x04, 0x0c, 0x38, 0x4c, 0xa8, 0x1a, 0xc4,
- 0xb3, 0x7a, 0x46, 0xd7, 0x0b, 0x49, 0x72, 0xc7, 0x23, 0xa9, 0xf8, 0x88, 0x01, 0x4b, 0x8f, 0xdf,
- 0x88, 0x1d, 0x5a, 0x94, 0x39, 0x70, 0x40, 0x93, 0x3f, 0x1e, 0x4d, 0x35, 0x80, 0x0c, 0x78, 0xde,
- 0x87, 0x05, 0xd7, 0x20, 0xc6, 0x90, 0xd6, 0x80, 0x3b, 0xe6, 0x0c, 0x3b, 0x71, 0xec, 0xeb, 0x37,
- 0x3a, 0x1c, 0x47, 0xfe, 0x48, 0x26, 0x40, 0xd5, 0x6b, 0x50, 0x8a, 0x74, 0x7f, 0xd5, 0x19, 0x72,
- 0x31, 0x5a, 0xc9, 0xff, 0x27, 0x0f, 0x6c, 0x21, 0x89, 0x0c, 0xae, 0x41, 0x5e, 0xac, 0x44, 0xf2,
- 0xe2, 0xfa, 0x45, 0x16, 0xee, 0x1c, 0x37, 0xbd, 0x3b, 0xd6, 0x5f, 0x66, 0x1e, 0x1c, 0x6a, 0x3c,
- 0x0b, 0x7d, 0xf3, 0x53, 0x05, 0x96, 0xb6, 0xb1, 0x31, 0xbc, 0x61, 0xcb, 0x05, 0x80, 0xbe, 0xab,
- 0xc0, 0xa2, 0xff, 0x9c, 0x36, 0x69, 0x4e, 0xf8, 0xf9, 0x58, 0xbd, 0x36, 0x8f, 0xae, 0x08, 0xe6,
- 0xa7, 0x36, 0x94, 0x8b, 0xca, 0xe6, 0x47, 0x00, 0x62, 0x64, 0xbc, 0x96, 0xb4, 0x65, 0x4d, 0x79,
- 0x61, 0xc6, 0xba, 0x54, 0x9d, 0x55, 0x41, 0xb2, 0x7f, 0x4f, 0x81, 0x92, 0xa0, 0x17, 0x1b, 0xf9,
- 0x13, 0xc8, 0x8b, 0x87, 0xcb, 0xb3, 0x24, 0x74, 0x72, 0x46, 0xea, 0x95, 0xd9, 0x94, 0xe4, 0xf6,
- 0x25, 0x46, 0xf2, 0xfd, 0xe0, 0x13, 0xed, 0x8a, 0xf5, 0x8a, 0x9e, 0xc0, 0xa2, 0xff, 0x78, 0x65,
- 0xd6, 0x2d, 0x8c, 0x05, 0x6e, 0xf5, 0x52, 0x7a, 0x2d, 0x3f, 0x2e, 0x8a, 0xb1, 0xfc, 0x2a, 0x03,
- 0x35, 0x31, 0x96, 0xd6, 0x13, 0x0f, 0x13, 0xdb, 0x18, 0x08, 0x2f, 0xeb, 0x38, 0xc2, 0x73, 0x4a,
- 0x11, 0xbf, 0x46, 0xd7, 0xe6, 0x5e, 0x70, 0xea, 0xeb, 0xf3, 0xa8, 0xfa, 0x56, 0x43, 0xdf, 0x56,
- 0x00, 0xc2, 0x15, 0x80, 0xd2, 0xd7, 0xb7, 0xb1, 0x65, 0xa6, 0x5e, 0x9b, 0x43, 0xd3, 0x1f, 0xc5,
- 0xf6, 0x16, 0xfc, 0xc7, 0x34, 0xed, 0xa8, 0xf2, 0x76, 0x51, 0x18, 0x74, 0xcb, 0xb5, 0xde, 0x5d,
- 0x8e, 0x88, 0xf4, 0xc3, 0x4b, 0x07, 0x0b, 0x3c, 0x79, 0xb8, 0xfc, 0x8f, 0x00, 0x00, 0x00, 0xff,
- 0xff, 0xdb, 0x09, 0x17, 0x4b, 0xda, 0x33, 0x00, 0x00,
+ // 3212 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xcd, 0x6f, 0x1b, 0xc7,
+ 0xd9, 0xf7, 0x8a, 0x94, 0x44, 0x3e, 0xa4, 0x24, 0x72, 0x24, 0xd9, 0xf4, 0xbe, 0xce, 0xfb, 0x3a,
+ 0x7c, 0x1b, 0x40, 0x48, 0x11, 0xfa, 0x13, 0x89, 0x9d, 0x26, 0x4e, 0x24, 0x8a, 0xb6, 0x19, 0xcb,
+ 0x36, 0xbb, 0xb2, 0xeb, 0x36, 0x69, 0xb2, 0x58, 0x71, 0x87, 0xf4, 0xc2, 0xcb, 0xdd, 0xcd, 0xcc,
+ 0x52, 0x96, 0xdc, 0xa0, 0xe9, 0x17, 0x5a, 0xb4, 0x68, 0x9b, 0x4b, 0x0f, 0x69, 0x6f, 0x6d, 0x81,
+ 0x02, 0xbd, 0xf4, 0x0f, 0xc8, 0xb5, 0x87, 0xa2, 0xa7, 0xfe, 0x03, 0xb9, 0x14, 0x68, 0x81, 0xb6,
+ 0xe9, 0xb9, 0x40, 0x6f, 0xc5, 0x7c, 0xec, 0x07, 0x97, 0x4b, 0x67, 0x49, 0xc9, 0xbd, 0xed, 0xcc,
+ 0xec, 0xf3, 0xfb, 0xcd, 0x3c, 0xfb, 0x9b, 0x67, 0x9e, 0x99, 0x59, 0xa8, 0xee, 0x61, 0x63, 0xa0,
+ 0xf7, 0x1c, 0xdd, 0xf0, 0xac, 0x86, 0x47, 0x5c, 0xdf, 0x45, 0x2f, 0xb8, 0xa4, 0xdf, 0x30, 0x3c,
+ 0xa3, 0xfb, 0x10, 0x37, 0x58, 0x6b, 0x63, 0xe0, 0x9a, 0xd8, 0x6e, 0xf4, 0x1c, 0x1d, 0x1f, 0xe0,
+ 0xee, 0xd0, 0xb7, 0x5c, 0xa7, 0xb1, 0x7f, 0x41, 0x5d, 0xe7, 0x96, 0x64, 0xe8, 0x38, 0x98, 0x44,
+ 0xd6, 0xea, 0x0a, 0x76, 0x4c, 0xcf, 0xb5, 0x1c, 0x9f, 0xca, 0x8a, 0xb3, 0x7d, 0xd7, 0xed, 0xdb,
+ 0xf8, 0x1c, 0x2f, 0xed, 0x0d, 0x7b, 0xe7, 0x4c, 0x4c, 0xbb, 0xc4, 0xf2, 0x7c, 0x97, 0xc8, 0x37,
+ 0xfe, 0x2f, 0xf9, 0x86, 0x6f, 0x0d, 0x30, 0xf5, 0x8d, 0x81, 0x27, 0x5f, 0xf8, 0xdf, 0x31, 0x88,
+ 0x21, 0x31, 0x78, 0x3f, 0x26, 0xb4, 0x3f, 0x26, 0x86, 0xe7, 0x61, 0x12, 0x74, 0x61, 0x69, 0x80,
+ 0x7d, 0x62, 0x75, 0x65, 0xb1, 0xfe, 0x0b, 0x05, 0x96, 0x35, 0x3c, 0x70, 0x7d, 0x7c, 0x83, 0x78,
+ 0xdd, 0x8e, 0x4b, 0x7c, 0x34, 0x80, 0x93, 0x86, 0x67, 0xe9, 0x14, 0x93, 0x7d, 0xab, 0x8b, 0xf5,
+ 0xa8, 0x8b, 0x35, 0xe5, 0xac, 0xb2, 0x51, 0xba, 0xf8, 0x4a, 0x23, 0xdd, 0x29, 0x9e, 0xe5, 0x61,
+ 0xdb, 0x72, 0x70, 0x63, 0xff, 0x42, 0x63, 0xd3, 0xb3, 0x76, 0x85, 0xfd, 0x76, 0x68, 0xae, 0xad,
+ 0x19, 0x29, 0xb5, 0xe8, 0x34, 0x14, 0xba, 0xae, 0x89, 0x89, 0x6e, 0x99, 0xb5, 0xb9, 0xb3, 0xca,
+ 0x46, 0x51, 0x5b, 0xe4, 0xe5, 0xb6, 0x59, 0xff, 0x6b, 0x1e, 0x50, 0xdb, 0xa1, 0x3e, 0x19, 0x76,
+ 0xd9, 0x08, 0x35, 0xfc, 0xfe, 0x10, 0x53, 0x1f, 0xbd, 0x00, 0xcb, 0x56, 0x54, 0xcb, 0xec, 0x14,
+ 0x6e, 0xb7, 0x14, 0xab, 0x6d, 0x9b, 0xe8, 0x3e, 0x14, 0x08, 0xee, 0x5b, 0xd4, 0xc7, 0xa4, 0xf6,
+ 0xb7, 0x45, 0xde, 0xf5, 0x97, 0x1b, 0x99, 0xbe, 0x67, 0x43, 0x93, 0x76, 0x92, 0xf1, 0xe6, 0x09,
+ 0x2d, 0x84, 0x42, 0x18, 0x96, 0x3d, 0xe2, 0x76, 0x31, 0xa5, 0xfa, 0xde, 0xd0, 0x31, 0x6d, 0x5c,
+ 0xfb, 0xbb, 0x00, 0xff, 0x52, 0x46, 0xf0, 0x8e, 0xb0, 0xde, 0xe2, 0xc6, 0x11, 0xc3, 0x92, 0x17,
+ 0xaf, 0x47, 0xdf, 0x84, 0x53, 0xa3, 0x34, 0xba, 0x47, 0xdc, 0x3e, 0xc1, 0x94, 0xd6, 0xfe, 0x21,
+ 0xf8, 0x9a, 0xb3, 0xf0, 0x75, 0x24, 0x48, 0xc4, 0xbb, 0xee, 0xa5, 0xb5, 0xa3, 0x21, 0xac, 0x25,
+ 0xf8, 0xa9, 0x67, 0x5b, 0x7e, 0xed, 0x33, 0x41, 0xfe, 0xe6, 0x2c, 0xe4, 0xbb, 0x0c, 0x21, 0x62,
+ 0x46, 0xde, 0x58, 0x23, 0x7a, 0x08, 0x2b, 0x3d, 0xcb, 0x31, 0x6c, 0xeb, 0x09, 0x0e, 0xdc, 0xfb,
+ 0x4f, 0xc1, 0xf8, 0x5a, 0x46, 0xc6, 0xeb, 0xd2, 0x3c, 0xe9, 0xdf, 0xe5, 0xde, 0x48, 0xc3, 0x56,
+ 0x11, 0x16, 0x89, 0x68, 0xac, 0x7f, 0x67, 0x1e, 0x56, 0x47, 0x74, 0x46, 0x3d, 0xd7, 0xa1, 0x38,
+ 0xab, 0xd0, 0xd6, 0x60, 0x1e, 0x13, 0xe2, 0x12, 0x29, 0x5f, 0x51, 0x40, 0x5f, 0x19, 0x97, 0xdf,
+ 0x2b, 0x53, 0xcb, 0x4f, 0x74, 0x64, 0x44, 0x7f, 0xbd, 0x49, 0xfa, 0x7b, 0x6d, 0x36, 0xfd, 0x85,
+ 0x14, 0x09, 0x01, 0x7e, 0xf8, 0xb9, 0x02, 0xdc, 0x3e, 0x9a, 0x00, 0x43, 0xe2, 0x09, 0x0a, 0xdc,
+ 0x7f, 0xba, 0x02, 0x37, 0x8f, 0xa0, 0xc0, 0x90, 0x3a, 0x4d, 0x82, 0xd6, 0x44, 0x09, 0xbe, 0x3e,
+ 0xa3, 0x04, 0x43, 0xba, 0xa4, 0x06, 0x81, 0x69, 0x44, 0xb4, 0xd6, 0x7f, 0xa2, 0xc0, 0x4a, 0x22,
+ 0xee, 0xa0, 0x27, 0x70, 0x3a, 0xe1, 0x82, 0x91, 0x68, 0x9c, 0xdb, 0x28, 0x5d, 0xbc, 0x36, 0x8b,
+ 0x1b, 0x62, 0x41, 0xf9, 0x94, 0x97, 0xde, 0x50, 0x47, 0x50, 0x49, 0xea, 0xb0, 0xfe, 0x6b, 0x80,
+ 0x53, 0x13, 0x80, 0xd0, 0x32, 0xcc, 0x85, 0x13, 0x64, 0xce, 0x32, 0x91, 0x03, 0xe0, 0x13, 0xc3,
+ 0xa1, 0x3d, 0x97, 0x0c, 0x68, 0x6d, 0x8e, 0x77, 0xf6, 0xce, 0xd1, 0x3a, 0xdb, 0xb8, 0x17, 0x02,
+ 0xb6, 0x1c, 0x9f, 0x1c, 0x6a, 0x31, 0x06, 0xe4, 0x43, 0xd9, 0xeb, 0xba, 0xb6, 0x8d, 0xf9, 0xb4,
+ 0xa4, 0xb5, 0x1c, 0x67, 0xec, 0x1c, 0x91, 0xb1, 0x13, 0x83, 0x14, 0x9c, 0x23, 0x2c, 0xe8, 0x47,
+ 0x0a, 0xac, 0x3d, 0xb6, 0x1c, 0xd3, 0x7d, 0x6c, 0x39, 0x7d, 0x9d, 0xfa, 0xc4, 0xf0, 0x71, 0xdf,
+ 0xc2, 0xb4, 0x96, 0xe7, 0xf4, 0x0f, 0x8e, 0x48, 0xff, 0x20, 0x80, 0xde, 0x0d, 0x91, 0x45, 0x2f,
+ 0x56, 0x1f, 0x8f, 0xb7, 0xa0, 0x3d, 0x58, 0xe0, 0x4b, 0x27, 0xad, 0xcd, 0x73, 0xf6, 0xb7, 0x8e,
+ 0xc8, 0xde, 0xe4, 0x60, 0x82, 0x50, 0x22, 0x33, 0x37, 0x63, 0x67, 0xdf, 0x22, 0xae, 0x33, 0xc0,
+ 0x8e, 0x4f, 0x6b, 0x0b, 0xc7, 0xe2, 0xe6, 0x56, 0x0c, 0x52, 0xba, 0x39, 0xce, 0x82, 0x0e, 0xe0,
+ 0x0c, 0xf5, 0x0d, 0x1f, 0xeb, 0x13, 0x32, 0x93, 0xc5, 0xa3, 0x65, 0x26, 0xa7, 0x39, 0x78, 0x5a,
+ 0x93, 0x6a, 0xc3, 0x4a, 0x42, 0x75, 0xa8, 0x02, 0xb9, 0x47, 0xf8, 0x50, 0x4a, 0x9d, 0x3d, 0xa2,
+ 0x26, 0xcc, 0xef, 0x1b, 0xf6, 0x10, 0xf3, 0x15, 0xa0, 0x74, 0xf1, 0xa5, 0x0c, 0xfd, 0xe8, 0x84,
+ 0xa8, 0x9a, 0xb0, 0x7d, 0x75, 0xee, 0x8a, 0xa2, 0xba, 0x50, 0x1d, 0x53, 0x5c, 0x0a, 0xdf, 0xf6,
+ 0x28, 0x5f, 0x23, 0x0b, 0x5f, 0x33, 0x84, 0x8d, 0x13, 0x7e, 0x00, 0xb5, 0x49, 0x1a, 0x4b, 0xe1,
+ 0x7d, 0x6b, 0x94, 0xf7, 0x72, 0x06, 0xde, 0x24, 0xfa, 0x61, 0x9c, 0xbd, 0x0b, 0xa5, 0x98, 0xc6,
+ 0x52, 0x08, 0xaf, 0x8d, 0x12, 0x6e, 0x64, 0x20, 0xe4, 0x80, 0x09, 0x9f, 0x8e, 0xc9, 0xeb, 0x78,
+ 0x7c, 0x1a, 0x83, 0x8d, 0x11, 0xd6, 0xff, 0x9d, 0x83, 0xaa, 0x50, 0xf8, 0xa6, 0xe7, 0xd9, 0x56,
+ 0x97, 0xa7, 0xe7, 0xe8, 0x79, 0x28, 0x87, 0xd1, 0x2a, 0x4a, 0x25, 0x4a, 0x61, 0x5d, 0xdb, 0x64,
+ 0xa9, 0xb0, 0xe5, 0x78, 0x43, 0x3f, 0x96, 0x0a, 0xf3, 0x72, 0xdb, 0x44, 0x35, 0x58, 0xc4, 0x36,
+ 0x66, 0x4c, 0xb5, 0xdc, 0x59, 0x65, 0xa3, 0xac, 0x05, 0x45, 0xf4, 0x0d, 0xa8, 0xba, 0x43, 0x9f,
+ 0x59, 0x3d, 0x36, 0x7c, 0x4c, 0x06, 0x06, 0x79, 0x14, 0x44, 0x9f, 0xac, 0xe1, 0x76, 0xac, 0xb3,
+ 0x8d, 0xbb, 0x1c, 0xf1, 0x41, 0x08, 0x28, 0xe6, 0x64, 0xc5, 0x4d, 0x54, 0xa3, 0x0e, 0x80, 0x45,
+ 0xf5, 0x3d, 0x77, 0xe8, 0x98, 0xd8, 0xac, 0xcd, 0x9f, 0x55, 0x36, 0x96, 0x2f, 0x5e, 0xc8, 0xe0,
+ 0xb9, 0x36, 0xdd, 0x12, 0x36, 0x8d, 0x96, 0x33, 0x1c, 0x68, 0x45, 0x2b, 0x28, 0xa3, 0xaf, 0x43,
+ 0x65, 0xe0, 0x3a, 0x96, 0xef, 0x12, 0x16, 0x50, 0x2d, 0xa7, 0xe7, 0x06, 0x31, 0x26, 0x0b, 0xee,
+ 0xed, 0xd0, 0xb4, 0xed, 0xf4, 0x5c, 0x6d, 0x65, 0x30, 0x52, 0xa6, 0xaa, 0x0e, 0xeb, 0xa9, 0x43,
+ 0x4b, 0xd1, 0xc3, 0xf9, 0x51, 0x3d, 0xa8, 0x0d, 0xb1, 0xb1, 0x6a, 0x04, 0x1b, 0xab, 0xc6, 0xbd,
+ 0x60, 0x67, 0x16, 0xff, 0xf6, 0xbf, 0x57, 0xa0, 0xb6, 0x8d, 0x6d, 0xe3, 0x10, 0x9b, 0xe3, 0x12,
+ 0x78, 0x1b, 0x4a, 0x46, 0x54, 0x94, 0xdb, 0xa9, 0x2b, 0xb3, 0x7e, 0x24, 0x2d, 0x0e, 0x86, 0x6e,
+ 0xc1, 0x9a, 0x4c, 0x67, 0xb1, 0xa9, 0xb3, 0x4d, 0xa3, 0x6e, 0xb2, 0x6e, 0xc8, 0xde, 0x9f, 0x1e,
+ 0xeb, 0xfd, 0xb6, 0xdc, 0x36, 0x6a, 0x28, 0x34, 0x63, 0x03, 0xe2, 0x7d, 0xaf, 0xff, 0x34, 0x0f,
+ 0x6b, 0x69, 0xdb, 0x14, 0xf4, 0x06, 0x9c, 0x99, 0x98, 0x90, 0x44, 0xa2, 0x3e, 0x3d, 0x21, 0xa7,
+ 0x68, 0x9b, 0xc8, 0x82, 0x72, 0x97, 0x8d, 0x54, 0xf7, 0xdd, 0x47, 0xd8, 0x09, 0xf2, 0x82, 0xeb,
+ 0x47, 0xd8, 0x3a, 0x35, 0x9a, 0xcc, 0xea, 0x1e, 0x83, 0xd3, 0x4a, 0xdd, 0xf0, 0x99, 0xaa, 0x7f,
+ 0x9c, 0x03, 0x88, 0xda, 0xd0, 0xfb, 0x00, 0x43, 0x8a, 0x89, 0xce, 0x43, 0xbd, 0xf4, 0x7d, 0xe7,
+ 0x78, 0x78, 0x1b, 0xf7, 0x29, 0x26, 0xbb, 0x0c, 0xf7, 0xe6, 0x09, 0xad, 0x38, 0x0c, 0x0a, 0x8c,
+ 0x92, 0x5a, 0x26, 0xd6, 0xf9, 0x24, 0x96, 0x5f, 0xe2, 0xb8, 0x28, 0x77, 0x2d, 0x13, 0xb7, 0x19,
+ 0x2e, 0xa3, 0xa4, 0x41, 0x81, 0xed, 0x45, 0xb8, 0x67, 0x6b, 0xc0, 0xa3, 0x84, 0x28, 0xa8, 0x25,
+ 0x28, 0x86, 0x5d, 0x54, 0x5f, 0x84, 0x62, 0x68, 0x8c, 0x9e, 0x1b, 0xe9, 0xa2, 0xf8, 0x7c, 0x11,
+ 0xdc, 0xd6, 0x02, 0xe4, 0xfd, 0x43, 0x0f, 0xd7, 0x3f, 0x9d, 0x83, 0xf5, 0xd4, 0x7d, 0x03, 0xba,
+ 0x09, 0x8b, 0xf2, 0x44, 0x41, 0xfa, 0xb4, 0x91, 0x71, 0x80, 0xb7, 0x85, 0x95, 0x16, 0x98, 0xb3,
+ 0x8d, 0x0d, 0xc1, 0xd4, 0x32, 0x87, 0x86, 0xad, 0x13, 0xd7, 0xf5, 0x03, 0x71, 0xbc, 0x91, 0x11,
+ 0x70, 0xd2, 0xb4, 0xd3, 0x96, 0x02, 0x58, 0x8d, 0xa1, 0xa6, 0x46, 0x98, 0xdc, 0x71, 0x45, 0x18,
+ 0x74, 0x09, 0xd6, 0xd9, 0x84, 0xb2, 0x08, 0xa6, 0xba, 0xcc, 0xf6, 0xc5, 0x6c, 0xcf, 0x9f, 0x55,
+ 0x36, 0x0a, 0xda, 0x5a, 0xd0, 0x78, 0x3d, 0xd6, 0x56, 0x6f, 0xc1, 0x99, 0xa7, 0xed, 0xd2, 0x33,
+ 0x6e, 0x44, 0xeb, 0x1f, 0xaf, 0xc2, 0xa2, 0x74, 0x2b, 0x32, 0xa0, 0xe4, 0xc5, 0xf2, 0x6f, 0x65,
+ 0x2a, 0x57, 0x4a, 0x90, 0x46, 0xc7, 0x4f, 0x24, 0xdc, 0x71, 0x4c, 0xf5, 0xd3, 0x12, 0x40, 0x94,
+ 0xc6, 0xa0, 0x27, 0x10, 0xec, 0xa6, 0xb0, 0xa9, 0xcb, 0xd5, 0x29, 0x10, 0xc5, 0xad, 0x69, 0x89,
+ 0x43, 0xd8, 0x60, 0x22, 0x60, 0xb3, 0x25, 0x21, 0xb5, 0xaa, 0x97, 0xac, 0x42, 0xef, 0xc3, 0x8a,
+ 0xd1, 0xf5, 0xad, 0x7d, 0x1c, 0x11, 0x8b, 0xe9, 0x76, 0x73, 0x76, 0xe2, 0x4d, 0x0e, 0x18, 0xb2,
+ 0x2e, 0x1b, 0x23, 0x65, 0x64, 0x01, 0xc4, 0x16, 0x5c, 0x21, 0xa0, 0xf6, 0xec, 0x6c, 0xc9, 0xb5,
+ 0x36, 0x06, 0x8e, 0x6e, 0x40, 0x9e, 0x05, 0x15, 0xb9, 0xaa, 0x5f, 0x9a, 0x92, 0x84, 0xcd, 0x7c,
+ 0x8d, 0x03, 0xa8, 0x7f, 0xc9, 0x41, 0xe1, 0x36, 0x36, 0xe8, 0x90, 0x60, 0x13, 0xfd, 0x58, 0x81,
+ 0x35, 0x91, 0x6e, 0x48, 0x9f, 0xe9, 0x5d, 0x77, 0x28, 0x3e, 0x19, 0xa3, 0x79, 0x7b, 0xf6, 0xb1,
+ 0x04, 0x14, 0x0d, 0x1e, 0x44, 0xa4, 0xc7, 0x9a, 0x1c, 0x5c, 0x0c, 0x0e, 0x59, 0x63, 0x0d, 0xe8,
+ 0x23, 0x05, 0xd6, 0x65, 0x22, 0x93, 0xe8, 0x8f, 0x08, 0x03, 0xef, 0x1c, 0x43, 0x7f, 0xc4, 0xda,
+ 0x9f, 0xd2, 0xa1, 0x55, 0x77, 0xbc, 0x05, 0x6d, 0x40, 0xc5, 0x77, 0x7d, 0xc3, 0x16, 0xcb, 0x29,
+ 0xf5, 0x82, 0xe4, 0x4b, 0xd1, 0x96, 0x79, 0x3d, 0x5b, 0x2f, 0x77, 0x59, 0xad, 0xda, 0x82, 0x53,
+ 0x13, 0x86, 0x9a, 0x92, 0x58, 0xac, 0xc5, 0x13, 0x8b, 0x5c, 0x3c, 0x53, 0xbd, 0x0e, 0xb5, 0x49,
+ 0x3d, 0x9c, 0x0a, 0x87, 0x42, 0x75, 0x6c, 0xd6, 0xa0, 0xf7, 0xa0, 0x30, 0x90, 0x7e, 0x90, 0x93,
+ 0x72, 0xeb, 0xe8, 0x1e, 0xd5, 0x42, 0x4c, 0xf5, 0xa3, 0x1c, 0x2c, 0x8f, 0x4e, 0x99, 0x67, 0x4d,
+ 0x89, 0x5e, 0x02, 0xd4, 0x23, 0x86, 0x88, 0x89, 0x04, 0x0f, 0x0c, 0xcb, 0xb1, 0x9c, 0x3e, 0x77,
+ 0x87, 0xa2, 0x55, 0x83, 0x16, 0x2d, 0x68, 0x40, 0xbf, 0x54, 0xe0, 0xf4, 0xa8, 0xc2, 0x68, 0xcc,
+ 0x4c, 0xcc, 0x60, 0x7c, 0x5c, 0xf1, 0x62, 0x54, 0x6b, 0x34, 0xec, 0x85, 0xd0, 0xdb, 0x29, 0x37,
+ 0xbd, 0x55, 0x7d, 0x0b, 0xce, 0x3c, 0xcd, 0x70, 0x2a, 0x19, 0xbc, 0x0e, 0x2b, 0x9f, 0x9f, 0xe6,
+ 0x4e, 0x36, 0xff, 0xd3, 0x3c, 0xe4, 0x59, 0xec, 0x40, 0x3a, 0x94, 0xc4, 0x1a, 0xad, 0x3b, 0xc6,
+ 0x20, 0x48, 0x9d, 0xae, 0xcd, 0x10, 0x85, 0x64, 0xe1, 0x8e, 0x31, 0xc0, 0x1a, 0x0c, 0xc2, 0x67,
+ 0x84, 0xa1, 0xcc, 0xa7, 0x3a, 0x26, 0xba, 0x69, 0xf8, 0x46, 0x70, 0xa0, 0xf9, 0xc6, 0x2c, 0x14,
+ 0x4d, 0x01, 0xb4, 0x6d, 0xf8, 0xc6, 0xcd, 0x13, 0x5a, 0xa9, 0x1b, 0x15, 0x91, 0x0f, 0x55, 0xd3,
+ 0xa2, 0x3e, 0xb1, 0xf6, 0xb8, 0xa9, 0xe0, 0x9a, 0xf2, 0x2c, 0x73, 0x84, 0x6b, 0x3b, 0x86, 0x26,
+ 0x09, 0x2b, 0x66, 0xa2, 0x0e, 0xe9, 0x00, 0x7d, 0x63, 0xd8, 0xc7, 0x82, 0xee, 0xb3, 0xe9, 0x4e,
+ 0x12, 0x47, 0xe8, 0x6e, 0x30, 0x18, 0xc9, 0x53, 0xec, 0x07, 0x05, 0xf5, 0x1a, 0x40, 0xe4, 0x57,
+ 0x74, 0x06, 0x8a, 0xec, 0x2b, 0x51, 0xcf, 0xe8, 0x62, 0xb9, 0x89, 0x8c, 0x2a, 0x10, 0x82, 0x3c,
+ 0xff, 0x86, 0x39, 0xde, 0xc0, 0x9f, 0xd5, 0xff, 0x67, 0x9b, 0xf0, 0xc8, 0x4b, 0xa1, 0x20, 0x94,
+ 0x98, 0x20, 0xd4, 0xf7, 0xa0, 0x92, 0x1c, 0x2d, 0x7b, 0x93, 0xbb, 0x37, 0x78, 0x93, 0x17, 0x98,
+ 0xc4, 0xe8, 0x70, 0x20, 0xe5, 0xc4, 0x1e, 0x59, 0xcd, 0xc0, 0x72, 0x38, 0x67, 0x4e, 0x63, 0x8f,
+ 0xbc, 0xc6, 0x38, 0xe0, 0x29, 0x11, 0xab, 0x31, 0x0e, 0xd4, 0x77, 0xa0, 0x18, 0x0e, 0x2f, 0xbd,
+ 0x0b, 0xe8, 0x0a, 0x14, 0xc3, 0xcb, 0xb0, 0x0c, 0x9b, 0xb2, 0xe8, 0x65, 0x96, 0xc5, 0x32, 0xe7,
+ 0xab, 0x87, 0x50, 0x49, 0x66, 0x34, 0x29, 0x33, 0xe2, 0xee, 0xe8, 0xc6, 0xef, 0xea, 0xcc, 0x11,
+ 0x21, 0xbe, 0x2f, 0xfc, 0xcd, 0x1c, 0x3c, 0xf7, 0xd4, 0x73, 0xf0, 0x63, 0x4c, 0xa4, 0x9f, 0x6d,
+ 0x82, 0xfb, 0x2e, 0x2c, 0x79, 0xc4, 0x1a, 0x18, 0xe4, 0x50, 0x66, 0xe9, 0x22, 0x2b, 0x99, 0x7d,
+ 0x1b, 0x5b, 0x96, 0x70, 0x3c, 0x3b, 0xaf, 0x7f, 0x3b, 0x0f, 0xa7, 0x27, 0x5e, 0x1a, 0x65, 0xbd,
+ 0x91, 0x79, 0x02, 0xcb, 0x26, 0xa6, 0x16, 0xc1, 0xa6, 0xb8, 0x33, 0x08, 0xc6, 0xbf, 0x7b, 0xd4,
+ 0x5b, 0xab, 0xc6, 0xb6, 0x80, 0xe5, 0x75, 0x32, 0x77, 0x58, 0x32, 0xe3, 0x75, 0xea, 0xef, 0x14,
+ 0x28, 0xc7, 0xdf, 0x42, 0x17, 0x61, 0x3d, 0x5c, 0xa5, 0xdc, 0x9e, 0x5c, 0x71, 0x4c, 0x2c, 0xae,
+ 0x53, 0x15, 0x6d, 0x35, 0x68, 0xbc, 0xdb, 0xd3, 0x82, 0x26, 0x74, 0x1e, 0xd6, 0x0c, 0xdb, 0x76,
+ 0x1f, 0x07, 0x03, 0xd0, 0xc5, 0x35, 0x32, 0x1f, 0x46, 0x4e, 0x43, 0xb2, 0x8d, 0xe3, 0x77, 0x78,
+ 0x0b, 0xba, 0x02, 0x35, 0x4c, 0x7d, 0x6b, 0x60, 0xb0, 0xfd, 0xff, 0x48, 0x5a, 0x47, 0xe5, 0x5c,
+ 0x3c, 0x19, 0xb6, 0xc7, 0x73, 0x15, 0xaa, 0x7e, 0xa4, 0x00, 0x1a, 0x1f, 0x56, 0xca, 0xc4, 0xe8,
+ 0x8e, 0x4e, 0x8c, 0xdb, 0xc7, 0xea, 0xcc, 0xf8, 0x64, 0xf9, 0x57, 0x0e, 0xd4, 0xc9, 0xd7, 0x36,
+ 0xe3, 0x0a, 0x54, 0x8e, 0x53, 0x81, 0xff, 0xb5, 0x7d, 0xe8, 0x10, 0x96, 0xbb, 0x0f, 0x0d, 0xc7,
+ 0xc1, 0xf6, 0xa8, 0x48, 0xef, 0x1c, 0xf9, 0x62, 0xab, 0xd1, 0x14, 0xb8, 0xa2, 0x72, 0xa9, 0x1b,
+ 0x2b, 0x51, 0xf5, 0xe7, 0x0a, 0x94, 0xe3, 0xed, 0x59, 0x0e, 0x26, 0xcf, 0xc3, 0x9a, 0x6d, 0x50,
+ 0x5f, 0x0f, 0xdc, 0x1e, 0x1c, 0x45, 0x0a, 0x61, 0x21, 0xd6, 0xd6, 0x11, 0x4d, 0x52, 0x55, 0xe8,
+ 0x32, 0x9c, 0xec, 0x59, 0x84, 0xfa, 0x7a, 0xe8, 0xca, 0xf8, 0xf1, 0x65, 0x4e, 0x5b, 0xe3, 0xad,
+ 0x9a, 0x6c, 0x94, 0x56, 0xf5, 0x6b, 0xb0, 0x9e, 0x7a, 0x7d, 0x9b, 0x75, 0x03, 0x5c, 0x83, 0x93,
+ 0xe9, 0x77, 0x6f, 0xf5, 0x4f, 0x14, 0x28, 0x84, 0x79, 0xe9, 0x4d, 0xb1, 0x1e, 0x48, 0xdd, 0x5c,
+ 0xce, 0xe8, 0xef, 0x30, 0xb3, 0x63, 0x6b, 0x94, 0x26, 0x56, 0x14, 0x13, 0xf2, 0x7c, 0xc5, 0xca,
+ 0x18, 0x97, 0x92, 0xae, 0x9e, 0x1b, 0x77, 0x35, 0x92, 0x7d, 0x13, 0xa7, 0xbc, 0xfc, 0xb9, 0xfe,
+ 0xb3, 0x1c, 0x94, 0xf9, 0xd9, 0x4d, 0xe0, 0x8e, 0xe4, 0x5d, 0xdb, 0x38, 0xfd, 0x5c, 0x1a, 0xfd,
+ 0x0e, 0x14, 0xc5, 0x2d, 0x0a, 0x9b, 0xd8, 0x39, 0x3e, 0x89, 0xcf, 0x65, 0x1c, 0x3c, 0xa7, 0xbf,
+ 0x85, 0x0f, 0xb5, 0x02, 0x95, 0x4f, 0xe8, 0x16, 0xe4, 0xfa, 0xd8, 0x9f, 0xf6, 0xd7, 0x0a, 0x0e,
+ 0x74, 0x03, 0xc7, 0x7e, 0x03, 0x60, 0x28, 0xe8, 0x1e, 0x2c, 0x18, 0x9e, 0x87, 0x1d, 0x33, 0x48,
+ 0xfe, 0xae, 0x4e, 0x83, 0xb7, 0xc9, 0x4d, 0x23, 0x48, 0x89, 0x85, 0xbe, 0x0c, 0xf3, 0x5d, 0x1b,
+ 0x1b, 0x24, 0xc8, 0xf2, 0xae, 0x4c, 0x03, 0xda, 0x64, 0x96, 0x11, 0xa6, 0x40, 0x8a, 0xff, 0x36,
+ 0xf0, 0xc9, 0x1c, 0x2c, 0xc9, 0xcf, 0x22, 0x23, 0x53, 0xf2, 0xbb, 0xa4, 0xff, 0x19, 0xb0, 0x33,
+ 0xe2, 0xb8, 0x57, 0xa6, 0x76, 0x5c, 0x78, 0x9d, 0xcc, 0x3d, 0x77, 0x3f, 0xe9, 0xb9, 0x57, 0x67,
+ 0xf1, 0x5c, 0x88, 0x19, 0xb8, 0x4e, 0x4b, 0xb8, 0xee, 0xea, 0x0c, 0xae, 0x0b, 0x41, 0xa5, 0xef,
+ 0xe2, 0xd7, 0xdd, 0x7f, 0x28, 0x40, 0x21, 0x10, 0x15, 0xea, 0xc0, 0x82, 0xf8, 0x79, 0x4a, 0xa6,
+ 0x3e, 0x2f, 0x4f, 0xa9, 0xca, 0x86, 0xc6, 0xad, 0x59, 0xf7, 0x05, 0x0e, 0xa2, 0xb0, 0x3a, 0x18,
+ 0xda, 0x6c, 0xbd, 0xf3, 0xf4, 0xb1, 0x33, 0xd8, 0xcd, 0x69, 0xe1, 0x6f, 0x4b, 0xa8, 0xf8, 0xa1,
+ 0x6b, 0x75, 0x90, 0xac, 0x44, 0x26, 0x2c, 0xef, 0x19, 0x7d, 0x3d, 0x76, 0xcc, 0x9c, 0x9b, 0xea,
+ 0xcf, 0x8c, 0x90, 0x6f, 0xcb, 0xe8, 0xc7, 0x8f, 0x94, 0xcb, 0x7b, 0xb1, 0x32, 0x1b, 0x9a, 0xe5,
+ 0x63, 0x62, 0xec, 0xd9, 0x38, 0x3e, 0xb4, 0xfc, 0x6c, 0x43, 0x6b, 0x4b, 0xa8, 0x91, 0xa1, 0x59,
+ 0xc9, 0x4a, 0xf4, 0x2d, 0x05, 0x6a, 0xa1, 0x43, 0x1f, 0xe1, 0x43, 0x1a, 0xa7, 0x9e, 0xe7, 0xd4,
+ 0xad, 0x59, 0xbd, 0x7a, 0x0b, 0x1f, 0xd2, 0x38, 0xfd, 0xfa, 0x20, 0xad, 0x41, 0x55, 0x61, 0x41,
+ 0x7c, 0xe6, 0x78, 0x6a, 0x52, 0xe6, 0xa9, 0x89, 0x4a, 0xa0, 0x3a, 0x36, 0x90, 0x2c, 0x0b, 0x5b,
+ 0x1d, 0x96, 0xa2, 0x71, 0xc4, 0x22, 0x72, 0x78, 0x02, 0xde, 0x36, 0xd1, 0x49, 0x58, 0x10, 0x97,
+ 0xed, 0x32, 0x26, 0xcb, 0x92, 0xfa, 0x3d, 0x05, 0xaa, 0x63, 0xc2, 0x78, 0xc6, 0xa4, 0xc1, 0xd0,
+ 0xf3, 0xd1, 0xd0, 0xf7, 0x61, 0x3d, 0xd5, 0x91, 0xcf, 0x7a, 0xf8, 0x1f, 0x42, 0x39, 0x2e, 0xd3,
+ 0x8c, 0x74, 0xd1, 0xdc, 0x88, 0xd1, 0x85, 0x37, 0x26, 0xd3, 0x0c, 0x3c, 0xbc, 0x9b, 0x78, 0x13,
+ 0x56, 0x12, 0x8b, 0x0a, 0x7a, 0x09, 0x50, 0xd7, 0x75, 0x7c, 0xcb, 0x19, 0xf2, 0xcc, 0x4b, 0x5c,
+ 0x36, 0x49, 0xbd, 0x54, 0xe3, 0x2d, 0xfc, 0x2e, 0xa5, 0x7e, 0x1f, 0x2a, 0xc9, 0xe8, 0x3a, 0x25,
+ 0x44, 0xb8, 0x6c, 0xcf, 0xc5, 0x96, 0xed, 0x0d, 0x40, 0xe3, 0xab, 0x53, 0xf8, 0xa6, 0x12, 0x7b,
+ 0x73, 0x1d, 0x56, 0x53, 0xa2, 0x71, 0x7d, 0x15, 0xaa, 0x63, 0x2b, 0x51, 0x7d, 0x4d, 0xa2, 0x8e,
+ 0xc4, 0xd8, 0xfa, 0xaf, 0xf2, 0x50, 0xd8, 0x71, 0xe5, 0x21, 0xd1, 0xd7, 0xa0, 0x40, 0xf1, 0x3e,
+ 0x26, 0x96, 0x2f, 0x26, 0xc9, 0x72, 0xe6, 0xf3, 0x86, 0x00, 0xa2, 0xb1, 0x2b, 0xed, 0xc5, 0xfd,
+ 0x6c, 0x08, 0x37, 0xfb, 0x26, 0x1c, 0xd5, 0xd8, 0xfe, 0x96, 0x52, 0xa3, 0x1f, 0x9c, 0x3e, 0x04,
+ 0x45, 0x7e, 0x67, 0x45, 0x8c, 0x2e, 0xe6, 0x1f, 0xb7, 0xa8, 0x89, 0x42, 0x4a, 0x4e, 0x33, 0x9f,
+ 0x25, 0xa5, 0x5a, 0x18, 0x97, 0xdd, 0xf3, 0x50, 0xb6, 0xdd, 0xbe, 0x6e, 0xbb, 0xf2, 0xde, 0x75,
+ 0x51, 0xbc, 0x62, 0xbb, 0xfd, 0x1d, 0x59, 0xc5, 0x54, 0xe7, 0x3f, 0x24, 0xd8, 0x30, 0x6b, 0x05,
+ 0xde, 0x28, 0x4b, 0xea, 0x57, 0x21, 0xbf, 0x63, 0x51, 0x1f, 0x75, 0x80, 0xbd, 0xae, 0x63, 0xc7,
+ 0x27, 0x16, 0x0e, 0x36, 0x1c, 0xe7, 0xa6, 0x74, 0xaa, 0x06, 0xb6, 0x78, 0xb2, 0x30, 0x55, 0x09,
+ 0x14, 0x02, 0x1f, 0xd7, 0x7b, 0x90, 0x67, 0x6e, 0x46, 0x2b, 0x50, 0xba, 0x7f, 0x67, 0xb7, 0xd3,
+ 0x6a, 0xb6, 0xaf, 0xb7, 0x5b, 0xdb, 0x95, 0x13, 0xa8, 0x08, 0xf3, 0xf7, 0xb4, 0xcd, 0x66, 0xab,
+ 0xa2, 0xb0, 0xc7, 0xed, 0xd6, 0xd6, 0xfd, 0x1b, 0x95, 0x39, 0x54, 0x80, 0x7c, 0xfb, 0xce, 0xf5,
+ 0xbb, 0x95, 0x1c, 0x02, 0x58, 0xb8, 0x73, 0xf7, 0x5e, 0xbb, 0xd9, 0xaa, 0xe4, 0x59, 0xed, 0x83,
+ 0x4d, 0xed, 0x4e, 0x65, 0x9e, 0xbd, 0xda, 0xd2, 0xb4, 0xbb, 0x5a, 0x65, 0x01, 0x95, 0xa1, 0xd0,
+ 0xd4, 0xda, 0xf7, 0xda, 0xcd, 0xcd, 0x9d, 0xca, 0x62, 0xbd, 0x0c, 0xb0, 0xe3, 0xf6, 0x9b, 0xae,
+ 0xe3, 0x13, 0xd7, 0xae, 0xff, 0x39, 0xcf, 0x95, 0x44, 0xfc, 0x07, 0x2e, 0x79, 0x14, 0xfd, 0x73,
+ 0xf6, 0x3f, 0x50, 0x7c, 0xcc, 0x2b, 0xa2, 0x49, 0x5c, 0x10, 0x15, 0x6d, 0x13, 0xed, 0x41, 0xa5,
+ 0x2b, 0xcc, 0xf5, 0xe0, 0xdf, 0x66, 0xa9, 0x82, 0x99, 0xff, 0xbd, 0x59, 0x91, 0x80, 0x2d, 0x89,
+ 0xc7, 0x38, 0x6c, 0xb7, 0xdf, 0xb7, 0x9c, 0x7e, 0xc4, 0x91, 0x3b, 0x22, 0x87, 0x04, 0x0c, 0x39,
+ 0x4c, 0xa8, 0x1a, 0xc4, 0xb7, 0x7a, 0x46, 0xd7, 0x8f, 0x48, 0xf2, 0x47, 0x23, 0xa9, 0x04, 0x88,
+ 0x21, 0x4b, 0x8f, 0xdf, 0x88, 0xed, 0x5b, 0x94, 0x09, 0x38, 0xa4, 0x99, 0x3f, 0x1a, 0x4d, 0x35,
+ 0x84, 0x0c, 0x79, 0xde, 0x85, 0x05, 0xcf, 0x20, 0xc6, 0x80, 0xd6, 0x80, 0x0b, 0x73, 0x8a, 0x95,
+ 0x38, 0xf1, 0xf5, 0x1b, 0x1d, 0x8e, 0x23, 0x7f, 0xf9, 0x12, 0xa0, 0xea, 0x55, 0x28, 0xc5, 0xaa,
+ 0x3f, 0xef, 0x0c, 0xb9, 0x18, 0xdf, 0xc9, 0x7f, 0x91, 0x07, 0xb6, 0x88, 0x44, 0x06, 0xd7, 0x30,
+ 0x2f, 0x56, 0x62, 0x79, 0x71, 0xfd, 0x3c, 0x0b, 0x77, 0xae, 0x97, 0x5d, 0x8e, 0xf5, 0x17, 0x99,
+ 0x82, 0x23, 0x8b, 0xa7, 0xa1, 0x5f, 0xfc, 0x58, 0x81, 0xa5, 0x2d, 0x6c, 0x0c, 0xae, 0x3b, 0x72,
+ 0x02, 0xa0, 0xef, 0x2b, 0xb0, 0x18, 0x3c, 0x67, 0x4d, 0x9a, 0x53, 0x7e, 0x13, 0x56, 0xaf, 0xce,
+ 0x62, 0x2b, 0x82, 0xf9, 0x89, 0x0d, 0xe5, 0xbc, 0x72, 0xf1, 0x03, 0x00, 0xd1, 0x33, 0xbe, 0x97,
+ 0x74, 0xe4, 0x9e, 0xf2, 0xdc, 0x94, 0xfb, 0x52, 0x75, 0x5a, 0x03, 0xc9, 0xfe, 0x03, 0x05, 0x4a,
+ 0x82, 0x5e, 0x2c, 0xe4, 0x07, 0x30, 0x2f, 0x1e, 0x2e, 0x4d, 0x93, 0xd0, 0xc9, 0x11, 0xa9, 0x97,
+ 0xa7, 0x33, 0x92, 0xcb, 0x97, 0xe8, 0xc9, 0x0f, 0xc3, 0x4f, 0xb4, 0x23, 0xe6, 0x2b, 0x3a, 0x80,
+ 0xc5, 0xe0, 0xf1, 0xf2, 0xb4, 0x4b, 0x18, 0x0b, 0xdc, 0xea, 0x85, 0xec, 0x56, 0x41, 0x5c, 0x14,
+ 0x7d, 0xf9, 0xed, 0x1c, 0xd4, 0x44, 0x5f, 0x5a, 0x07, 0x3e, 0x26, 0x8e, 0x61, 0x0b, 0x95, 0x75,
+ 0x5c, 0xa1, 0x9c, 0x52, 0x4c, 0xd7, 0xe8, 0xea, 0xcc, 0x13, 0x4e, 0x7d, 0x75, 0x16, 0xd3, 0xc0,
+ 0x6b, 0xe8, 0xbb, 0x0a, 0x40, 0x34, 0x03, 0x50, 0xf6, 0xfd, 0x6d, 0x62, 0x9a, 0xa9, 0x57, 0x67,
+ 0xb0, 0x0c, 0x7a, 0xb1, 0xb5, 0x09, 0x5f, 0x98, 0x64, 0x1d, 0x37, 0xde, 0x2a, 0x0a, 0x87, 0x6e,
+ 0x7a, 0xd6, 0xdb, 0xcb, 0xb1, 0x26, 0x7d, 0xff, 0xc2, 0xde, 0x02, 0x4f, 0x1e, 0x2e, 0xfd, 0x27,
+ 0x00, 0x00, 0xff, 0xff, 0x08, 0x83, 0xb9, 0x20, 0x84, 0x33, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
index cd8b029..cbaae18 100644
--- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
@@ -6,6 +6,7 @@
import (
context "context"
fmt "fmt"
+ pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
proto "github.com/golang/protobuf/proto"
_struct "github.com/golang/protobuf/ptypes/struct"
grpc "google.golang.org/grpc"
@@ -117,10 +118,15 @@
ResourceLimits *Resources `protobuf:"bytes,4,opt,name=resource_limits,json=resourceLimits,proto3" json:"resource_limits,omitempty"`
// (required) The artifact retrieval token produced by
// ArtifactStagingService.CommitManifestResponse.
- RetrievalToken string `protobuf:"bytes,6,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ RetrievalToken string `protobuf:"bytes,6,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
+ // (optional) The endpoint that the runner is hosting for the SDK to submit
+ // status reports to during pipeline execution. This field will only be
+ // populated if the runner supports SDK status reports. For more details see
+ // https://s.apache.org/beam-fn-api-harness-status
+ StatusEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,7,opt,name=status_endpoint,json=statusEndpoint,proto3" json:"status_endpoint,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *ProvisionInfo) Reset() { *m = ProvisionInfo{} }
@@ -190,6 +196,13 @@
return ""
}
+func (m *ProvisionInfo) GetStatusEndpoint() *pipeline_v1.ApiServiceDescriptor {
+ if m != nil {
+ return m.StatusEndpoint
+ }
+ return nil
+}
+
// Resources specify limits for local resources, such memory and cpu. It
// is used to inform SDK harnesses of their allocated footprint.
type Resources struct {
@@ -388,39 +401,42 @@
func init() { proto.RegisterFile("beam_provision_api.proto", fileDescriptor_92e393e5933c7d6f) }
var fileDescriptor_92e393e5933c7d6f = []byte{
- // 506 bytes of a gzipped FileDescriptorProto
+ // 556 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xdb, 0x6e, 0xd3, 0x40,
- 0x10, 0x86, 0x95, 0x43, 0x4d, 0x33, 0x40, 0x1a, 0xad, 0x80, 0xba, 0xa6, 0x48, 0x28, 0x02, 0xc1,
- 0xd5, 0x96, 0x16, 0x10, 0x77, 0x20, 0xd2, 0x0a, 0x88, 0x04, 0xb4, 0x72, 0xb9, 0x81, 0x1b, 0xcb,
- 0x87, 0x49, 0xba, 0x49, 0xec, 0x59, 0x76, 0xd7, 0xe1, 0xf0, 0x1a, 0xbc, 0x03, 0xe2, 0xc9, 0x78,
- 0x0e, 0xb4, 0xeb, 0xc4, 0xb4, 0x40, 0xa5, 0x94, 0x3b, 0xfb, 0x9f, 0xfd, 0x3f, 0xcd, 0xfc, 0xbb,
- 0x03, 0x7e, 0x82, 0x71, 0x1e, 0x49, 0x45, 0x73, 0xa1, 0x05, 0x15, 0x51, 0x2c, 0x05, 0x97, 0x8a,
- 0x0c, 0xb1, 0xbb, 0xa4, 0xc6, 0x3c, 0x96, 0x71, 0x7a, 0x82, 0xdc, 0x1e, 0xe2, 0x39, 0x65, 0x38,
- 0xe3, 0xa3, 0x22, 0xc2, 0xcf, 0x98, 0x96, 0x46, 0x50, 0xc1, 0xe7, 0xbb, 0xc1, 0xf6, 0x98, 0x68,
- 0x3c, 0xc3, 0x1d, 0x67, 0x4a, 0xca, 0xd1, 0x8e, 0x36, 0xaa, 0x4c, 0x4d, 0x05, 0xe9, 0x6f, 0xc1,
- 0xe6, 0x4b, 0x34, 0x47, 0x4b, 0xfc, 0xb0, 0x18, 0x51, 0x88, 0x1f, 0x4b, 0xd4, 0xa6, 0x9f, 0x81,
- 0xff, 0x77, 0x49, 0x4b, 0x2a, 0x34, 0xb2, 0x57, 0xd0, 0x16, 0xc5, 0x88, 0xfc, 0xc6, 0xed, 0xc6,
- 0xfd, 0xcb, 0x7b, 0x8f, 0xf8, 0x4a, 0xad, 0xf0, 0xb3, 0x2c, 0x47, 0xe8, 0x7f, 0x6f, 0xc2, 0xd5,
- 0x33, 0x3a, 0xbb, 0x0e, 0xde, 0x84, 0x92, 0x48, 0x64, 0x8e, 0xde, 0x09, 0xd7, 0x26, 0x94, 0x0c,
- 0x33, 0xb6, 0x05, 0xeb, 0x56, 0x2e, 0xe2, 0x1c, 0xfd, 0xa6, 0x2b, 0x5c, 0x9a, 0x50, 0xf2, 0x36,
- 0xce, 0x91, 0xdd, 0x84, 0xce, 0x27, 0x52, 0x53, 0x54, 0xd6, 0xb4, 0xe6, 0x6a, 0xeb, 0x95, 0x30,
- 0xcc, 0xd8, 0x00, 0x7a, 0x52, 0x48, 0x9c, 0x89, 0x02, 0x23, 0x92, 0xb6, 0x15, 0xed, 0xb7, 0x5c,
- 0xdb, 0x9b, 0xbc, 0x8a, 0x86, 0x2f, 0xa3, 0xe1, 0xc7, 0x2e, 0x9a, 0x70, 0x63, 0x69, 0x38, 0xac,
- 0xce, 0xb3, 0xf7, 0xb0, 0xa1, 0x50, 0x53, 0xa9, 0x52, 0x8c, 0x66, 0x22, 0x17, 0x46, 0xfb, 0x6d,
- 0x87, 0x78, 0xb0, 0xe2, 0xe4, 0xe1, 0xc2, 0xad, 0xc3, 0xee, 0x12, 0xf4, 0xda, 0x71, 0xd8, 0x3d,
- 0x8b, 0x36, 0x4a, 0xe0, 0x3c, 0x9e, 0x45, 0x86, 0xa6, 0x58, 0xf8, 0x9e, 0x9b, 0xa0, 0x5b, 0xcb,
- 0xef, 0xac, 0xda, 0xff, 0xd9, 0x84, 0x4e, 0x8d, 0x61, 0x87, 0xe0, 0xe5, 0x98, 0x93, 0xfa, 0xb2,
- 0xb8, 0x82, 0x27, 0x17, 0x6d, 0x84, 0xbf, 0x71, 0xf6, 0x70, 0x81, 0x61, 0x2f, 0xa0, 0x95, 0xca,
- 0xd2, 0x25, 0xbb, 0xfa, 0x85, 0xfe, 0xa6, 0xed, 0xcb, 0x32, 0xb4, 0x00, 0x36, 0x86, 0x6b, 0x1a,
- 0x73, 0x11, 0x49, 0x54, 0x5a, 0x68, 0x83, 0x85, 0x89, 0x32, 0xa1, 0xa7, 0x8b, 0xc8, 0x1f, 0x5f,
- 0x18, 0x7c, 0x20, 0xf4, 0x34, 0x64, 0x16, 0x79, 0x54, 0x13, 0xad, 0x16, 0x6c, 0x83, 0x57, 0x8d,
- 0xc0, 0x18, 0xb4, 0xb5, 0xf8, 0x8a, 0x2e, 0x89, 0x76, 0xe8, 0xbe, 0x83, 0x5b, 0xd0, 0xda, 0x97,
- 0x25, 0xbb, 0x01, 0x9e, 0x3e, 0x89, 0x15, 0x6a, 0x57, 0x6c, 0x86, 0x8b, 0xbf, 0x20, 0x80, 0xb6,
- 0x85, 0xfc, 0xcb, 0xba, 0xf7, 0xa3, 0x01, 0xbd, 0xfa, 0x45, 0x1e, 0xa3, 0x9a, 0x8b, 0x14, 0xd9,
- 0xb7, 0x06, 0xf4, 0xfe, 0xdc, 0x06, 0xf6, 0x74, 0xc5, 0x69, 0xce, 0xd9, 0xb0, 0xe0, 0xd9, 0x7f,
- 0xfb, 0xab, 0x35, 0x1c, 0x1c, 0xc0, 0x9d, 0xf3, 0x08, 0xa7, 0x01, 0x83, 0x2b, 0xb5, 0xfd, 0xb9,
- 0x14, 0x1f, 0xba, 0xa7, 0xaa, 0xd1, 0x7c, 0x37, 0xf1, 0xdc, 0xfb, 0x7f, 0xf8, 0x2b, 0x00, 0x00,
- 0xff, 0xff, 0xb1, 0xb0, 0xe9, 0xeb, 0x6b, 0x04, 0x00, 0x00,
+ 0x10, 0x86, 0x95, 0x43, 0xdd, 0x66, 0x80, 0x24, 0x5a, 0x01, 0x75, 0x4d, 0x91, 0x50, 0x04, 0x82,
+ 0xab, 0x2d, 0x2d, 0xa0, 0xde, 0x81, 0x9a, 0x86, 0x43, 0x24, 0xa0, 0x95, 0xcb, 0x0d, 0xdc, 0x18,
+ 0x1f, 0x26, 0xe9, 0x26, 0xf1, 0xee, 0xb2, 0xbb, 0x0e, 0x87, 0x27, 0xe0, 0x9e, 0x97, 0xe0, 0xc9,
+ 0x78, 0x0e, 0xe4, 0xb5, 0x1d, 0x5a, 0x20, 0x52, 0xda, 0x3b, 0x7b, 0x66, 0xff, 0x6f, 0xe7, 0x9f,
+ 0xd9, 0x01, 0x37, 0xc2, 0x30, 0x0d, 0xa4, 0x12, 0x73, 0xa6, 0x99, 0xe0, 0x41, 0x28, 0x19, 0x95,
+ 0x4a, 0x18, 0x41, 0xee, 0x09, 0x35, 0xa6, 0xa1, 0x0c, 0xe3, 0x53, 0xa4, 0xf9, 0x21, 0x9a, 0x8a,
+ 0x04, 0x67, 0x74, 0xc4, 0x03, 0xfc, 0x82, 0x71, 0x66, 0x98, 0xe0, 0x74, 0xbe, 0xeb, 0x75, 0x90,
+ 0x27, 0x52, 0x30, 0x6e, 0x74, 0xa1, 0xf3, 0xb6, 0xc7, 0x42, 0x8c, 0x67, 0xb8, 0x63, 0xff, 0xa2,
+ 0x6c, 0xb4, 0xa3, 0x8d, 0xca, 0x62, 0x53, 0x64, 0x7b, 0x5b, 0xb0, 0xf9, 0x12, 0xcd, 0x71, 0x75,
+ 0xdf, 0x90, 0x8f, 0x84, 0x8f, 0x9f, 0x32, 0xd4, 0xa6, 0x97, 0x80, 0xfb, 0x6f, 0x4a, 0x4b, 0xc1,
+ 0x35, 0x92, 0x57, 0xd0, 0x64, 0x7c, 0x24, 0xdc, 0xda, 0x9d, 0xda, 0x83, 0x2b, 0x7b, 0x8f, 0xe9,
+ 0x4a, 0xb5, 0xd1, 0xf3, 0x2c, 0x4b, 0xe8, 0x7d, 0x6f, 0xc0, 0xb5, 0x73, 0x71, 0x72, 0x03, 0x9c,
+ 0x89, 0x88, 0x02, 0x96, 0x58, 0x7a, 0xcb, 0x5f, 0x9b, 0x88, 0x68, 0x98, 0x90, 0x2d, 0xd8, 0xc8,
+ 0xc3, 0x3c, 0x4c, 0xd1, 0xad, 0xdb, 0xc4, 0xfa, 0x44, 0x44, 0x6f, 0xc3, 0x14, 0xc9, 0x2d, 0x68,
+ 0x7d, 0x16, 0x6a, 0x8a, 0x2a, 0x17, 0xad, 0xd9, 0xdc, 0x46, 0x11, 0x18, 0x26, 0xa4, 0x0f, 0x5d,
+ 0xc9, 0x24, 0xce, 0x18, 0xc7, 0x40, 0xc8, 0xbc, 0x14, 0xed, 0x36, 0x6c, 0xd9, 0x9b, 0xb4, 0x68,
+ 0x0d, 0xad, 0x5a, 0x43, 0x4f, 0x6c, 0x6b, 0xfc, 0x4e, 0x25, 0x38, 0x2a, 0xce, 0x93, 0xf7, 0xd0,
+ 0x51, 0xa8, 0x45, 0xa6, 0x62, 0x0c, 0x66, 0x2c, 0x65, 0x46, 0xbb, 0x4d, 0x8b, 0x78, 0xb8, 0xa2,
+ 0x73, 0xbf, 0x54, 0x6b, 0xbf, 0x5d, 0x81, 0x5e, 0x5b, 0x0e, 0xb9, 0x9f, 0xa3, 0x8d, 0x62, 0x38,
+ 0x0f, 0x67, 0x81, 0x11, 0x53, 0xe4, 0xae, 0x63, 0x1d, 0xb4, 0x17, 0xe1, 0x77, 0x79, 0x94, 0x7c,
+ 0x84, 0x8e, 0x36, 0xa1, 0xc9, 0x74, 0x50, 0x4d, 0xd8, 0x5d, 0xb7, 0x35, 0xec, 0x2f, 0xa9, 0xa1,
+ 0x32, 0x91, 0xdf, 0x7f, 0x20, 0xd9, 0x09, 0xaa, 0x39, 0x8b, 0x71, 0x80, 0x3a, 0x56, 0x4c, 0x1a,
+ 0xa1, 0xfc, 0x76, 0xc1, 0x7b, 0x5e, 0xe2, 0x7a, 0xbf, 0xea, 0xd0, 0x5a, 0x14, 0x4a, 0x8e, 0xc0,
+ 0x49, 0x31, 0x15, 0xea, 0x6b, 0x39, 0xe4, 0xfd, 0x8b, 0x5a, 0xa5, 0x6f, 0xac, 0xdc, 0x2f, 0x31,
+ 0xe4, 0x05, 0x34, 0x62, 0x99, 0xd9, 0xd9, 0xad, 0xfe, 0x64, 0xfe, 0xd0, 0x0e, 0x65, 0xe6, 0xe7,
+ 0x00, 0x32, 0x86, 0xeb, 0x1a, 0x53, 0x16, 0x48, 0x54, 0x9a, 0x69, 0x83, 0xdc, 0x04, 0x09, 0xd3,
+ 0xd3, 0x72, 0xa8, 0x4f, 0x2e, 0x0c, 0x1e, 0x30, 0x3d, 0xf5, 0x49, 0x8e, 0x3c, 0x5e, 0x10, 0xf3,
+ 0x98, 0xb7, 0x0d, 0x4e, 0x61, 0x81, 0x10, 0x68, 0x6a, 0xf6, 0x0d, 0x6d, 0x27, 0x9a, 0xbe, 0xfd,
+ 0xf6, 0x6e, 0x43, 0xe3, 0x50, 0x66, 0xe4, 0x26, 0x38, 0xfa, 0x34, 0x54, 0xa8, 0x6d, 0xb2, 0xee,
+ 0x97, 0x7f, 0x9e, 0x07, 0xcd, 0x1c, 0xf2, 0x3f, 0xe9, 0xde, 0xcf, 0x1a, 0x74, 0x17, 0x6f, 0xbe,
+ 0x9c, 0x0b, 0xf9, 0x51, 0x83, 0xee, 0xdf, 0xfb, 0x46, 0x9e, 0xae, 0xe8, 0x66, 0xc9, 0x0e, 0x7b,
+ 0xcf, 0x2e, 0xad, 0x2f, 0x16, 0xbd, 0x3f, 0x80, 0xbb, 0xcb, 0x08, 0x67, 0x01, 0xfd, 0xab, 0x0b,
+ 0xf9, 0x81, 0x64, 0x1f, 0xda, 0x67, 0xb2, 0xc1, 0x7c, 0x37, 0x72, 0xec, 0x86, 0x3d, 0xfa, 0x1d,
+ 0x00, 0x00, 0xff, 0xff, 0xb2, 0x30, 0x35, 0xd6, 0xde, 0x04, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
index 03dce6c..7959fba 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
@@ -481,6 +481,15 @@
// of the element
// Components: The element coder and the window coder, in that order
StandardCoders_WINDOWED_VALUE StandardCoders_Enum = 8
+ // A windowed value coder with parameterized timestamp, windows and pane info.
+ // Encodes an element with only the value of the windowed value.
+ // Decodes the value and assigns the parameterized timestamp, windows and pane info to the
+ // windowed value.
+ // Components: The element coder and the window coder, in that order
+ // The payload of this coder is an encoded windowed value using the
+ // beam:coder:windowed_value:v1 coder parameterized by a beam:coder:bytes:v1
+ // element coder and the window coder that this param_windowed_value coder uses.
+ StandardCoders_PARAM_WINDOWED_VALUE StandardCoders_Enum = 14
// Encodes an iterable of elements, some of which may be stored elsewhere.
//
// The encoding for a state-backed iterable is the same as that for
@@ -549,6 +558,7 @@
6: "LENGTH_PREFIX",
7: "GLOBAL_WINDOW",
8: "WINDOWED_VALUE",
+ 14: "PARAM_WINDOWED_VALUE",
9: "STATE_BACKED_ITERABLE",
13: "ROW",
}
@@ -566,6 +576,7 @@
"LENGTH_PREFIX": 6,
"GLOBAL_WINDOW": 7,
"WINDOWED_VALUE": 8,
+ "PARAM_WINDOWED_VALUE": 14,
"STATE_BACKED_ITERABLE": 9,
"ROW": 13,
}
@@ -813,7 +824,7 @@
}
func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{36, 0}
+ return fileDescriptor_cf57597c3a9659a9, []int{37, 0}
}
type DisplayData_Type_Enum int32
@@ -1102,10 +1113,14 @@
// (Optional) Static display data for this PTransform application. If
// there is none, or it is not relevant (such as use by the Fn API)
// then it may be omitted.
- DisplayData *DisplayData `protobuf:"bytes,6,opt,name=display_data,json=displayData,proto3" json:"display_data,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ DisplayData *DisplayData `protobuf:"bytes,6,opt,name=display_data,json=displayData,proto3" json:"display_data,omitempty"`
+ // (Optional) Environment where the current PTransform should be executed in.
+ // Runner that executes the pipeline may choose to override this if needed. If
+ // not specified, environment will be decided by the runner.
+ EnvironmentId string `protobuf:"bytes,7,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *PTransform) Reset() { *m = PTransform{} }
@@ -1175,6 +1190,13 @@
return nil
}
+func (m *PTransform) GetEnvironmentId() string {
+ if m != nil {
+ return m.EnvironmentId
+ }
+ return ""
+}
+
type StandardPTransforms struct {
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@@ -1325,8 +1347,8 @@
// The payload for the primitive ParDo transform.
type ParDoPayload struct {
- // (Required) The SdkFunctionSpec of the DoFn.
- DoFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=do_fn,json=doFn,proto3" json:"do_fn,omitempty"`
+ // (Required) The FunctionSpec of the DoFn.
+ DoFn *FunctionSpec `protobuf:"bytes,1,opt,name=do_fn,json=doFn,proto3" json:"do_fn,omitempty"`
// (Required) Additional pieces of context the DoFn may require that
// are not otherwise represented in the payload.
// (may force runners to execute the ParDo differently)
@@ -1374,7 +1396,7 @@
var xxx_messageInfo_ParDoPayload proto.InternalMessageInfo
-func (m *ParDoPayload) GetDoFn() *SdkFunctionSpec {
+func (m *ParDoPayload) GetDoFn() *FunctionSpec {
if m != nil {
return m.DoFn
}
@@ -1718,11 +1740,11 @@
}
type CombiningStateSpec struct {
- AccumulatorCoderId string `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
- CombineFn *SdkFunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ AccumulatorCoderId string `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
+ CombineFn *FunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *CombiningStateSpec) Reset() { *m = CombiningStateSpec{} }
@@ -1757,7 +1779,7 @@
return ""
}
-func (m *CombiningStateSpec) GetCombineFn() *SdkFunctionSpec {
+func (m *CombiningStateSpec) GetCombineFn() *FunctionSpec {
if m != nil {
return m.CombineFn
}
@@ -1930,8 +1952,8 @@
// The payload for the primitive Read transform.
type ReadPayload struct {
- // (Required) The SdkFunctionSpec of the source for this Read.
- Source *SdkFunctionSpec `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"`
+ // (Required) The FunctionSpec of the source for this Read.
+ Source *FunctionSpec `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"`
// (Required) Whether the source is bounded or unbounded
IsBounded IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@@ -1964,7 +1986,7 @@
var xxx_messageInfo_ReadPayload proto.InternalMessageInfo
-func (m *ReadPayload) GetSource() *SdkFunctionSpec {
+func (m *ReadPayload) GetSource() *FunctionSpec {
if m != nil {
return m.Source
}
@@ -1980,11 +2002,11 @@
// The payload for the WindowInto transform.
type WindowIntoPayload struct {
- // (Required) The SdkFunctionSpec of the WindowFn.
- WindowFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ // (Required) The FunctionSpec of the WindowFn.
+ WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *WindowIntoPayload) Reset() { *m = WindowIntoPayload{} }
@@ -2012,7 +2034,7 @@
var xxx_messageInfo_WindowIntoPayload proto.InternalMessageInfo
-func (m *WindowIntoPayload) GetWindowFn() *SdkFunctionSpec {
+func (m *WindowIntoPayload) GetWindowFn() *FunctionSpec {
if m != nil {
return m.WindowFn
}
@@ -2021,8 +2043,8 @@
// The payload for the special-but-not-primitive Combine transform.
type CombinePayload struct {
- // (Required) The SdkFunctionSpec of the CombineFn.
- CombineFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
+ // (Required) The FunctionSpec of the CombineFn.
+ CombineFn *FunctionSpec `protobuf:"bytes,1,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
// (Required) A reference to the Coder to use for accumulators of the CombineFn
AccumulatorCoderId string `protobuf:"bytes,2,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@@ -2055,7 +2077,7 @@
var xxx_messageInfo_CombinePayload proto.InternalMessageInfo
-func (m *CombinePayload) GetCombineFn() *SdkFunctionSpec {
+func (m *CombinePayload) GetCombineFn() *FunctionSpec {
if m != nil {
return m.CombineFn
}
@@ -2459,10 +2481,10 @@
// The payload for the special-but-not-primitive WriteFiles transform.
type WriteFilesPayload struct {
- // (Required) The SdkFunctionSpec of the FileBasedSink.
- Sink *SdkFunctionSpec `protobuf:"bytes,1,opt,name=sink,proto3" json:"sink,omitempty"`
+ // (Required) The FunctionSpec of the FileBasedSink.
+ Sink *FunctionSpec `protobuf:"bytes,1,opt,name=sink,proto3" json:"sink,omitempty"`
// (Required) The format function.
- FormatFunction *SdkFunctionSpec `protobuf:"bytes,2,opt,name=format_function,json=formatFunction,proto3" json:"format_function,omitempty"`
+ FormatFunction *FunctionSpec `protobuf:"bytes,2,opt,name=format_function,json=formatFunction,proto3" json:"format_function,omitempty"`
WindowedWrites bool `protobuf:"varint,3,opt,name=windowed_writes,json=windowedWrites,proto3" json:"windowed_writes,omitempty"`
RunnerDeterminedSharding bool `protobuf:"varint,4,opt,name=runner_determined_sharding,json=runnerDeterminedSharding,proto3" json:"runner_determined_sharding,omitempty"`
SideInputs map[string]*SideInput `protobuf:"bytes,5,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
@@ -2496,14 +2518,14 @@
var xxx_messageInfo_WriteFilesPayload proto.InternalMessageInfo
-func (m *WriteFilesPayload) GetSink() *SdkFunctionSpec {
+func (m *WriteFilesPayload) GetSink() *FunctionSpec {
if m != nil {
return m.Sink
}
return nil
}
-func (m *WriteFilesPayload) GetFormatFunction() *SdkFunctionSpec {
+func (m *WriteFilesPayload) GetFormatFunction() *FunctionSpec {
if m != nil {
return m.FormatFunction
}
@@ -2542,7 +2564,7 @@
Spec *FunctionSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"`
// (Optional) If this coder is parametric, such as ListCoder(VarIntCoder),
// this is a list of the components. In order for encodings to be identical,
- // the SdkFunctionSpec and all components must be identical, recursively.
+ // the FunctionSpec and all components must be identical, recursively.
ComponentCoderIds []string `protobuf:"bytes,2,rep,name=component_coder_ids,json=componentCoderIds,proto3" json:"component_coder_ids,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@@ -2624,10 +2646,10 @@
//
// TODO: consider inlining field on PCollection
type WindowingStrategy struct {
- // (Required) The SdkFunctionSpec of the UDF that assigns windows,
+ // (Required) The FunctionSpec of the UDF that assigns windows,
// merges windows, and shifts timestamps before they are
// combined according to the OutputTime.
- WindowFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"`
+ WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"`
// (Required) Whether or not the window fn is merging.
//
// This knowledge is required for many optimizations.
@@ -2660,7 +2682,11 @@
// (Required) Whether or not the window fn assigns inputs to exactly one window
//
// This knowledge is required for some optimizations
- AssignsToOneWindow bool `protobuf:"varint,10,opt,name=assigns_to_one_window,json=assignsToOneWindow,proto3" json:"assigns_to_one_window,omitempty"`
+ AssignsToOneWindow bool `protobuf:"varint,10,opt,name=assigns_to_one_window,json=assignsToOneWindow,proto3" json:"assigns_to_one_window,omitempty"`
+ // (Optional) Environment where the current window_fn should be applied in.
+ // Runner that executes the pipeline may choose to override this if needed.
+ // If not specified, environment will be decided by the runner.
+ EnvironmentId string `protobuf:"bytes,11,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@@ -2691,7 +2717,7 @@
var xxx_messageInfo_WindowingStrategy proto.InternalMessageInfo
-func (m *WindowingStrategy) GetWindowFn() *SdkFunctionSpec {
+func (m *WindowingStrategy) GetWindowFn() *FunctionSpec {
if m != nil {
return m.WindowFn
}
@@ -2761,6 +2787,13 @@
return false
}
+func (m *WindowingStrategy) GetEnvironmentId() string {
+ if m != nil {
+ return m.EnvironmentId
+ }
+ return ""
+}
+
// Whether or not a PCollection's WindowFn is non-merging, merging, or
// merging-but-already-merged, in which case a subsequent GroupByKey is almost
// always going to do something the user does not want
@@ -3867,22 +3900,22 @@
// performance possibilities, is "beam:sideinput:multimap" (or some such
// URN)
AccessPattern *FunctionSpec `protobuf:"bytes,1,opt,name=access_pattern,json=accessPattern,proto3" json:"access_pattern,omitempty"`
- // (Required) The SdkFunctionSpec of the UDF that adapts a particular
+ // (Required) The FunctionSpec of the UDF that adapts a particular
// access_pattern to a user-facing view type.
//
// For example, View.asSingleton() may include a `view_fn` that adapts a
// specially-designed multimap to a single value per window.
- ViewFn *SdkFunctionSpec `protobuf:"bytes,2,opt,name=view_fn,json=viewFn,proto3" json:"view_fn,omitempty"`
- // (Required) The SdkFunctionSpec of the UDF that maps a main input window
+ ViewFn *FunctionSpec `protobuf:"bytes,2,opt,name=view_fn,json=viewFn,proto3" json:"view_fn,omitempty"`
+ // (Required) The FunctionSpec of the UDF that maps a main input window
// to a side input window.
//
// For example, when the main input is in fixed windows of one hour, this
// can specify that the side input should be accessed according to the day
// in which that hour falls.
- WindowMappingFn *SdkFunctionSpec `protobuf:"bytes,3,opt,name=window_mapping_fn,json=windowMappingFn,proto3" json:"window_mapping_fn,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
+ WindowMappingFn *FunctionSpec `protobuf:"bytes,3,opt,name=window_mapping_fn,json=windowMappingFn,proto3" json:"window_mapping_fn,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
}
func (m *SideInput) Reset() { *m = SideInput{} }
@@ -3917,20 +3950,76 @@
return nil
}
-func (m *SideInput) GetViewFn() *SdkFunctionSpec {
+func (m *SideInput) GetViewFn() *FunctionSpec {
if m != nil {
return m.ViewFn
}
return nil
}
-func (m *SideInput) GetWindowMappingFn() *SdkFunctionSpec {
+func (m *SideInput) GetWindowMappingFn() *FunctionSpec {
if m != nil {
return m.WindowMappingFn
}
return nil
}
+// Settings that decide the coder type of wire coder.
+type WireCoderSetting struct {
+ // (Required) The URN of the wire coder.
+ // Note that only windowed value coder or parameterized windowed value coder are supported.
+ Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
+ // (Optional) The data specifying any parameters to the URN. If
+ // the URN is beam:coder:windowed_value:v1, this may be omitted. If the URN is
+ // beam:coder:param_windowed_value:v1, the payload is an encoded windowed
+ // value using the beam:coder:windowed_value:v1 coder parameterized by
+ // a beam:coder:bytes:v1 element coder and the window coder that this
+ // param_windowed_value coder uses.
+ Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
+}
+
+func (m *WireCoderSetting) Reset() { *m = WireCoderSetting{} }
+func (m *WireCoderSetting) String() string { return proto.CompactTextString(m) }
+func (*WireCoderSetting) ProtoMessage() {}
+func (*WireCoderSetting) Descriptor() ([]byte, []int) {
+ return fileDescriptor_cf57597c3a9659a9, []int{35}
+}
+
+func (m *WireCoderSetting) XXX_Unmarshal(b []byte) error {
+ return xxx_messageInfo_WireCoderSetting.Unmarshal(m, b)
+}
+func (m *WireCoderSetting) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ return xxx_messageInfo_WireCoderSetting.Marshal(b, m, deterministic)
+}
+func (m *WireCoderSetting) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_WireCoderSetting.Merge(m, src)
+}
+func (m *WireCoderSetting) XXX_Size() int {
+ return xxx_messageInfo_WireCoderSetting.Size(m)
+}
+func (m *WireCoderSetting) XXX_DiscardUnknown() {
+ xxx_messageInfo_WireCoderSetting.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_WireCoderSetting proto.InternalMessageInfo
+
+func (m *WireCoderSetting) GetUrn() string {
+ if m != nil {
+ return m.Urn
+ }
+ return ""
+}
+
+func (m *WireCoderSetting) GetPayload() []byte {
+ if m != nil {
+ return m.Payload
+ }
+ return nil
+}
+
// An environment for executing UDFs. By default, an SDK container URL, but
// can also be a process forked by a command, or an externally managed process.
type Environment struct {
@@ -3948,7 +4037,7 @@
func (m *Environment) String() string { return proto.CompactTextString(m) }
func (*Environment) ProtoMessage() {}
func (*Environment) Descriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{35}
+ return fileDescriptor_cf57597c3a9659a9, []int{36}
}
func (m *Environment) XXX_Unmarshal(b []byte) error {
@@ -3993,7 +4082,7 @@
func (m *StandardEnvironments) String() string { return proto.CompactTextString(m) }
func (*StandardEnvironments) ProtoMessage() {}
func (*StandardEnvironments) Descriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{36}
+ return fileDescriptor_cf57597c3a9659a9, []int{37}
}
func (m *StandardEnvironments) XXX_Unmarshal(b []byte) error {
@@ -4026,7 +4115,7 @@
func (m *DockerPayload) String() string { return proto.CompactTextString(m) }
func (*DockerPayload) ProtoMessage() {}
func (*DockerPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{37}
+ return fileDescriptor_cf57597c3a9659a9, []int{38}
}
func (m *DockerPayload) XXX_Unmarshal(b []byte) error {
@@ -4068,7 +4157,7 @@
func (m *ProcessPayload) String() string { return proto.CompactTextString(m) }
func (*ProcessPayload) ProtoMessage() {}
func (*ProcessPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{38}
+ return fileDescriptor_cf57597c3a9659a9, []int{39}
}
func (m *ProcessPayload) XXX_Unmarshal(b []byte) error {
@@ -4129,7 +4218,7 @@
func (m *ExternalPayload) String() string { return proto.CompactTextString(m) }
func (*ExternalPayload) ProtoMessage() {}
func (*ExternalPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{39}
+ return fileDescriptor_cf57597c3a9659a9, []int{40}
}
func (m *ExternalPayload) XXX_Unmarshal(b []byte) error {
@@ -4164,58 +4253,6 @@
return nil
}
-// A specification of a user defined function.
-//
-type SdkFunctionSpec struct {
- // (Required) A full specification of this function.
- Spec *FunctionSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"`
- // (Required) Reference to an execution environment capable of
- // invoking this function.
- EnvironmentId string `protobuf:"bytes,2,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
-}
-
-func (m *SdkFunctionSpec) Reset() { *m = SdkFunctionSpec{} }
-func (m *SdkFunctionSpec) String() string { return proto.CompactTextString(m) }
-func (*SdkFunctionSpec) ProtoMessage() {}
-func (*SdkFunctionSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_cf57597c3a9659a9, []int{40}
-}
-
-func (m *SdkFunctionSpec) XXX_Unmarshal(b []byte) error {
- return xxx_messageInfo_SdkFunctionSpec.Unmarshal(m, b)
-}
-func (m *SdkFunctionSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
- return xxx_messageInfo_SdkFunctionSpec.Marshal(b, m, deterministic)
-}
-func (m *SdkFunctionSpec) XXX_Merge(src proto.Message) {
- xxx_messageInfo_SdkFunctionSpec.Merge(m, src)
-}
-func (m *SdkFunctionSpec) XXX_Size() int {
- return xxx_messageInfo_SdkFunctionSpec.Size(m)
-}
-func (m *SdkFunctionSpec) XXX_DiscardUnknown() {
- xxx_messageInfo_SdkFunctionSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SdkFunctionSpec proto.InternalMessageInfo
-
-func (m *SdkFunctionSpec) GetSpec() *FunctionSpec {
- if m != nil {
- return m.Spec
- }
- return nil
-}
-
-func (m *SdkFunctionSpec) GetEnvironmentId() string {
- if m != nil {
- return m.EnvironmentId
- }
- return ""
-}
-
// A URN along with a parameter object whose schema is determined by the
// URN.
//
@@ -4528,7 +4565,7 @@
// Types that are valid to be assigned to Root:
// *MessageWithComponents_Coder
// *MessageWithComponents_CombinePayload
- // *MessageWithComponents_SdkFunctionSpec
+ // *MessageWithComponents_FunctionSpec
// *MessageWithComponents_ParDoPayload
// *MessageWithComponents_Ptransform
// *MessageWithComponents_Pcollection
@@ -4536,7 +4573,6 @@
// *MessageWithComponents_SideInput
// *MessageWithComponents_WindowIntoPayload
// *MessageWithComponents_WindowingStrategy
- // *MessageWithComponents_FunctionSpec
Root isMessageWithComponents_Root `protobuf_oneof:"root"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@@ -4587,8 +4623,8 @@
CombinePayload *CombinePayload `protobuf:"bytes,3,opt,name=combine_payload,json=combinePayload,proto3,oneof"`
}
-type MessageWithComponents_SdkFunctionSpec struct {
- SdkFunctionSpec *SdkFunctionSpec `protobuf:"bytes,4,opt,name=sdk_function_spec,json=sdkFunctionSpec,proto3,oneof"`
+type MessageWithComponents_FunctionSpec struct {
+ FunctionSpec *FunctionSpec `protobuf:"bytes,4,opt,name=function_spec,json=functionSpec,proto3,oneof"`
}
type MessageWithComponents_ParDoPayload struct {
@@ -4619,15 +4655,11 @@
WindowingStrategy *WindowingStrategy `protobuf:"bytes,13,opt,name=windowing_strategy,json=windowingStrategy,proto3,oneof"`
}
-type MessageWithComponents_FunctionSpec struct {
- FunctionSpec *FunctionSpec `protobuf:"bytes,14,opt,name=function_spec,json=functionSpec,proto3,oneof"`
-}
-
func (*MessageWithComponents_Coder) isMessageWithComponents_Root() {}
func (*MessageWithComponents_CombinePayload) isMessageWithComponents_Root() {}
-func (*MessageWithComponents_SdkFunctionSpec) isMessageWithComponents_Root() {}
+func (*MessageWithComponents_FunctionSpec) isMessageWithComponents_Root() {}
func (*MessageWithComponents_ParDoPayload) isMessageWithComponents_Root() {}
@@ -4643,8 +4675,6 @@
func (*MessageWithComponents_WindowingStrategy) isMessageWithComponents_Root() {}
-func (*MessageWithComponents_FunctionSpec) isMessageWithComponents_Root() {}
-
func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root {
if m != nil {
return m.Root
@@ -4666,9 +4696,9 @@
return nil
}
-func (m *MessageWithComponents) GetSdkFunctionSpec() *SdkFunctionSpec {
- if x, ok := m.GetRoot().(*MessageWithComponents_SdkFunctionSpec); ok {
- return x.SdkFunctionSpec
+func (m *MessageWithComponents) GetFunctionSpec() *FunctionSpec {
+ if x, ok := m.GetRoot().(*MessageWithComponents_FunctionSpec); ok {
+ return x.FunctionSpec
}
return nil
}
@@ -4722,19 +4752,12 @@
return nil
}
-func (m *MessageWithComponents) GetFunctionSpec() *FunctionSpec {
- if x, ok := m.GetRoot().(*MessageWithComponents_FunctionSpec); ok {
- return x.FunctionSpec
- }
- return nil
-}
-
// XXX_OneofWrappers is for the internal use of the proto package.
func (*MessageWithComponents) XXX_OneofWrappers() []interface{} {
return []interface{}{
(*MessageWithComponents_Coder)(nil),
(*MessageWithComponents_CombinePayload)(nil),
- (*MessageWithComponents_SdkFunctionSpec)(nil),
+ (*MessageWithComponents_FunctionSpec)(nil),
(*MessageWithComponents_ParDoPayload)(nil),
(*MessageWithComponents_Ptransform)(nil),
(*MessageWithComponents_Pcollection)(nil),
@@ -4742,7 +4765,6 @@
(*MessageWithComponents_SideInput)(nil),
(*MessageWithComponents_WindowIntoPayload)(nil),
(*MessageWithComponents_WindowingStrategy)(nil),
- (*MessageWithComponents_FunctionSpec)(nil),
}
}
@@ -4754,6 +4776,8 @@
// We use an environment rather than environment id
// because ExecutableStages use environments directly. This may change in the future.
Environment *Environment `protobuf:"bytes,1,opt,name=environment,proto3" json:"environment,omitempty"`
+ // set the wire coder of this executable stage
+ WireCoderSetting *WireCoderSetting `protobuf:"bytes,9,opt,name=wire_coder_setting,json=wireCoderSetting,proto3" json:"wire_coder_setting,omitempty"`
// (Required) Input PCollection id. This must be present as a value in the inputs of any
// PTransform the ExecutableStagePayload is the payload of.
Input string `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"`
@@ -4812,6 +4836,13 @@
return nil
}
+func (m *ExecutableStagePayload) GetWireCoderSetting() *WireCoderSetting {
+ if m != nil {
+ return m.WireCoderSetting
+ }
+ return nil
+}
+
func (m *ExecutableStagePayload) GetInput() string {
if m != nil {
return m.Input
@@ -5117,6 +5148,7 @@
proto.RegisterType((*TimestampTransform_Delay)(nil), "org.apache.beam.model.pipeline.v1.TimestampTransform.Delay")
proto.RegisterType((*TimestampTransform_AlignTo)(nil), "org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo")
proto.RegisterType((*SideInput)(nil), "org.apache.beam.model.pipeline.v1.SideInput")
+ proto.RegisterType((*WireCoderSetting)(nil), "org.apache.beam.model.pipeline.v1.WireCoderSetting")
proto.RegisterType((*Environment)(nil), "org.apache.beam.model.pipeline.v1.Environment")
proto.RegisterType((*StandardEnvironments)(nil), "org.apache.beam.model.pipeline.v1.StandardEnvironments")
proto.RegisterType((*DockerPayload)(nil), "org.apache.beam.model.pipeline.v1.DockerPayload")
@@ -5124,7 +5156,6 @@
proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry")
proto.RegisterType((*ExternalPayload)(nil), "org.apache.beam.model.pipeline.v1.ExternalPayload")
proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.ExternalPayload.ParamsEntry")
- proto.RegisterType((*SdkFunctionSpec)(nil), "org.apache.beam.model.pipeline.v1.SdkFunctionSpec")
proto.RegisterType((*FunctionSpec)(nil), "org.apache.beam.model.pipeline.v1.FunctionSpec")
proto.RegisterType((*DisplayData)(nil), "org.apache.beam.model.pipeline.v1.DisplayData")
proto.RegisterType((*DisplayData_Identifier)(nil), "org.apache.beam.model.pipeline.v1.DisplayData.Identifier")
@@ -5142,333 +5173,335 @@
func init() { proto.RegisterFile("beam_runner_api.proto", fileDescriptor_cf57597c3a9659a9) }
var fileDescriptor_cf57597c3a9659a9 = []byte{
- // 5205 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xdb, 0x6f, 0x23, 0xc9,
- 0x75, 0x37, 0x2f, 0xe2, 0xed, 0x90, 0xa2, 0x5a, 0x25, 0xcd, 0xac, 0xb6, 0xbd, 0xde, 0x99, 0xed,
- 0x5d, 0xef, 0x8e, 0xf7, 0x5b, 0x73, 0x47, 0x9a, 0x99, 0x9d, 0x19, 0xd9, 0x9e, 0x35, 0x29, 0x36,
- 0x47, 0x3d, 0xc3, 0x9b, 0x9b, 0x94, 0x34, 0xb3, 0xb6, 0xb7, 0xdd, 0x62, 0x17, 0xa5, 0x86, 0x9a,
- 0xdd, 0x74, 0x77, 0x53, 0xb3, 0x34, 0x6c, 0x7c, 0x40, 0x10, 0x18, 0x41, 0x02, 0xe4, 0xf2, 0x90,
- 0x00, 0x7e, 0x08, 0x02, 0xd8, 0x40, 0x80, 0x24, 0x0f, 0x09, 0xe2, 0x24, 0x40, 0x5e, 0x9d, 0xe4,
- 0x31, 0x0f, 0x01, 0xf2, 0x94, 0x3f, 0x23, 0x81, 0x1f, 0x92, 0xa7, 0xa0, 0x2e, 0xdd, 0x6c, 0x52,
- 0xd2, 0x2c, 0x29, 0x09, 0x79, 0x63, 0x9f, 0xae, 0xf3, 0x3b, 0x55, 0xa7, 0xaa, 0x4e, 0x9d, 0x73,
- 0xea, 0x34, 0xe1, 0xc6, 0x21, 0xd6, 0x07, 0x9a, 0x3b, 0xb2, 0x6d, 0xec, 0x6a, 0xfa, 0xd0, 0x2c,
- 0x0d, 0x5d, 0xc7, 0x77, 0xd0, 0x3b, 0x8e, 0x7b, 0x54, 0xd2, 0x87, 0x7a, 0xef, 0x18, 0x97, 0x48,
- 0x8b, 0xd2, 0xc0, 0x31, 0xb0, 0x55, 0x1a, 0x9a, 0x43, 0x6c, 0x99, 0x36, 0x2e, 0x9d, 0x6e, 0x8a,
- 0x2b, 0xd8, 0x36, 0x86, 0x8e, 0x69, 0xfb, 0x1e, 0xe3, 0x11, 0xdf, 0x3c, 0x72, 0x9c, 0x23, 0x0b,
- 0x7f, 0x4c, 0x9f, 0x0e, 0x47, 0xfd, 0x8f, 0x75, 0x7b, 0xcc, 0x5f, 0xdd, 0x9e, 0x7d, 0x65, 0x60,
- 0xaf, 0xe7, 0x9a, 0x43, 0xdf, 0x71, 0x79, 0x8b, 0x5b, 0xb3, 0x2d, 0x7c, 0x73, 0x80, 0x3d, 0x5f,
- 0x1f, 0x0c, 0x59, 0x03, 0xe9, 0xd7, 0x71, 0x58, 0xae, 0x60, 0x7d, 0xb0, 0xe3, 0xd8, 0x9e, 0xaf,
- 0xdb, 0xbe, 0x27, 0xfd, 0x4d, 0x1c, 0x72, 0xe1, 0x13, 0xda, 0x84, 0xf5, 0x86, 0xd2, 0xd4, 0xba,
- 0x4a, 0x43, 0xee, 0x74, 0xcb, 0x8d, 0xb6, 0xd6, 0x50, 0xea, 0x75, 0xa5, 0x23, 0xc4, 0xc4, 0x37,
- 0xfe, 0xf2, 0xef, 0xfe, 0xe7, 0xd7, 0xa9, 0xd5, 0x6f, 0x3c, 0xde, 0xda, 0xba, 0x77, 0xef, 0xe1,
- 0xd6, 0xdd, 0x7b, 0x9f, 0x3c, 0x7a, 0x70, 0xff, 0xe1, 0xc3, 0x07, 0xe8, 0x2e, 0xac, 0x37, 0xca,
- 0x2f, 0xce, 0xb2, 0xc4, 0xc5, 0x9b, 0x94, 0x45, 0x38, 0xc3, 0xf1, 0x04, 0xa4, 0xa7, 0xf5, 0x56,
- 0xa5, 0x5c, 0xd7, 0x0e, 0x94, 0x66, 0xb5, 0x75, 0xa0, 0x9d, 0xcb, 0x9f, 0x98, 0xe6, 0xdf, 0x7c,
- 0xfc, 0xe0, 0xee, 0x7d, 0xca, 0x2f, 0xfd, 0x43, 0x16, 0x60, 0xc7, 0x19, 0x0c, 0x1d, 0x1b, 0x93,
- 0x3e, 0xff, 0x00, 0xc0, 0x77, 0x75, 0xdb, 0xeb, 0x3b, 0xee, 0xc0, 0xdb, 0x88, 0xdf, 0x4e, 0xde,
- 0xc9, 0x6f, 0x7d, 0xbb, 0xf4, 0xa5, 0xaa, 0x2f, 0x4d, 0x20, 0x4a, 0xdd, 0x90, 0x5f, 0xb6, 0x7d,
- 0x77, 0xac, 0x46, 0x00, 0x51, 0x0f, 0x0a, 0xc3, 0x9e, 0x63, 0x59, 0xb8, 0xe7, 0x9b, 0x8e, 0xed,
- 0x6d, 0x24, 0xa8, 0x80, 0x4f, 0x17, 0x13, 0xd0, 0x8e, 0x20, 0x30, 0x11, 0x53, 0xa0, 0x68, 0x0c,
- 0xeb, 0xaf, 0x4c, 0xdb, 0x70, 0x5e, 0x99, 0xf6, 0x91, 0xe6, 0xf9, 0xae, 0xee, 0xe3, 0x23, 0x13,
- 0x7b, 0x1b, 0x49, 0x2a, 0xac, 0xb6, 0x98, 0xb0, 0x83, 0x00, 0xa9, 0x13, 0x02, 0x31, 0x99, 0x6b,
- 0xaf, 0xce, 0xbe, 0x41, 0xdf, 0x85, 0x74, 0xcf, 0x31, 0xb0, 0xeb, 0x6d, 0x2c, 0x51, 0x61, 0x8f,
- 0x17, 0x13, 0xb6, 0x43, 0x79, 0x19, 0x3e, 0x07, 0x22, 0x2a, 0xc3, 0xf6, 0xa9, 0xe9, 0x3a, 0xf6,
- 0x80, 0xb4, 0xd9, 0x48, 0x5d, 0x46, 0x65, 0x72, 0x04, 0x81, 0xab, 0x2c, 0x0a, 0x2a, 0x5a, 0xb0,
- 0x32, 0x33, 0x6d, 0x48, 0x80, 0xe4, 0x09, 0x1e, 0x6f, 0xc4, 0x6f, 0xc7, 0xef, 0xe4, 0x54, 0xf2,
- 0x13, 0xed, 0x40, 0xea, 0x54, 0xb7, 0x46, 0x78, 0x23, 0x71, 0x3b, 0x7e, 0x27, 0xbf, 0xf5, 0x8d,
- 0x39, 0xba, 0xd0, 0x0e, 0x51, 0x55, 0xc6, 0xbb, 0x9d, 0x78, 0x14, 0x17, 0x1d, 0x58, 0x3d, 0x33,
- 0x87, 0xe7, 0xc8, 0xab, 0x4e, 0xcb, 0x2b, 0xcd, 0x23, 0x6f, 0x27, 0x84, 0x8d, 0x0a, 0xfc, 0x09,
- 0x6c, 0x5c, 0x34, 0x8f, 0xe7, 0xc8, 0x7d, 0x36, 0x2d, 0xf7, 0xfe, 0x1c, 0x72, 0x67, 0xd1, 0xc7,
- 0x51, 0xe9, 0x3d, 0xc8, 0x47, 0x26, 0xf6, 0x1c, 0x81, 0x4f, 0xa6, 0x05, 0xde, 0x99, 0x6b, 0x6e,
- 0x0d, 0xec, 0xce, 0xe8, 0xf4, 0xcc, 0x24, 0x5f, 0x8f, 0x4e, 0x23, 0xb0, 0x11, 0x81, 0xd2, 0x7f,
- 0xc4, 0x21, 0xdb, 0xe6, 0xcd, 0x50, 0x03, 0xa0, 0x17, 0xae, 0x36, 0x2a, 0x6f, 0xbe, 0xf5, 0x31,
- 0x59, 0xa2, 0x6a, 0x04, 0x00, 0x7d, 0x04, 0xc8, 0x75, 0x1c, 0x5f, 0x0b, 0x2d, 0x87, 0x66, 0x1a,
- 0xcc, 0x58, 0xe4, 0x54, 0x81, 0xbc, 0x09, 0x97, 0x95, 0x62, 0x90, 0x4d, 0x57, 0x30, 0x4c, 0x6f,
- 0x68, 0xe9, 0x63, 0xcd, 0xd0, 0x7d, 0x7d, 0x23, 0x39, 0xf7, 0xd0, 0xaa, 0x8c, 0xad, 0xaa, 0xfb,
- 0xba, 0x9a, 0x37, 0x26, 0x0f, 0xd2, 0xef, 0x2d, 0x01, 0x4c, 0xd6, 0x2e, 0xba, 0x05, 0xf9, 0x91,
- 0x6d, 0xfe, 0x68, 0x84, 0x35, 0x5b, 0x1f, 0xe0, 0x8d, 0x14, 0xd5, 0x27, 0x30, 0x52, 0x53, 0x1f,
- 0x60, 0xb4, 0x03, 0x4b, 0xde, 0x10, 0xf7, 0xf8, 0xc8, 0x3f, 0x9e, 0x43, 0x74, 0x6d, 0x64, 0xd3,
- 0x65, 0xda, 0x19, 0xe2, 0x9e, 0x4a, 0x99, 0xd1, 0x7b, 0xb0, 0xec, 0x8d, 0x0e, 0x23, 0xe6, 0x97,
- 0x0d, 0x78, 0x9a, 0x48, 0x4c, 0x8c, 0x69, 0x0f, 0x47, 0x7e, 0x60, 0xcf, 0x1e, 0x2f, 0xb4, 0x0d,
- 0x4b, 0x0a, 0xe5, 0xe5, 0x26, 0x86, 0x01, 0xa1, 0x2e, 0x64, 0x9c, 0x91, 0x4f, 0x31, 0x99, 0xd9,
- 0xda, 0x5e, 0x0c, 0xb3, 0xc5, 0x98, 0x19, 0x68, 0x00, 0x75, 0x66, 0x5a, 0xd2, 0x57, 0x9e, 0x16,
- 0xf1, 0x31, 0xe4, 0x23, 0xfd, 0x3f, 0x67, 0x79, 0xaf, 0x47, 0x97, 0x77, 0x2e, 0xba, 0x3f, 0xb6,
- 0xa1, 0x10, 0xed, 0xe6, 0x22, 0xbc, 0xd2, 0xdf, 0x2f, 0xc3, 0x5a, 0xc7, 0xd7, 0x6d, 0x43, 0x77,
- 0x8d, 0xc9, 0xb0, 0x3d, 0xe9, 0x2f, 0x92, 0x00, 0x6d, 0xd7, 0x1c, 0x98, 0xbe, 0x79, 0x8a, 0x3d,
- 0xf4, 0x75, 0x48, 0xb7, 0xcb, 0xaa, 0x56, 0x6d, 0x09, 0x31, 0xf1, 0xab, 0xbf, 0x20, 0xc7, 0xed,
- 0x1b, 0x64, 0x80, 0xdb, 0xe1, 0xe4, 0x6d, 0x0f, 0x75, 0xd7, 0x70, 0xb6, 0x4f, 0x37, 0xd1, 0x47,
- 0x90, 0xa9, 0xd5, 0xcb, 0xdd, 0xae, 0xdc, 0x14, 0xe2, 0xe2, 0x2d, 0xda, 0xf6, 0xcd, 0x99, 0xb6,
- 0x7d, 0x4b, 0xf7, 0x7d, 0x6c, 0x93, 0xd6, 0x9f, 0x40, 0xe1, 0xa9, 0xda, 0xda, 0x6b, 0x6b, 0x95,
- 0x97, 0xda, 0x73, 0xf9, 0xa5, 0x90, 0x10, 0xdf, 0xa3, 0x2c, 0x6f, 0xcf, 0xb0, 0x1c, 0xb9, 0xce,
- 0x68, 0xa8, 0x1d, 0x8e, 0xb5, 0x13, 0x3c, 0xe6, 0x52, 0x94, 0x46, 0x7b, 0xaf, 0xde, 0x91, 0x85,
- 0xe4, 0x05, 0x52, 0xcc, 0xc1, 0x70, 0x64, 0x79, 0x98, 0xb4, 0x7e, 0x08, 0xc5, 0x72, 0xa7, 0xa3,
- 0x3c, 0x6d, 0x72, 0x4f, 0xa2, 0x23, 0x2c, 0x89, 0xef, 0x52, 0xa6, 0xaf, 0xce, 0x30, 0xb1, 0x93,
- 0x4f, 0x33, 0x6d, 0x9f, 0x0e, 0xe6, 0x1e, 0xe4, 0xbb, 0x72, 0xa7, 0xab, 0x75, 0xba, 0xaa, 0x5c,
- 0x6e, 0x08, 0x29, 0x51, 0xa2, 0x5c, 0x6f, 0xcd, 0x70, 0xf9, 0xd8, 0xf3, 0x3d, 0xdf, 0x25, 0xc4,
- 0xd3, 0x4d, 0x74, 0x1f, 0xf2, 0x8d, 0x72, 0x3b, 0x14, 0x95, 0xbe, 0x40, 0xd4, 0x40, 0x1f, 0x6a,
- 0x4c, 0x9c, 0x47, 0xb8, 0x1e, 0xc1, 0x72, 0x43, 0x56, 0x9f, 0xca, 0x21, 0x5f, 0x46, 0xfc, 0x1a,
- 0xe5, 0xbb, 0x35, 0xcb, 0x87, 0xdd, 0x23, 0x1c, 0xe1, 0x94, 0x7c, 0x58, 0xaf, 0xe2, 0xa1, 0x8b,
- 0x7b, 0xba, 0x8f, 0x8d, 0xc8, 0xa4, 0xbd, 0x0f, 0x4b, 0xaa, 0x5c, 0xae, 0x0a, 0x31, 0xf1, 0x2d,
- 0x0a, 0x74, 0x73, 0x06, 0xc8, 0xc5, 0xba, 0xc1, 0xfb, 0xbb, 0xa3, 0xca, 0xe5, 0xae, 0xac, 0xed,
- 0x2b, 0xf2, 0x81, 0x10, 0xbf, 0xa0, 0xbf, 0x3d, 0x17, 0xeb, 0x3e, 0xd6, 0x4e, 0x4d, 0xfc, 0x8a,
- 0x48, 0xfd, 0xcf, 0x38, 0xf7, 0xae, 0x3c, 0xd3, 0xc7, 0x1e, 0xfa, 0x16, 0xac, 0xec, 0xb4, 0x1a,
- 0x15, 0xa5, 0x29, 0x6b, 0x6d, 0x59, 0xa5, 0x73, 0x19, 0x13, 0x3f, 0xa0, 0x40, 0xef, 0xcc, 0x02,
- 0x39, 0x83, 0x43, 0xd3, 0xc6, 0xda, 0x10, 0xbb, 0xc1, 0x74, 0x3e, 0x01, 0x21, 0xe0, 0x66, 0x2e,
- 0x5f, 0xfd, 0xa5, 0x10, 0x17, 0xef, 0x50, 0x76, 0xe9, 0x02, 0xf6, 0x23, 0xcb, 0x39, 0xd4, 0x2d,
- 0x8b, 0xf2, 0xdf, 0x85, 0x9c, 0x2a, 0x77, 0x76, 0xf7, 0x6a, 0xb5, 0xba, 0x2c, 0x24, 0xc4, 0x77,
- 0x28, 0xe3, 0x57, 0xce, 0x8c, 0xd7, 0x3b, 0x1e, 0xf5, 0xfb, 0x16, 0xe6, 0x83, 0x3e, 0x50, 0x95,
- 0xae, 0xac, 0xd5, 0x94, 0xba, 0xdc, 0x11, 0x92, 0x17, 0xad, 0x07, 0xd7, 0xf4, 0xb1, 0xd6, 0x37,
- 0x2d, 0x4c, 0x55, 0xfd, 0x9b, 0x04, 0xac, 0xee, 0x30, 0xf9, 0x11, 0xcf, 0x52, 0x05, 0x71, 0x66,
- 0xec, 0x5a, 0x5b, 0x95, 0x39, 0x49, 0x88, 0x89, 0x5b, 0x14, 0xfa, 0xa3, 0xd7, 0xab, 0x41, 0x23,
- 0x33, 0xc8, 0x48, 0xa4, 0x7f, 0x87, 0x20, 0xcd, 0x62, 0xb2, 0xe5, 0x51, 0xde, 0xd9, 0xd9, 0x6b,
- 0xec, 0xd5, 0xcb, 0xdd, 0x96, 0x4a, 0x9c, 0xe7, 0x6d, 0x8a, 0x7d, 0xff, 0x4b, 0xb0, 0xd9, 0x9a,
- 0xd1, 0x7b, 0xbd, 0xd1, 0x60, 0x64, 0xe9, 0xbe, 0xe3, 0xd2, 0x25, 0xf7, 0x7d, 0xb8, 0x35, 0x2b,
- 0x43, 0x7e, 0xd1, 0x55, 0xcb, 0x3b, 0x5d, 0xad, 0xb5, 0xd7, 0x6d, 0xef, 0x75, 0x89, 0x77, 0xfd,
- 0x90, 0x0a, 0xd8, 0xfc, 0x12, 0x01, 0xf8, 0x0b, 0xdf, 0xd5, 0x7b, 0xbe, 0xc6, 0x2d, 0x24, 0x41,
- 0x7f, 0x06, 0x37, 0xc3, 0x39, 0x25, 0x5b, 0x5c, 0xae, 0x6a, 0xfb, 0xe5, 0xfa, 0x1e, 0x55, 0x76,
- 0x89, 0x82, 0xde, 0xb9, 0x68, 0x66, 0xc9, 0x66, 0xc7, 0x86, 0x46, 0xcd, 0x14, 0xd5, 0xfb, 0xef,
- 0x2f, 0xc1, 0x9b, 0x9d, 0xa1, 0x65, 0xfa, 0xbe, 0x7e, 0x68, 0xe1, 0xb6, 0xee, 0x56, 0x9d, 0x88,
- 0xfe, 0xeb, 0x70, 0xa3, 0x5d, 0x56, 0x54, 0xed, 0x40, 0xe9, 0xee, 0x6a, 0xaa, 0xdc, 0xe9, 0xaa,
- 0xca, 0x4e, 0x57, 0x69, 0x35, 0x85, 0x98, 0xb8, 0x49, 0x05, 0xfd, 0xbf, 0x19, 0x41, 0x9e, 0xd1,
- 0xd7, 0x86, 0xba, 0xe9, 0x6a, 0xaf, 0x4c, 0xff, 0x58, 0x73, 0xb1, 0xe7, 0xbb, 0x26, 0x3d, 0xb2,
- 0x48, 0xbf, 0xab, 0xb0, 0xda, 0x69, 0xd7, 0x95, 0xee, 0x14, 0x52, 0x5c, 0xfc, 0x06, 0x45, 0xfa,
- 0xe0, 0x1c, 0x24, 0x8f, 0x74, 0x6c, 0x16, 0xa5, 0x09, 0x37, 0xdb, 0x6a, 0x6b, 0x47, 0xee, 0x74,
- 0x88, 0x5e, 0xe5, 0xaa, 0x26, 0xd7, 0xe5, 0x86, 0xdc, 0xa4, 0x2a, 0x3d, 0x7f, 0x3d, 0xd0, 0x4e,
- 0xb9, 0x4e, 0x0f, 0x7b, 0x1e, 0x51, 0x29, 0x36, 0x34, 0x6c, 0x61, 0xea, 0xf1, 0x10, 0xbc, 0x0a,
- 0x08, 0x01, 0x5e, 0x88, 0x94, 0x14, 0x3f, 0xa2, 0x48, 0xef, 0xbf, 0x06, 0x29, 0x8a, 0xf1, 0x02,
- 0xbe, 0xc2, 0x46, 0x56, 0x6e, 0x56, 0xb5, 0x8e, 0xf2, 0x99, 0x1c, 0x1d, 0x22, 0xb1, 0x89, 0xe7,
- 0xcf, 0xf5, 0x64, 0x8c, 0xba, 0x6d, 0x68, 0x9e, 0xf9, 0x63, 0x1c, 0x1d, 0x2c, 0x45, 0x76, 0xe0,
- 0x83, 0xa0, 0x77, 0x04, 0x77, 0x32, 0x5a, 0x2a, 0x6a, 0x4a, 0x4a, 0x4a, 0xac, 0x50, 0x29, 0xdf,
- 0x7a, 0x4d, 0xa7, 0x89, 0x8c, 0x70, 0xf8, 0x54, 0xea, 0x8c, 0x40, 0xe9, 0xb7, 0xe2, 0x70, 0x33,
- 0x38, 0xb7, 0x3a, 0xa6, 0x81, 0xe9, 0xd9, 0xd9, 0x1d, 0x0f, 0xb1, 0x27, 0x1d, 0xc3, 0x92, 0x6c,
- 0x8f, 0x06, 0xe8, 0x63, 0xc8, 0x2a, 0x5d, 0x59, 0x2d, 0x57, 0xea, 0x64, 0x0f, 0x46, 0x4d, 0x82,
- 0x67, 0x1a, 0x58, 0xa3, 0x0e, 0xc2, 0xb6, 0xe9, 0x63, 0x97, 0x2c, 0x29, 0x32, 0x88, 0x8f, 0x21,
- 0xdb, 0xd8, 0xab, 0x77, 0x95, 0x46, 0xb9, 0x2d, 0xc4, 0x2f, 0x62, 0x18, 0x8c, 0x2c, 0xdf, 0x1c,
- 0xe8, 0x43, 0xd2, 0x89, 0x5f, 0x24, 0x20, 0x1f, 0x71, 0xcb, 0x67, 0x7d, 0xa9, 0xf8, 0x19, 0x5f,
- 0xea, 0x4d, 0xc8, 0xd2, 0xd0, 0x47, 0x33, 0x0d, 0x7e, 0x14, 0x67, 0xe8, 0xb3, 0x62, 0xa0, 0x36,
- 0x80, 0xe9, 0x69, 0x87, 0xce, 0xc8, 0x36, 0xb0, 0x41, 0xfd, 0xbc, 0xe2, 0xd6, 0xe6, 0x1c, 0x0e,
- 0x85, 0xe2, 0x55, 0x18, 0x4f, 0x89, 0x0c, 0x5a, 0xcd, 0x99, 0xc1, 0x33, 0xda, 0x82, 0x1b, 0x67,
- 0x62, 0xc5, 0x31, 0x91, 0xbc, 0x44, 0x25, 0x9f, 0x09, 0xf2, 0xc6, 0x8a, 0x71, 0xc6, 0xb1, 0x49,
- 0x5d, 0xdd, 0xdf, 0xfc, 0x79, 0x06, 0x0a, 0x74, 0xc3, 0xb6, 0xf5, 0xb1, 0xe5, 0xe8, 0x06, 0x7a,
- 0x0a, 0x29, 0xc3, 0xd1, 0xfa, 0x36, 0xf7, 0x28, 0xb7, 0xe6, 0x00, 0xef, 0x18, 0x27, 0xd3, 0x4e,
- 0xa5, 0xe1, 0xd4, 0x6c, 0x54, 0x07, 0x18, 0xea, 0xae, 0x3e, 0xc0, 0x3e, 0x89, 0x4a, 0x59, 0xbc,
- 0xfd, 0xd1, 0x3c, 0xee, 0x5d, 0xc0, 0xa4, 0x46, 0xf8, 0xd1, 0x0f, 0x21, 0x3f, 0x99, 0xe6, 0xc0,
- 0x03, 0xfd, 0x74, 0x3e, 0xb8, 0x70, 0x70, 0xa5, 0x70, 0x2d, 0x06, 0x19, 0x02, 0x2f, 0x24, 0x50,
- 0x09, 0x3e, 0x39, 0x42, 0x89, 0x4b, 0x1c, 0xf8, 0xa3, 0x8b, 0x4b, 0x20, 0x10, 0x44, 0x0b, 0xa1,
- 0x84, 0x90, 0x40, 0x24, 0xf8, 0xe6, 0x00, 0xbb, 0x5c, 0x42, 0xea, 0x72, 0x12, 0xba, 0x04, 0x22,
- 0x2a, 0xc1, 0x0f, 0x09, 0xe8, 0x6d, 0x00, 0x2f, 0xb4, 0xc3, 0xd4, 0xef, 0xcd, 0xaa, 0x11, 0x0a,
- 0xba, 0x0b, 0xeb, 0x91, 0xad, 0xaa, 0x85, 0xab, 0x3d, 0x43, 0xd7, 0x1c, 0x8a, 0xbc, 0xdb, 0xe1,
- 0x0b, 0xff, 0x1e, 0xdc, 0x70, 0xf1, 0x8f, 0x46, 0xc4, 0x83, 0xd2, 0xfa, 0xa6, 0xad, 0x5b, 0xe6,
- 0x8f, 0x75, 0xf2, 0x7e, 0x23, 0x4b, 0xc1, 0xd7, 0x83, 0x97, 0xb5, 0xc8, 0x3b, 0xf1, 0x04, 0x56,
- 0x66, 0x34, 0x7d, 0x8e, 0xd7, 0x5b, 0x99, 0x0e, 0x08, 0xe7, 0x59, 0x1a, 0x21, 0x68, 0xd4, 0xbf,
- 0x26, 0xc2, 0xa6, 0x95, 0x7e, 0x4d, 0xc2, 0x02, 0xd0, 0x19, 0x61, 0x33, 0xfa, 0xbf, 0x1e, 0x61,
- 0x21, 0x68, 0xd4, 0xfb, 0xff, 0x55, 0x1c, 0x72, 0xe1, 0x6e, 0x40, 0xcf, 0x60, 0xc9, 0x1f, 0x0f,
- 0x99, 0xdd, 0x2a, 0x6e, 0x7d, 0xb2, 0xc8, 0x4e, 0x2a, 0x11, 0xd3, 0xcb, 0x2c, 0x10, 0xc5, 0x10,
- 0x3f, 0x83, 0x25, 0x42, 0x92, 0x54, 0x6e, 0x8c, 0x57, 0x20, 0xbf, 0xd7, 0xec, 0xb4, 0xe5, 0x1d,
- 0xa5, 0xa6, 0xc8, 0x55, 0x21, 0x86, 0x00, 0xd2, 0xcc, 0xd1, 0x15, 0xe2, 0x68, 0x1d, 0x84, 0xb6,
- 0xd2, 0x96, 0xeb, 0xc4, 0x55, 0x68, 0xb5, 0xd9, 0x31, 0x91, 0x40, 0x6f, 0xc0, 0x5a, 0xe4, 0xe0,
- 0xd0, 0x88, 0x5f, 0xf2, 0x5c, 0x56, 0x85, 0xa4, 0xf4, 0x6f, 0x49, 0xc8, 0x85, 0xba, 0x43, 0x2e,
- 0xdc, 0x24, 0x8e, 0xac, 0x36, 0x70, 0x0c, 0xb3, 0x3f, 0xd6, 0x98, 0xc3, 0x16, 0x89, 0x58, 0xbf,
- 0x39, 0xc7, 0x38, 0x54, 0xac, 0x1b, 0x0d, 0xca, 0x7f, 0x40, 0xd8, 0x43, 0xf0, 0xdd, 0x98, 0xba,
- 0xe6, 0xce, 0xbc, 0x23, 0x32, 0xeb, 0x90, 0x3d, 0xd4, 0x8f, 0x98, 0x94, 0xc4, 0xdc, 0x71, 0x71,
- 0x45, 0x3f, 0x8a, 0x22, 0x67, 0x0e, 0xf5, 0x23, 0x8a, 0xf6, 0x39, 0x14, 0x99, 0xe7, 0x43, 0x0d,
- 0x35, 0xc1, 0x64, 0x61, 0xfe, 0x83, 0xf9, 0xb2, 0x0c, 0x8c, 0x31, 0x8a, 0xbc, 0x1c, 0xc2, 0x05,
- 0xbd, 0x25, 0xb1, 0x06, 0x45, 0x5e, 0x9a, 0xbb, 0xb7, 0x0d, 0x7d, 0x38, 0xd5, 0xdb, 0x81, 0x3e,
- 0x0c, 0xd0, 0x3c, 0xec, 0x33, 0xb4, 0xd4, 0xdc, 0x68, 0x1d, 0xec, 0x4f, 0xa1, 0x79, 0xd8, 0x27,
- 0x3f, 0x2b, 0x69, 0x96, 0x5d, 0x90, 0x1e, 0xc0, 0xc6, 0x45, 0x93, 0x30, 0x75, 0x6a, 0xc6, 0xa7,
- 0x4e, 0x4d, 0xe9, 0x11, 0x14, 0xa2, 0x5a, 0x45, 0x77, 0x40, 0x08, 0xbc, 0x86, 0x19, 0x96, 0x22,
- 0xa7, 0x73, 0xb3, 0x23, 0xfd, 0x3c, 0x0e, 0xe8, 0xac, 0xf2, 0x88, 0xfd, 0x8a, 0x78, 0xc9, 0xb3,
- 0x20, 0x28, 0xf2, 0x2e, 0xb0, 0x5f, 0xdf, 0xa5, 0xf9, 0x21, 0xea, 0xb7, 0xf6, 0x6d, 0xbe, 0x1a,
- 0x2e, 0x73, 0xa6, 0xe5, 0x38, 0x4a, 0xcd, 0x96, 0xf6, 0xa1, 0x10, 0xd5, 0x3e, 0xba, 0x0d, 0x05,
- 0xe2, 0x63, 0xcf, 0x74, 0x06, 0x4e, 0xf0, 0x38, 0xe8, 0xc4, 0x7b, 0x50, 0xa4, 0xbb, 0x5a, 0x9b,
- 0x71, 0x2f, 0x0a, 0x94, 0xba, 0x33, 0xd1, 0x56, 0x74, 0x1e, 0x16, 0xd0, 0xd6, 0xcf, 0xe2, 0x90,
- 0x0b, 0x2d, 0x08, 0xea, 0xb0, 0x63, 0x46, 0x33, 0x9c, 0x81, 0x6e, 0xda, 0xdc, 0x5e, 0x6c, 0xcd,
- 0x69, 0x84, 0xaa, 0x94, 0x89, 0xd9, 0x0a, 0x7a, 0xb2, 0x30, 0x02, 0x19, 0x02, 0x3b, 0xbb, 0x66,
- 0x87, 0x40, 0xa9, 0x41, 0x47, 0xbe, 0x03, 0xb9, 0xd0, 0xe3, 0x91, 0xee, 0x5d, 0x64, 0x5c, 0x96,
- 0x21, 0xb7, 0xd7, 0xac, 0xb4, 0xf6, 0x9a, 0x55, 0xb9, 0x2a, 0xc4, 0x51, 0x1e, 0x32, 0xc1, 0x43,
- 0x42, 0xfa, 0xab, 0x38, 0xe4, 0xc9, 0x52, 0x0b, 0xdc, 0x91, 0x67, 0x90, 0xf6, 0x9c, 0x91, 0xdb,
- 0xc3, 0x57, 0xf0, 0x47, 0x38, 0xc2, 0x8c, 0x13, 0x97, 0xb8, 0xba, 0x13, 0x27, 0x19, 0xb0, 0xca,
- 0x12, 0xb0, 0x8a, 0xed, 0x87, 0x1e, 0x54, 0x0b, 0x72, 0x3c, 0x4f, 0x71, 0x25, 0x2f, 0x2a, 0xcb,
- 0x40, 0x6a, 0xb6, 0xf4, 0xc7, 0x71, 0x28, 0xf2, 0xb0, 0x36, 0x90, 0x31, 0xbd, 0xac, 0xe3, 0xd7,
- 0xb0, 0xac, 0x2f, 0xdc, 0x5b, 0x89, 0x8b, 0xf6, 0x96, 0xf4, 0xaf, 0x19, 0x58, 0xed, 0x62, 0xcf,
- 0xef, 0xd0, 0xdc, 0x4a, 0xd0, 0xb5, 0x8b, 0xed, 0x01, 0x52, 0x21, 0x8d, 0x4f, 0x69, 0xa2, 0x36,
- 0x31, 0x77, 0xb6, 0xef, 0x8c, 0x80, 0x92, 0x4c, 0x20, 0x54, 0x8e, 0x84, 0x3a, 0x90, 0x0d, 0x2e,
- 0xdf, 0xb8, 0x61, 0x7e, 0x38, 0x07, 0x6a, 0x79, 0x68, 0x76, 0xb0, 0x7b, 0x6a, 0xf6, 0x70, 0x35,
- 0xbc, 0x7d, 0x53, 0x43, 0x20, 0xf1, 0x4f, 0x52, 0x90, 0xa2, 0x62, 0xd0, 0x29, 0xac, 0xbc, 0xd2,
- 0x7d, 0xec, 0x0e, 0x74, 0xf7, 0x44, 0xa3, 0x22, 0xb9, 0xb6, 0x9f, 0x5f, 0xbe, 0xef, 0xa5, 0xb2,
- 0x71, 0xaa, 0xdb, 0x3d, 0x7c, 0x10, 0x00, 0xef, 0xc6, 0xd4, 0x62, 0x28, 0x85, 0xc9, 0xfd, 0x59,
- 0x1c, 0x6e, 0xf0, 0x78, 0x8b, 0x9c, 0x3b, 0x74, 0x43, 0x33, 0xf1, 0xcc, 0x86, 0xb5, 0xaf, 0x2e,
- 0xbe, 0x1d, 0xc2, 0x93, 0x8d, 0x4f, 0x0e, 0xd3, 0xe1, 0x14, 0x85, 0x75, 0x64, 0x00, 0xcb, 0x81,
- 0x15, 0x62, 0xf2, 0x99, 0x92, 0x6b, 0x57, 0x92, 0x6f, 0xc8, 0x3c, 0xee, 0xdd, 0x8d, 0xa9, 0x05,
- 0x0e, 0x4f, 0xdf, 0x89, 0x0a, 0x08, 0xb3, 0xda, 0x41, 0xef, 0xc2, 0xb2, 0x8d, 0x5f, 0x69, 0xa1,
- 0x86, 0xe8, 0x0c, 0x24, 0xd5, 0x82, 0x8d, 0x5f, 0x4d, 0x1a, 0x09, 0x90, 0xf4, 0xf5, 0x23, 0xbe,
- 0x5a, 0xc9, 0x4f, 0xb1, 0x02, 0x37, 0xce, 0x1d, 0x29, 0xfa, 0x3a, 0x08, 0x3a, 0x7b, 0xa1, 0x19,
- 0x23, 0x97, 0xb9, 0xb3, 0x0c, 0x72, 0x85, 0xd3, 0xab, 0x9c, 0x2c, 0xfe, 0x76, 0x1c, 0xf2, 0x91,
- 0xee, 0xa2, 0x1e, 0x64, 0x83, 0x90, 0x9d, 0xdf, 0x51, 0x3e, 0xbd, 0x94, 0x22, 0xba, 0xc1, 0x8d,
- 0x2e, 0x0e, 0xb0, 0xd5, 0x10, 0x38, 0x18, 0x4a, 0x32, 0x1c, 0x4a, 0x25, 0x03, 0x29, 0xaa, 0x7c,
- 0xf1, 0x7b, 0x80, 0xce, 0xb2, 0xa2, 0x0f, 0x60, 0x05, 0xdb, 0x64, 0x93, 0x85, 0x51, 0x39, 0x1d,
- 0x4f, 0x41, 0x2d, 0x72, 0x72, 0xd0, 0xf0, 0x2d, 0xc8, 0x85, 0x77, 0xc9, 0x54, 0x55, 0x49, 0x75,
- 0x42, 0x90, 0x56, 0x60, 0x99, 0x4e, 0x82, 0xa7, 0x32, 0xa7, 0x5e, 0xfa, 0xaf, 0x24, 0xac, 0xd2,
- 0xd3, 0xbe, 0x66, 0x5a, 0xd8, 0x0b, 0x36, 0x78, 0x0d, 0x96, 0x3c, 0xd3, 0x3e, 0xb9, 0x4a, 0x80,
- 0x48, 0xf8, 0xd1, 0xf7, 0x60, 0xa5, 0xef, 0xb8, 0x03, 0xdd, 0xd7, 0xfa, 0xfc, 0xe5, 0x15, 0xce,
- 0xe7, 0x22, 0x83, 0x0a, 0x68, 0x44, 0x25, 0xcc, 0x7e, 0x62, 0x83, 0x39, 0x9d, 0x1e, 0xd5, 0x67,
- 0x56, 0x2d, 0x06, 0x64, 0x3a, 0x30, 0x0f, 0x7d, 0x0b, 0x44, 0x7e, 0xe3, 0x6f, 0x10, 0x57, 0x79,
- 0x60, 0xda, 0xd8, 0xd0, 0xbc, 0x63, 0xdd, 0x35, 0x4c, 0xfb, 0x88, 0x3a, 0x64, 0x59, 0x75, 0x83,
- 0xb5, 0xa8, 0x86, 0x0d, 0x3a, 0xfc, 0x3d, 0xc2, 0xd3, 0x61, 0x29, 0x0b, 0xe9, 0xaa, 0xf3, 0xdc,
- 0xdb, 0xcd, 0xaa, 0xf5, 0x75, 0xb1, 0xe9, 0xff, 0x69, 0x40, 0x25, 0xfd, 0x04, 0x52, 0xd4, 0xc2,
- 0x5f, 0xcf, 0xdd, 0x52, 0x09, 0xd6, 0xc2, 0xfb, 0xb5, 0xf0, 0x50, 0x09, 0x6e, 0x98, 0x56, 0xc3,
- 0x57, 0xfc, 0x4c, 0xf1, 0xa4, 0x3f, 0x4d, 0x41, 0x31, 0x48, 0x1d, 0xb1, 0xcb, 0x4b, 0xe9, 0x77,
- 0x53, 0xdc, 0x93, 0x78, 0x0f, 0x52, 0x95, 0x97, 0x5d, 0xb9, 0x23, 0xc4, 0xc4, 0x37, 0x69, 0xfe,
- 0x67, 0x8d, 0xe6, 0x7f, 0x28, 0xea, 0xf6, 0xe1, 0xd8, 0xa7, 0xd9, 0x48, 0x74, 0x17, 0xf2, 0x24,
- 0x2e, 0x69, 0x3e, 0xd5, 0xf6, 0xba, 0xb5, 0x47, 0x02, 0x4c, 0x5d, 0x40, 0xb0, 0xb6, 0x24, 0xcc,
- 0xb5, 0x8f, 0xb4, 0x91, 0xdf, 0x7f, 0x44, 0x38, 0xde, 0x86, 0xc4, 0xf3, 0x7d, 0x21, 0x2e, 0xde,
- 0xa4, 0x0d, 0x85, 0x48, 0xc3, 0x93, 0x53, 0xf2, 0x5e, 0x82, 0xa5, 0x4a, 0xab, 0x55, 0x17, 0x0a,
- 0xe2, 0x06, 0x6d, 0x81, 0xa2, 0x62, 0x1d, 0xc7, 0x22, 0x6d, 0xde, 0x87, 0xf4, 0x7e, 0x59, 0x55,
- 0x9a, 0x5d, 0x21, 0x21, 0x8a, 0xb4, 0xd5, 0x7a, 0xa4, 0xd5, 0xa9, 0xee, 0x9a, 0xb6, 0xcf, 0xdb,
- 0x55, 0x5b, 0x7b, 0x95, 0xba, 0x2c, 0xe4, 0xcf, 0x69, 0x67, 0x38, 0x23, 0x9e, 0xee, 0xfa, 0x30,
- 0x92, 0x1f, 0x4b, 0x4e, 0x5d, 0x11, 0xb0, 0x96, 0xd1, 0xd4, 0xd8, 0x7b, 0x90, 0xea, 0x2a, 0x0d,
- 0x59, 0x15, 0x96, 0xce, 0xd1, 0x0b, 0x75, 0xd0, 0xd8, 0x15, 0xc6, 0x8a, 0xd2, 0xec, 0xca, 0xea,
- 0x7e, 0x58, 0xb2, 0x21, 0xa4, 0xa6, 0xf2, 0xea, 0x1c, 0xd8, 0xf6, 0xb1, 0x7b, 0xaa, 0x5b, 0xfc,
- 0x0e, 0x83, 0x65, 0xe3, 0x97, 0xeb, 0x72, 0xf3, 0x69, 0x77, 0x57, 0x6b, 0xab, 0x72, 0x4d, 0x79,
- 0x21, 0xa4, 0xa7, 0xf2, 0x6f, 0x8c, 0xcf, 0xc2, 0xf6, 0x91, 0x7f, 0xac, 0x0d, 0x5d, 0xdc, 0x37,
- 0xbf, 0xe0, 0x5c, 0x53, 0x05, 0x22, 0x42, 0xe6, 0x1c, 0x2e, 0x76, 0x4d, 0x10, 0x91, 0xf5, 0x09,
- 0x14, 0x59, 0xf3, 0x20, 0x21, 0x2d, 0x64, 0xa7, 0xae, 0x75, 0x18, 0x5b, 0xb8, 0xb7, 0xd9, 0xb2,
- 0xa5, 0x79, 0xe1, 0x1b, 0x9d, 0x6e, 0xb9, 0x2b, 0x6b, 0x15, 0x12, 0x88, 0x56, 0xb5, 0x50, 0x79,
- 0x39, 0xf1, 0xeb, 0x94, 0xfd, 0xdd, 0xa9, 0xf9, 0xd7, 0x7d, 0xac, 0x1d, 0xea, 0xbd, 0x13, 0x6c,
- 0x68, 0x51, 0x4d, 0xde, 0x86, 0xa4, 0xda, 0x3a, 0x10, 0x96, 0xc5, 0x37, 0x28, 0xcf, 0x6a, 0x84,
- 0xc7, 0xa5, 0xfd, 0x93, 0x7e, 0x27, 0x1d, 0xf8, 0x7c, 0x91, 0xdc, 0xdc, 0xb5, 0xfb, 0x7c, 0x68,
- 0x1f, 0x0a, 0xec, 0x56, 0x80, 0x74, 0x75, 0xe4, 0x71, 0x6f, 0xf5, 0xde, 0x3c, 0x91, 0x21, 0x61,
- 0xeb, 0x50, 0x2e, 0xe6, 0xaf, 0xe6, 0x07, 0x13, 0x0a, 0x7a, 0x3f, 0xb0, 0x8b, 0x13, 0x07, 0x8f,
- 0x9d, 0x33, 0xcb, 0x8c, 0x1c, 0x84, 0x2c, 0x55, 0xc8, 0xf8, 0xae, 0x79, 0x74, 0x84, 0x5d, 0x1e,
- 0x94, 0x7e, 0x38, 0xcf, 0x39, 0xc7, 0x38, 0xd4, 0x80, 0x15, 0x61, 0x58, 0x0d, 0xfd, 0x46, 0xd3,
- 0xb1, 0x35, 0xc2, 0x42, 0xc3, 0xd2, 0xe2, 0xd6, 0xa3, 0x79, 0xbc, 0xb4, 0x08, 0x6f, 0xc3, 0x31,
- 0x78, 0x0a, 0x43, 0xd0, 0x67, 0xc8, 0x24, 0xe2, 0x61, 0x37, 0x1b, 0xd4, 0x4f, 0xa2, 0x79, 0xaf,
- 0xf9, 0x22, 0x1e, 0x76, 0x31, 0x4b, 0x4e, 0x54, 0x1e, 0xf1, 0x38, 0x21, 0x01, 0x1d, 0x82, 0xd0,
- 0xb3, 0x1c, 0xea, 0x7d, 0x1d, 0xe2, 0x63, 0xfd, 0xd4, 0x74, 0x5c, 0x9a, 0x27, 0x2b, 0xce, 0xe5,
- 0x60, 0xee, 0x30, 0xd6, 0x0a, 0xe7, 0x64, 0xf0, 0x2b, 0xbd, 0x69, 0x2a, 0xf5, 0x44, 0x2c, 0x8b,
- 0x2e, 0x64, 0x4b, 0xf7, 0xb1, 0x8d, 0x3d, 0x8f, 0x26, 0xd6, 0x88, 0x27, 0xc2, 0xe8, 0x75, 0x4e,
- 0x46, 0x9f, 0x43, 0xb1, 0x65, 0x93, 0x8e, 0x05, 0xcc, 0x1b, 0xb9, 0xb9, 0x13, 0x41, 0xd3, 0x8c,
- 0xac, 0x2f, 0x33, 0x68, 0x68, 0x13, 0x6e, 0xe8, 0x9e, 0x67, 0x1e, 0xd9, 0x9e, 0xe6, 0x3b, 0x9a,
- 0x63, 0x07, 0x77, 0x98, 0x1b, 0x40, 0x8f, 0x40, 0xc4, 0x5f, 0x76, 0x9d, 0x96, 0x8d, 0xd9, 0xfa,
- 0x97, 0xbe, 0x0f, 0xf9, 0xc8, 0x62, 0x93, 0x1a, 0x17, 0xc5, 0x7b, 0x2b, 0x90, 0x6f, 0xb6, 0x9a,
- 0xf4, 0x82, 0x4c, 0x69, 0x3e, 0x15, 0xe2, 0x94, 0x20, 0xcb, 0xd5, 0x0e, 0xbb, 0x33, 0x13, 0x12,
- 0x08, 0x41, 0xb1, 0x5c, 0x57, 0xe5, 0x72, 0x95, 0x5f, 0xa3, 0x55, 0x85, 0xa4, 0xf4, 0x03, 0x10,
- 0x66, 0xe7, 0x5f, 0x52, 0x2e, 0x12, 0x51, 0x04, 0xa8, 0x2a, 0x9d, 0x9d, 0xb2, 0x5a, 0x65, 0x12,
- 0x04, 0x28, 0x84, 0x37, 0x71, 0x84, 0x92, 0x20, 0x2d, 0x54, 0x99, 0xde, 0x9e, 0x91, 0xe7, 0xa4,
- 0xf4, 0x5d, 0x58, 0x99, 0x99, 0x23, 0xe9, 0xc9, 0x6b, 0x06, 0x20, 0x37, 0x94, 0xae, 0x56, 0xae,
- 0x1f, 0x94, 0x5f, 0x76, 0x58, 0x4a, 0x8c, 0x12, 0x94, 0x9a, 0xd6, 0x6c, 0x35, 0xe5, 0x46, 0xbb,
- 0xfb, 0x52, 0x48, 0x48, 0xed, 0xd9, 0x29, 0x7a, 0x2d, 0x62, 0x4d, 0x51, 0xe5, 0x29, 0x44, 0x4a,
- 0x98, 0x46, 0x3c, 0x04, 0x98, 0x2c, 0x51, 0xa9, 0x7b, 0x11, 0xda, 0x2a, 0x2c, 0xcb, 0xcd, 0xaa,
- 0xd6, 0xaa, 0x69, 0x61, 0xd2, 0x0e, 0x41, 0xb1, 0x5e, 0xa6, 0x97, 0xe3, 0x4a, 0x53, 0x6b, 0x97,
- 0x9b, 0x44, 0xcb, 0xa4, 0xd7, 0x65, 0xb5, 0xae, 0x44, 0xa9, 0x49, 0xc9, 0x02, 0x98, 0x04, 0xfe,
- 0xd2, 0xe7, 0xaf, 0xd1, 0xb0, 0xbc, 0x2f, 0x37, 0xbb, 0xb4, 0xc4, 0x4f, 0x88, 0xa3, 0x35, 0x58,
- 0xe1, 0x77, 0x4a, 0xe4, 0xa4, 0xa5, 0xc4, 0x04, 0xba, 0x0d, 0x6f, 0x75, 0x5e, 0x36, 0x77, 0x76,
- 0xd5, 0x56, 0x93, 0xde, 0x33, 0xcd, 0xb6, 0x48, 0x4a, 0xbf, 0x14, 0x20, 0xc3, 0xcd, 0x04, 0x52,
- 0x21, 0xa7, 0xf7, 0x7d, 0xec, 0x6a, 0xba, 0x65, 0x71, 0xa3, 0x79, 0x6f, 0x7e, 0x2b, 0x53, 0x2a,
- 0x13, 0xde, 0xb2, 0x65, 0xed, 0xc6, 0xd4, 0xac, 0xce, 0x7f, 0x47, 0x30, 0xed, 0x31, 0x77, 0x84,
- 0x16, 0xc7, 0xb4, 0xc7, 0x13, 0x4c, 0x7b, 0x8c, 0xf6, 0x00, 0x18, 0x26, 0xd6, 0x7b, 0xc7, 0x3c,
- 0xfe, 0xb9, 0xbf, 0x28, 0xa8, 0xac, 0xf7, 0x8e, 0x77, 0x63, 0x2a, 0xeb, 0x1d, 0x79, 0x40, 0x16,
- 0xac, 0x71, 0x58, 0xdb, 0xd0, 0x9c, 0x7e, 0xb0, 0xdf, 0x96, 0xe6, 0xce, 0x8b, 0x4e, 0xe3, 0xdb,
- 0x46, 0xab, 0xcf, 0x36, 0xe6, 0x6e, 0x4c, 0x15, 0xf4, 0x19, 0x1a, 0xf2, 0xe1, 0x06, 0x93, 0x36,
- 0x13, 0x55, 0xf2, 0x2c, 0xe1, 0x93, 0x45, 0xe5, 0x9d, 0x8d, 0x1e, 0xf5, 0xb3, 0x64, 0xf4, 0xf3,
- 0x38, 0x48, 0x4c, 0xac, 0x37, 0xb6, 0x7b, 0xc7, 0xae, 0x63, 0xd3, 0xbb, 0xc3, 0xd9, 0x3e, 0xb0,
- 0x0a, 0x9d, 0x67, 0x8b, 0xf6, 0xa1, 0x13, 0xc1, 0x3c, 0xd3, 0x9f, 0x5b, 0xfa, 0xeb, 0x9b, 0xa0,
- 0xe7, 0x90, 0xd6, 0xad, 0x57, 0xfa, 0xd8, 0xdb, 0x28, 0x50, 0xf1, 0x9b, 0x8b, 0x88, 0xa7, 0x8c,
- 0xbb, 0x31, 0x95, 0x43, 0xa0, 0x26, 0x64, 0x0c, 0xdc, 0xd7, 0x47, 0x96, 0x4f, 0x0f, 0x89, 0xf9,
- 0x8e, 0xff, 0x00, 0xad, 0xca, 0x38, 0x77, 0x63, 0x6a, 0x00, 0x82, 0x3e, 0x9f, 0x84, 0xdd, 0x3d,
- 0x67, 0x64, 0xfb, 0xf4, 0x58, 0x98, 0x2f, 0xb7, 0x11, 0xa0, 0xca, 0x41, 0x92, 0x70, 0x64, 0xfb,
- 0x91, 0x38, 0x9b, 0x3e, 0xa3, 0x5d, 0x48, 0xd9, 0xf8, 0x14, 0xb3, 0x53, 0x24, 0xbf, 0x75, 0x77,
- 0x01, 0xdc, 0x26, 0xe1, 0xdb, 0x8d, 0xa9, 0x0c, 0x80, 0xec, 0x0e, 0xc7, 0x65, 0x77, 0x43, 0xd6,
- 0x98, 0x9e, 0x16, 0x8b, 0xed, 0x8e, 0x96, 0x5b, 0x63, 0xbc, 0x64, 0x77, 0x38, 0xc1, 0x03, 0x99,
- 0x1d, 0x17, 0x0f, 0xb1, 0xee, 0x6f, 0xe4, 0x17, 0x9e, 0x1d, 0x95, 0x32, 0x92, 0xd9, 0x61, 0x10,
- 0xe2, 0x0b, 0xc8, 0x06, 0xd6, 0x02, 0xd5, 0x21, 0x4f, 0xeb, 0xda, 0x68, 0xd3, 0x20, 0x8a, 0x5f,
- 0xc4, 0xbb, 0x89, 0xb2, 0x4f, 0x90, 0xed, 0xf1, 0x35, 0x23, 0xbf, 0x84, 0x5c, 0x68, 0x38, 0xae,
- 0x19, 0xfa, 0x6f, 0xe3, 0x20, 0xcc, 0x1a, 0x0d, 0xd4, 0x82, 0x65, 0xac, 0xbb, 0xd6, 0x58, 0xeb,
- 0x9b, 0x24, 0x38, 0x0a, 0x8a, 0x29, 0x17, 0x11, 0x52, 0xa0, 0x00, 0x35, 0xc6, 0x8f, 0x1a, 0x50,
- 0x20, 0x4e, 0x4d, 0x88, 0x97, 0x58, 0x18, 0x2f, 0x4f, 0xf8, 0x39, 0x9c, 0xf8, 0xff, 0x61, 0xed,
- 0x1c, 0xc3, 0x83, 0x8e, 0x61, 0x3d, 0xcc, 0x60, 0x68, 0x67, 0x2a, 0xc8, 0x1f, 0xcc, 0x99, 0xf6,
- 0xa6, 0xec, 0x93, 0x92, 0xe1, 0x35, 0xff, 0x0c, 0xcd, 0x13, 0xdf, 0x81, 0x5b, 0x5f, 0x62, 0x75,
- 0xc4, 0x1c, 0x64, 0xf8, 0x5e, 0x16, 0xef, 0x41, 0x21, 0xba, 0x01, 0xd1, 0xbb, 0xb3, 0x1b, 0x9a,
- 0xa8, 0x37, 0x35, 0xbd, 0x2b, 0xc5, 0x0c, 0xa4, 0xe8, 0xee, 0x12, 0xb3, 0x90, 0x66, 0x26, 0x46,
- 0xfc, 0xa3, 0x38, 0xe4, 0xc2, 0x2d, 0x82, 0x9e, 0xc0, 0x52, 0x98, 0xd4, 0x5f, 0x4c, 0x97, 0x94,
- 0x8f, 0xb8, 0xf5, 0xc1, 0x4e, 0x5d, 0x7c, 0x3a, 0x02, 0x56, 0xb1, 0x0b, 0x69, 0xb6, 0xc5, 0xd0,
- 0x33, 0x80, 0xc9, 0xc2, 0xba, 0x44, 0xaf, 0x22, 0xdc, 0x95, 0x5c, 0x18, 0x72, 0x48, 0xff, 0x94,
- 0x88, 0xe4, 0xb9, 0x26, 0xd5, 0xb0, 0x1d, 0x48, 0x19, 0xd8, 0xd2, 0xc7, 0x0b, 0xdc, 0x1d, 0x9e,
- 0x45, 0x29, 0x55, 0x09, 0x04, 0xb1, 0x5f, 0x14, 0x0b, 0x7d, 0x06, 0x59, 0xdd, 0x32, 0x8f, 0x6c,
- 0xcd, 0x77, 0xb8, 0x4e, 0xbe, 0x7d, 0x39, 0xdc, 0x32, 0x41, 0xe9, 0x3a, 0xc4, 0x8a, 0xeb, 0xec,
- 0xa7, 0xf8, 0x21, 0xa4, 0xa8, 0x34, 0xf4, 0x0e, 0x14, 0xa8, 0x34, 0x6d, 0x60, 0x5a, 0x96, 0xe9,
- 0xf1, 0x74, 0x63, 0x9e, 0xd2, 0x1a, 0x94, 0x24, 0x3e, 0x86, 0x0c, 0x47, 0x40, 0x37, 0x21, 0x3d,
- 0xc4, 0xae, 0xe9, 0xb0, 0xd8, 0x2c, 0xa9, 0xf2, 0x27, 0x42, 0x77, 0xfa, 0x7d, 0x0f, 0xfb, 0xd4,
- 0x49, 0x48, 0xaa, 0xfc, 0xa9, 0x72, 0x03, 0xd6, 0xce, 0xd9, 0x03, 0xd2, 0x1f, 0x26, 0x20, 0x17,
- 0x66, 0x78, 0xd0, 0x3e, 0x14, 0xf5, 0x1e, 0xad, 0xdf, 0x19, 0xea, 0xbe, 0x8f, 0x5d, 0xfb, 0xb2,
- 0x79, 0x9d, 0x65, 0x06, 0xd3, 0x66, 0x28, 0xe8, 0x39, 0x64, 0x4e, 0x4d, 0xfc, 0xea, 0x6a, 0xd7,
- 0x6b, 0x69, 0x02, 0x51, 0xb3, 0xd1, 0xe7, 0xb0, 0xca, 0xc3, 0xd3, 0x81, 0x3e, 0x1c, 0x12, 0xff,
- 0xa0, 0x6f, 0x73, 0x8f, 0xeb, 0x32, 0xb0, 0x3c, 0xd6, 0x6d, 0x30, 0xac, 0x9a, 0x2d, 0x7d, 0x0a,
- 0xf9, 0x48, 0x55, 0x39, 0x12, 0x20, 0x39, 0x72, 0xed, 0x20, 0x69, 0x3c, 0x72, 0x6d, 0xb4, 0x01,
- 0x99, 0x21, 0x4b, 0xc8, 0x51, 0xb1, 0x05, 0x35, 0x78, 0x7c, 0xb6, 0x94, 0x8d, 0x0b, 0x09, 0xe9,
- 0xcf, 0xe2, 0xb0, 0x1e, 0xa4, 0xa7, 0xa2, 0x65, 0xef, 0xd2, 0xcf, 0xe2, 0x50, 0x88, 0x12, 0xd0,
- 0x7b, 0x90, 0xae, 0xb6, 0xe8, 0x9d, 0x78, 0x6c, 0x2a, 0x6d, 0x84, 0xed, 0xd3, 0x6d, 0xc3, 0xe9,
- 0x9d, 0xb0, 0xa4, 0xcc, 0xfb, 0x90, 0xe1, 0x4e, 0xb2, 0x10, 0x9f, 0x4a, 0xde, 0x90, 0x66, 0xdc,
- 0x4d, 0x22, 0xed, 0xee, 0x40, 0x56, 0x7e, 0xd1, 0x95, 0xd5, 0x66, 0xb9, 0x3e, 0x93, 0x60, 0x22,
- 0x0d, 0xf1, 0x17, 0x64, 0x2a, 0x74, 0x6b, 0xfb, 0x74, 0x53, 0x7a, 0x04, 0xcb, 0x55, 0x0a, 0x1f,
- 0xe4, 0x6b, 0x3f, 0x80, 0x95, 0x9e, 0x63, 0xfb, 0xba, 0x69, 0x93, 0x78, 0x7f, 0xa0, 0x1f, 0x05,
- 0xb5, 0x4f, 0xc5, 0x90, 0xac, 0x10, 0xaa, 0xf4, 0xef, 0x71, 0x28, 0x72, 0x83, 0x16, 0xf0, 0x16,
- 0x21, 0xe1, 0x78, 0xbc, 0x79, 0xc2, 0xf1, 0x10, 0x82, 0x25, 0xdd, 0xed, 0x1d, 0x73, 0x8d, 0xd1,
- 0xdf, 0x44, 0x65, 0x3d, 0x67, 0x30, 0xd0, 0xed, 0x20, 0x95, 0x10, 0x3c, 0xa2, 0x3a, 0x24, 0xb1,
- 0x7d, 0xba, 0x48, 0x69, 0xf7, 0x94, 0xf4, 0x92, 0x6c, 0x9f, 0xb2, 0x5c, 0x28, 0x81, 0x11, 0x3f,
- 0x81, 0x6c, 0x40, 0x58, 0xa8, 0x88, 0xfa, 0xbf, 0xe3, 0xb0, 0x22, 0x73, 0x05, 0x05, 0xe3, 0x8a,
- 0xde, 0x1a, 0xc5, 0xaf, 0xe9, 0xd6, 0x08, 0xed, 0x43, 0x9a, 0x56, 0x2c, 0x05, 0xd7, 0x5b, 0xf3,
- 0xf8, 0xd4, 0x33, 0x1d, 0x63, 0x35, 0x1b, 0x41, 0x95, 0x3c, 0x43, 0x13, 0x1f, 0x43, 0x3e, 0x42,
- 0x5e, 0x68, 0xec, 0x3f, 0x85, 0x95, 0x99, 0x3d, 0x71, 0x3d, 0x59, 0xdd, 0xaf, 0x41, 0x31, 0xf2,
- 0x19, 0xcf, 0xe4, 0x9a, 0x70, 0x39, 0x42, 0x55, 0x0c, 0x69, 0x1b, 0x0a, 0x53, 0xb2, 0xf9, 0x7e,
- 0x8b, 0xcf, 0xb1, 0xdf, 0xa4, 0xdf, 0x2c, 0x41, 0x3e, 0x52, 0xb6, 0x86, 0x14, 0x48, 0x99, 0x3e,
- 0x0e, 0x4f, 0xf6, 0x7b, 0x8b, 0x55, 0xbd, 0x95, 0x14, 0x1f, 0x0f, 0x54, 0x86, 0x20, 0xf6, 0x01,
- 0x14, 0x03, 0xdb, 0xbe, 0xd9, 0x37, 0xb1, 0x4b, 0x6c, 0x73, 0xf4, 0x73, 0x0f, 0xde, 0xbb, 0xbc,
- 0x3f, 0xf9, 0xd2, 0x83, 0x1c, 0xde, 0x93, 0x26, 0x13, 0x8b, 0x31, 0xe1, 0xdb, 0x73, 0xed, 0x60,
- 0x5e, 0x92, 0xe1, 0xbc, 0x88, 0xbf, 0x4a, 0xc0, 0x12, 0x91, 0x8b, 0x14, 0x48, 0x70, 0xe0, 0xf9,
- 0x3e, 0x9b, 0x98, 0xea, 0x78, 0xd8, 0x53, 0x35, 0x61, 0x92, 0x3d, 0xc5, 0xca, 0x80, 0x12, 0x73,
- 0x67, 0xd1, 0xa2, 0x60, 0x33, 0x85, 0x40, 0xe8, 0xc3, 0x60, 0xe5, 0x30, 0x1b, 0xbb, 0x5e, 0x62,
- 0x9f, 0x1e, 0x96, 0x82, 0x4f, 0x0f, 0x4b, 0x65, 0x3b, 0xf8, 0xa2, 0x08, 0x3d, 0x80, 0xbc, 0x77,
- 0xec, 0xb8, 0x3e, 0xcb, 0xb9, 0xf2, 0x38, 0xf5, 0x7c, 0x0e, 0xa0, 0x0d, 0xf7, 0x29, 0xdb, 0x3a,
- 0xa4, 0x2c, 0xfd, 0x10, 0x5b, 0xfc, 0xe3, 0x15, 0xf6, 0x80, 0xde, 0x84, 0xac, 0x65, 0xda, 0x27,
- 0xda, 0xc8, 0xb5, 0x68, 0xf4, 0x97, 0x53, 0x33, 0xe4, 0x79, 0xcf, 0xb5, 0xc4, 0x9f, 0xf2, 0xe2,
- 0xa4, 0xd1, 0x6b, 0x8a, 0x93, 0x58, 0x82, 0x9f, 0x15, 0x0f, 0x28, 0xcd, 0xae, 0xfc, 0x54, 0x56,
- 0x85, 0x04, 0xca, 0x41, 0xaa, 0x56, 0x6f, 0x95, 0xbb, 0x42, 0x92, 0x15, 0x15, 0xb4, 0xea, 0x72,
- 0xb9, 0x29, 0x2c, 0xa1, 0x65, 0xc8, 0x85, 0x1f, 0x26, 0x0a, 0x29, 0x54, 0x80, 0x6c, 0x75, 0x4f,
- 0x2d, 0xd3, 0xca, 0xe1, 0x34, 0x2a, 0x02, 0x3c, 0x2b, 0xef, 0x97, 0xb5, 0x9d, 0x7a, 0xb9, 0xd3,
- 0x11, 0x32, 0xd2, 0x3f, 0x66, 0xe1, 0x46, 0x03, 0x7b, 0x9e, 0x7e, 0x84, 0x0f, 0x4c, 0xff, 0x38,
- 0x52, 0xc8, 0x7c, 0xcd, 0xdf, 0x1a, 0x7d, 0x07, 0x52, 0x34, 0x07, 0xbb, 0xe8, 0xc7, 0x57, 0xc4,
- 0x75, 0xa1, 0x8c, 0xe8, 0xfb, 0xc4, 0xb2, 0xf3, 0x4a, 0xef, 0xc8, 0x26, 0x9a, 0x2f, 0x58, 0x9a,
- 0xae, 0x28, 0xd8, 0x8d, 0xa9, 0xbc, 0xcc, 0x29, 0xac, 0x31, 0xf8, 0x21, 0xac, 0x7a, 0xc6, 0x49,
- 0x78, 0x39, 0x17, 0xad, 0x50, 0xba, 0xc4, 0x59, 0xbc, 0x1b, 0x53, 0x57, 0xbc, 0x19, 0x53, 0x74,
- 0x00, 0xc5, 0xa1, 0xee, 0x6a, 0x86, 0x13, 0x76, 0x3f, 0x3d, 0xb7, 0x51, 0x8a, 0xd6, 0x44, 0x92,
- 0xe8, 0x76, 0x18, 0x2d, 0x62, 0x6d, 0x01, 0x0c, 0xc3, 0xbd, 0xc9, 0x03, 0xf2, 0xc5, 0xbe, 0x1a,
- 0xdc, 0x8d, 0xa9, 0x11, 0x08, 0xa4, 0x42, 0x3e, 0xf2, 0xa5, 0x27, 0x0f, 0xc6, 0x17, 0xfc, 0x2e,
- 0x70, 0x37, 0xa6, 0x46, 0x41, 0x50, 0x07, 0x0a, 0xb4, 0x34, 0x2e, 0x18, 0x7b, 0x6e, 0x6e, 0xd0,
- 0x48, 0x81, 0x0c, 0x01, 0x75, 0x23, 0xf5, 0x32, 0x0d, 0x80, 0xc9, 0x85, 0x24, 0x0f, 0x9d, 0x17,
- 0xba, 0x09, 0x24, 0x51, 0x78, 0x78, 0xf3, 0x88, 0xfa, 0xb0, 0x16, 0xf9, 0xe6, 0x26, 0xec, 0x6a,
- 0x61, 0xc1, 0xef, 0x13, 0x23, 0xe5, 0x31, 0xbb, 0x31, 0x95, 0xbb, 0x78, 0xd1, 0x9a, 0x19, 0x0c,
- 0xe8, 0x6c, 0x35, 0xf4, 0xc6, 0xf2, 0xe5, 0x3f, 0x83, 0x9c, 0x88, 0x89, 0x5e, 0xd3, 0xec, 0xc3,
- 0xf2, 0xf4, 0x72, 0x2e, 0x5e, 0xea, 0x10, 0x24, 0xeb, 0xad, 0x1f, 0x79, 0xae, 0xa4, 0x61, 0xc9,
- 0x75, 0x1c, 0x5f, 0xfa, 0x65, 0x1a, 0x6e, 0xca, 0x5f, 0xe0, 0xde, 0x88, 0x96, 0xdb, 0x76, 0x7c,
- 0xfd, 0x28, 0xdc, 0x4d, 0x6d, 0xc8, 0x47, 0xce, 0x46, 0x6e, 0x3d, 0x16, 0xfd, 0x0a, 0x32, 0x0a,
- 0x41, 0x0c, 0x2b, 0x9b, 0x65, 0x7e, 0xea, 0x9b, 0x7c, 0xc6, 0xce, 0x29, 0x94, 0x96, 0xe7, 0xf2,
- 0x44, 0xce, 0xeb, 0xf7, 0x64, 0x61, 0x28, 0xc6, 0x54, 0xb9, 0xf4, 0xdb, 0x53, 0xdf, 0x6b, 0x2f,
- 0xd1, 0xeb, 0xdc, 0xe8, 0x07, 0xd7, 0x1b, 0x93, 0x4f, 0xfb, 0x52, 0xf4, 0x65, 0xf8, 0x79, 0xde,
- 0xb4, 0x19, 0x4d, 0x5f, 0xd5, 0x8c, 0xf6, 0x21, 0x3f, 0xf2, 0xb0, 0x4b, 0x2f, 0xca, 0xb0, 0xb7,
- 0x91, 0xb9, 0xea, 0x80, 0xf7, 0x3c, 0xec, 0xd2, 0x22, 0x3c, 0x32, 0xe0, 0x51, 0xf0, 0xe0, 0xa1,
- 0x97, 0x90, 0xa6, 0x57, 0xa9, 0xde, 0x46, 0x96, 0x8a, 0x28, 0x5f, 0x5e, 0x04, 0xad, 0xd5, 0x53,
- 0x0c, 0x95, 0x03, 0x8a, 0x2d, 0xc8, 0x47, 0xd4, 0x3c, 0x8f, 0x43, 0xf2, 0x55, 0x00, 0xcb, 0xe9,
- 0xe9, 0x16, 0xfb, 0x94, 0x81, 0x2d, 0x80, 0x1c, 0xa5, 0x34, 0xf5, 0x01, 0x26, 0x80, 0x91, 0x61,
- 0x5c, 0x03, 0xe0, 0x73, 0xc8, 0xf0, 0x4e, 0x5f, 0x1d, 0x6c, 0xeb, 0x0f, 0xe2, 0xd1, 0xba, 0x31,
- 0xee, 0x51, 0xa3, 0x1f, 0x43, 0x9a, 0x55, 0x9f, 0xa0, 0x79, 0x92, 0x91, 0x53, 0x85, 0x2a, 0xe2,
- 0x15, 0x0a, 0xc9, 0xa4, 0xd8, 0xdd, 0xf8, 0xf6, 0xa7, 0x90, 0xa5, 0xff, 0xfd, 0x40, 0x3c, 0xd2,
- 0x77, 0xce, 0x78, 0x34, 0xc4, 0x0b, 0xa1, 0xbe, 0x4c, 0x6b, 0xc8, 0xfe, 0x3c, 0xe0, 0x9f, 0xff,
- 0xfc, 0xaf, 0x5f, 0x30, 0x9f, 0x85, 0x70, 0xed, 0xb9, 0xf6, 0xb6, 0x02, 0xcb, 0x14, 0xa0, 0xc7,
- 0xff, 0x83, 0x61, 0x1e, 0x94, 0x7f, 0x09, 0x50, 0x0a, 0x87, 0x91, 0xff, 0x72, 0xa8, 0x7c, 0x13,
- 0xbe, 0xfc, 0x0f, 0x27, 0x2a, 0x39, 0x95, 0x56, 0xa4, 0x94, 0x87, 0xe6, 0x67, 0xf9, 0x80, 0xae,
- 0x9d, 0x6e, 0x1e, 0xa6, 0xa9, 0xb8, 0x7b, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xa7, 0x38, 0x87,
- 0x1a, 0xcb, 0x42, 0x00, 0x00,
+ // 5243 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5c, 0xcd, 0x6f, 0x23, 0xc9,
+ 0x75, 0x17, 0x49, 0xf1, 0xeb, 0x91, 0xa2, 0x7a, 0x4a, 0x9a, 0x59, 0x6d, 0x7b, 0xbd, 0x33, 0xdb,
+ 0xbb, 0xde, 0x1d, 0x6f, 0xd6, 0xdc, 0x91, 0xb4, 0xb3, 0x3b, 0x23, 0xdb, 0xb3, 0x26, 0xc5, 0xd6,
+ 0xa8, 0x67, 0xf8, 0xe5, 0x26, 0x25, 0xcd, 0xac, 0xed, 0xed, 0x94, 0xd8, 0x45, 0xa9, 0x31, 0xcd,
+ 0x6e, 0xba, 0xbb, 0xa9, 0x59, 0x1a, 0x0e, 0x02, 0x04, 0xc1, 0x1e, 0x72, 0x48, 0x90, 0x1c, 0x0c,
+ 0xe4, 0x14, 0xc0, 0x01, 0x82, 0x24, 0x08, 0x90, 0xc0, 0x4e, 0xf2, 0x07, 0x38, 0xc9, 0x31, 0x01,
+ 0x02, 0x04, 0x08, 0x90, 0x5b, 0xfe, 0x80, 0x5c, 0x02, 0xf8, 0x90, 0x9c, 0x82, 0xfa, 0xe8, 0x66,
+ 0x93, 0x92, 0x66, 0x49, 0xcd, 0x20, 0x37, 0xf6, 0xeb, 0x7a, 0xbf, 0x57, 0xf5, 0xaa, 0xea, 0xd5,
+ 0xab, 0xf7, 0x5e, 0x13, 0xae, 0x1f, 0x13, 0x3c, 0x30, 0xbc, 0x91, 0xe3, 0x10, 0xcf, 0xc0, 0x43,
+ 0xab, 0x3c, 0xf4, 0xdc, 0xc0, 0x45, 0x6f, 0xb9, 0xde, 0x49, 0x19, 0x0f, 0x71, 0xef, 0x94, 0x94,
+ 0x69, 0x8b, 0xf2, 0xc0, 0x35, 0x89, 0x5d, 0x1e, 0x5a, 0x43, 0x62, 0x5b, 0x0e, 0x29, 0x9f, 0x6d,
+ 0xca, 0xab, 0xc4, 0x31, 0x87, 0xae, 0xe5, 0x04, 0x3e, 0xe7, 0x91, 0x5f, 0x3f, 0x71, 0xdd, 0x13,
+ 0x9b, 0x7c, 0xc8, 0x9e, 0x8e, 0x47, 0xfd, 0x0f, 0xb1, 0x33, 0x16, 0xaf, 0x6e, 0xcd, 0xbe, 0x32,
+ 0x89, 0xdf, 0xf3, 0xac, 0x61, 0xe0, 0x7a, 0xa2, 0xc5, 0xcd, 0xd9, 0x16, 0x81, 0x35, 0x20, 0x7e,
+ 0x80, 0x07, 0x43, 0xde, 0x40, 0xf9, 0x55, 0x02, 0x56, 0xaa, 0x04, 0x0f, 0x76, 0x5d, 0xc7, 0x0f,
+ 0xb0, 0x13, 0xf8, 0xca, 0xdf, 0x24, 0x20, 0x1f, 0x3d, 0xa1, 0x4d, 0x58, 0x6f, 0x68, 0x4d, 0xa3,
+ 0xab, 0x35, 0xd4, 0x4e, 0xb7, 0xd2, 0x68, 0x1b, 0x0d, 0xad, 0x5e, 0xd7, 0x3a, 0xd2, 0x92, 0xfc,
+ 0xda, 0x5f, 0xfe, 0xed, 0xff, 0xfe, 0x2a, 0x7d, 0xed, 0x5b, 0xf7, 0xb7, 0xb6, 0xb6, 0xb7, 0x3f,
+ 0xd9, 0xba, 0xb3, 0xfd, 0xf1, 0xbd, 0xbb, 0x1f, 0x7d, 0xf2, 0xc9, 0x5d, 0x74, 0x07, 0xd6, 0x1b,
+ 0x95, 0x27, 0xe7, 0x59, 0x12, 0xf2, 0x0d, 0xc6, 0x22, 0x9d, 0xe3, 0x78, 0x00, 0xca, 0xc3, 0x7a,
+ 0xab, 0x5a, 0xa9, 0x1b, 0x47, 0x5a, 0xb3, 0xd6, 0x3a, 0x32, 0x2e, 0xe4, 0x4f, 0x4e, 0xf3, 0x6f,
+ 0xde, 0xbf, 0x7b, 0xe7, 0x23, 0xc6, 0xaf, 0xfc, 0x7d, 0x0e, 0x60, 0xd7, 0x1d, 0x0c, 0x5d, 0x87,
+ 0xd0, 0x3e, 0xff, 0x08, 0x20, 0xf0, 0xb0, 0xe3, 0xf7, 0x5d, 0x6f, 0xe0, 0x6f, 0x24, 0x6e, 0xa5,
+ 0x6e, 0x17, 0xb6, 0xbe, 0x5b, 0xfe, 0x4a, 0xd5, 0x97, 0x27, 0x10, 0xe5, 0x6e, 0xc4, 0xaf, 0x3a,
+ 0x81, 0x37, 0xd6, 0x63, 0x80, 0xa8, 0x07, 0xc5, 0x61, 0xcf, 0xb5, 0x6d, 0xd2, 0x0b, 0x2c, 0xd7,
+ 0xf1, 0x37, 0x92, 0x4c, 0xc0, 0xa7, 0x8b, 0x09, 0x68, 0xc7, 0x10, 0xb8, 0x88, 0x29, 0x50, 0x34,
+ 0x86, 0xf5, 0xe7, 0x96, 0x63, 0xba, 0xcf, 0x2d, 0xe7, 0xc4, 0xf0, 0x03, 0x0f, 0x07, 0xe4, 0xc4,
+ 0x22, 0xfe, 0x46, 0x8a, 0x09, 0xdb, 0x5b, 0x4c, 0xd8, 0x51, 0x88, 0xd4, 0x89, 0x80, 0xb8, 0xcc,
+ 0xb5, 0xe7, 0xe7, 0xdf, 0xa0, 0xef, 0x43, 0xa6, 0xe7, 0x9a, 0xc4, 0xf3, 0x37, 0x96, 0x99, 0xb0,
+ 0xfb, 0x8b, 0x09, 0xdb, 0x65, 0xbc, 0x1c, 0x5f, 0x00, 0x51, 0x95, 0x11, 0xe7, 0xcc, 0xf2, 0x5c,
+ 0x67, 0x40, 0xdb, 0x6c, 0xa4, 0xaf, 0xa2, 0x32, 0x35, 0x86, 0x20, 0x54, 0x16, 0x07, 0x95, 0x6d,
+ 0x58, 0x9d, 0x99, 0x36, 0x24, 0x41, 0xea, 0x19, 0x19, 0x6f, 0x24, 0x6e, 0x25, 0x6e, 0xe7, 0x75,
+ 0xfa, 0x13, 0xed, 0x42, 0xfa, 0x0c, 0xdb, 0x23, 0xb2, 0x91, 0xbc, 0x95, 0xb8, 0x5d, 0xd8, 0xfa,
+ 0xd6, 0x1c, 0x5d, 0x68, 0x47, 0xa8, 0x3a, 0xe7, 0xdd, 0x49, 0xde, 0x4b, 0xc8, 0x2e, 0x5c, 0x3b,
+ 0x37, 0x87, 0x17, 0xc8, 0xab, 0x4d, 0xcb, 0x2b, 0xcf, 0x23, 0x6f, 0x37, 0x82, 0x8d, 0x0b, 0xfc,
+ 0x29, 0x6c, 0x5c, 0x36, 0x8f, 0x17, 0xc8, 0x7d, 0x34, 0x2d, 0xf7, 0xa3, 0x39, 0xe4, 0xce, 0xa2,
+ 0x8f, 0xe3, 0xd2, 0x7b, 0x50, 0x88, 0x4d, 0xec, 0x05, 0x02, 0x1f, 0x4c, 0x0b, 0xbc, 0x3d, 0xd7,
+ 0xdc, 0x9a, 0xc4, 0x9b, 0xd1, 0xe9, 0xb9, 0x49, 0x7e, 0x35, 0x3a, 0x8d, 0xc1, 0xc6, 0x04, 0x2a,
+ 0xff, 0x91, 0x80, 0x5c, 0x5b, 0x34, 0x43, 0x0d, 0x80, 0x5e, 0xb4, 0xda, 0x98, 0xbc, 0xf9, 0xd6,
+ 0xc7, 0x64, 0x89, 0xea, 0x31, 0x00, 0xf4, 0x01, 0x20, 0xcf, 0x75, 0x03, 0x23, 0xb2, 0x1c, 0x86,
+ 0x65, 0x72, 0x63, 0x91, 0xd7, 0x25, 0xfa, 0x26, 0x5a, 0x56, 0x9a, 0x49, 0x37, 0x5d, 0xd1, 0xb4,
+ 0xfc, 0xa1, 0x8d, 0xc7, 0x86, 0x89, 0x03, 0xbc, 0x91, 0x9a, 0x7b, 0x68, 0x35, 0xce, 0x56, 0xc3,
+ 0x01, 0xd6, 0x0b, 0xe6, 0xe4, 0x41, 0xf9, 0xc5, 0x32, 0xc0, 0x64, 0xed, 0xa2, 0x9b, 0x50, 0x18,
+ 0x39, 0xd6, 0x8f, 0x47, 0xc4, 0x70, 0xf0, 0x80, 0x6c, 0xa4, 0x99, 0x3e, 0x81, 0x93, 0x9a, 0x78,
+ 0x40, 0xd0, 0x2e, 0x2c, 0xfb, 0x43, 0xd2, 0x13, 0x23, 0xff, 0x70, 0x0e, 0xd1, 0x7b, 0x23, 0x87,
+ 0x2d, 0xd3, 0xce, 0x90, 0xf4, 0x74, 0xc6, 0x8c, 0xde, 0x81, 0x15, 0x7f, 0x74, 0x1c, 0x33, 0xbf,
+ 0x7c, 0xc0, 0xd3, 0x44, 0x6a, 0x62, 0x2c, 0x67, 0x38, 0x0a, 0x42, 0x7b, 0x76, 0x7f, 0xa1, 0x6d,
+ 0x58, 0xd6, 0x18, 0xaf, 0x30, 0x31, 0x1c, 0x08, 0x75, 0x21, 0xeb, 0x8e, 0x02, 0x86, 0xc9, 0xcd,
+ 0xd6, 0xce, 0x62, 0x98, 0x2d, 0xce, 0xcc, 0x41, 0x43, 0xa8, 0x73, 0xd3, 0x92, 0x79, 0xe9, 0x69,
+ 0x41, 0xdf, 0x80, 0x52, 0xcc, 0x6c, 0x19, 0x96, 0xb9, 0x91, 0x65, 0x53, 0xb1, 0x12, 0xa3, 0x6a,
+ 0xa6, 0x7c, 0x1f, 0x0a, 0xb1, 0x61, 0x5e, 0xb0, 0x0b, 0xd6, 0xe3, 0xbb, 0x20, 0x1f, 0xdf, 0x46,
+ 0x3b, 0x50, 0x8c, 0x8f, 0x66, 0x11, 0x5e, 0xe5, 0xef, 0x56, 0x60, 0xad, 0x13, 0x60, 0xc7, 0xc4,
+ 0x9e, 0x39, 0xd1, 0x8e, 0xaf, 0xfc, 0x45, 0x0a, 0xa0, 0xed, 0x59, 0x03, 0x2b, 0xb0, 0xce, 0x88,
+ 0x8f, 0xbe, 0x09, 0x99, 0x76, 0x45, 0x37, 0x6a, 0x2d, 0x69, 0x49, 0xfe, 0xfa, 0xcf, 0xe9, 0xa9,
+ 0xfc, 0x1a, 0xd5, 0xc3, 0x4e, 0x34, 0xc7, 0x3b, 0x43, 0xec, 0x99, 0xee, 0xce, 0xd9, 0x26, 0xfa,
+ 0x00, 0xb2, 0x7b, 0xf5, 0x4a, 0xb7, 0xab, 0x36, 0xa5, 0x84, 0x7c, 0x93, 0xb5, 0x7d, 0x7d, 0xa6,
+ 0x6d, 0xdf, 0xc6, 0x41, 0x40, 0x1c, 0xda, 0xfa, 0x63, 0x28, 0x3e, 0xd4, 0x5b, 0x07, 0x6d, 0xa3,
+ 0xfa, 0xd4, 0x78, 0xac, 0x3e, 0x95, 0x92, 0xf2, 0x3b, 0x8c, 0xe5, 0xcd, 0x19, 0x96, 0x13, 0xcf,
+ 0x1d, 0x0d, 0x8d, 0xe3, 0xb1, 0xf1, 0x8c, 0x8c, 0x85, 0x14, 0xad, 0xd1, 0x3e, 0xa8, 0x77, 0x54,
+ 0x29, 0x75, 0x89, 0x14, 0x6b, 0x30, 0x1c, 0xd9, 0x3e, 0xa1, 0xad, 0x3f, 0x81, 0x52, 0xa5, 0xd3,
+ 0xd1, 0x1e, 0x36, 0x85, 0xc3, 0xd1, 0x91, 0x96, 0xe5, 0xb7, 0x19, 0xd3, 0xd7, 0x67, 0x98, 0xf8,
+ 0x01, 0x69, 0x58, 0x4e, 0xc0, 0x06, 0xb3, 0x0d, 0x85, 0xae, 0xda, 0xe9, 0x1a, 0x9d, 0xae, 0xae,
+ 0x56, 0x1a, 0x52, 0x5a, 0x56, 0x18, 0xd7, 0x1b, 0x33, 0x5c, 0x01, 0xf1, 0x03, 0x3f, 0xf0, 0x28,
+ 0xf1, 0x6c, 0x13, 0x7d, 0x04, 0x85, 0x46, 0xa5, 0x1d, 0x89, 0xca, 0x5c, 0x22, 0x6a, 0x80, 0x87,
+ 0x06, 0x17, 0xe7, 0x53, 0xae, 0x7b, 0xb0, 0xd2, 0x50, 0xf5, 0x87, 0x6a, 0xc4, 0x97, 0x95, 0xbf,
+ 0xc1, 0xf8, 0x6e, 0xce, 0xf2, 0x11, 0xef, 0x84, 0xc4, 0x38, 0x95, 0x00, 0xd6, 0x6b, 0x64, 0xe8,
+ 0x91, 0x1e, 0x0e, 0x88, 0x19, 0x9b, 0xb4, 0x77, 0x61, 0x59, 0x57, 0x2b, 0x35, 0x69, 0x49, 0x7e,
+ 0x83, 0x01, 0xdd, 0x98, 0x01, 0xf2, 0x08, 0x36, 0x45, 0x7f, 0x77, 0x75, 0xb5, 0xd2, 0x55, 0x8d,
+ 0x43, 0x4d, 0x3d, 0x92, 0x12, 0x97, 0xf4, 0xb7, 0xe7, 0x11, 0x1c, 0x10, 0xe3, 0xcc, 0x22, 0xcf,
+ 0xa9, 0xd4, 0xff, 0x4e, 0x08, 0x27, 0xcc, 0xb7, 0x02, 0xe2, 0xa3, 0xef, 0xc0, 0xea, 0x6e, 0xab,
+ 0x51, 0xd5, 0x9a, 0xaa, 0xd1, 0x56, 0x75, 0x36, 0x97, 0x4b, 0xf2, 0x7b, 0x0c, 0xe8, 0xad, 0x59,
+ 0x20, 0x77, 0x70, 0x6c, 0x39, 0xc4, 0x18, 0x12, 0x2f, 0x9c, 0xce, 0x07, 0x20, 0x85, 0xdc, 0xdc,
+ 0x33, 0xac, 0x3f, 0x95, 0x12, 0xf2, 0x6d, 0xc6, 0xae, 0x5c, 0xc2, 0x7e, 0x62, 0xbb, 0xc7, 0xd8,
+ 0xb6, 0x19, 0xff, 0x1d, 0xc8, 0xeb, 0x6a, 0x67, 0xff, 0x60, 0x6f, 0xaf, 0xae, 0x4a, 0x49, 0xf9,
+ 0x2d, 0xc6, 0xf8, 0xb5, 0x73, 0xe3, 0xf5, 0x4f, 0x47, 0xfd, 0xbe, 0x4d, 0xc4, 0xa0, 0x8f, 0x74,
+ 0xad, 0xab, 0x1a, 0x7b, 0x5a, 0x5d, 0xed, 0x48, 0xa9, 0xcb, 0xd6, 0x83, 0x67, 0x05, 0xc4, 0xe8,
+ 0x5b, 0x36, 0x61, 0xaa, 0xfe, 0x75, 0x12, 0xae, 0xed, 0x72, 0xf9, 0x31, 0x07, 0x54, 0x07, 0x79,
+ 0x66, 0xec, 0x46, 0x5b, 0x57, 0x05, 0x49, 0x5a, 0x92, 0xb7, 0x18, 0xf4, 0x07, 0x2f, 0x56, 0x83,
+ 0x41, 0x67, 0x90, 0x93, 0x68, 0xff, 0x8e, 0x41, 0x99, 0xc5, 0xe4, 0xcb, 0xa3, 0xb2, 0xbb, 0x7b,
+ 0xd0, 0x38, 0xa8, 0x57, 0xba, 0x2d, 0x9d, 0xfa, 0xd8, 0x3b, 0x0c, 0xfb, 0xa3, 0xaf, 0xc0, 0xe6,
+ 0x6b, 0x06, 0xf7, 0x7a, 0xa3, 0xc1, 0xc8, 0xc6, 0x81, 0xeb, 0xb1, 0x25, 0xf7, 0x43, 0xb8, 0x39,
+ 0x2b, 0x43, 0x7d, 0xd2, 0xd5, 0x2b, 0xbb, 0x5d, 0xa3, 0x75, 0xd0, 0x6d, 0x1f, 0x74, 0xa9, 0x13,
+ 0xfe, 0x09, 0x13, 0xb0, 0xf9, 0x15, 0x02, 0xc8, 0x17, 0x81, 0x87, 0x7b, 0x81, 0x21, 0x0c, 0x29,
+ 0x45, 0x7f, 0x04, 0x37, 0xa2, 0x39, 0xa5, 0x5b, 0x5c, 0xad, 0x19, 0x87, 0x95, 0xfa, 0x01, 0x53,
+ 0x76, 0x99, 0x81, 0xde, 0xbe, 0x6c, 0x66, 0xe9, 0x66, 0x27, 0xa6, 0xc1, 0xcc, 0x14, 0xd3, 0xfb,
+ 0xef, 0x2f, 0xc3, 0xeb, 0x9d, 0xa1, 0x6d, 0x05, 0x01, 0x3e, 0xb6, 0x49, 0x1b, 0x7b, 0x35, 0x37,
+ 0xa6, 0xff, 0x3a, 0x5c, 0x6f, 0x57, 0x34, 0xdd, 0x38, 0xd2, 0xba, 0xfb, 0x86, 0xae, 0x76, 0xba,
+ 0xba, 0xb6, 0xdb, 0xd5, 0x5a, 0x4d, 0x69, 0x49, 0xde, 0x64, 0x82, 0x7e, 0x63, 0x46, 0x90, 0x6f,
+ 0xf6, 0x8d, 0x21, 0xb6, 0x3c, 0xe3, 0xb9, 0x15, 0x9c, 0x1a, 0x1e, 0xf1, 0x03, 0xcf, 0x62, 0x27,
+ 0x1b, 0xed, 0x77, 0x0d, 0xae, 0x75, 0xda, 0x75, 0xad, 0x3b, 0x85, 0x94, 0x90, 0xbf, 0xc5, 0x90,
+ 0xde, 0xbb, 0x00, 0xc9, 0xa7, 0x1d, 0x9b, 0x45, 0x69, 0xc2, 0x8d, 0xb6, 0xde, 0xda, 0x55, 0x3b,
+ 0x1d, 0xaa, 0x57, 0xb5, 0x66, 0xa8, 0x75, 0xb5, 0xa1, 0x36, 0x99, 0x4a, 0x2f, 0x5e, 0x0f, 0xac,
+ 0x53, 0x9e, 0xdb, 0x23, 0xbe, 0x4f, 0x55, 0x4a, 0x4c, 0x83, 0xd8, 0x84, 0x39, 0x46, 0x14, 0xaf,
+ 0x0a, 0x52, 0x88, 0x17, 0x21, 0xa5, 0xe4, 0x0f, 0x18, 0xd2, 0xbb, 0x2f, 0x40, 0x8a, 0x63, 0x3c,
+ 0x81, 0xaf, 0xf1, 0x91, 0x55, 0x9a, 0x35, 0xa3, 0xa3, 0x7d, 0xa6, 0xc6, 0x87, 0x48, 0x6d, 0xe2,
+ 0xc5, 0x73, 0x3d, 0x19, 0x23, 0x76, 0x4c, 0xc3, 0xb7, 0x7e, 0x42, 0xe2, 0x83, 0x65, 0xc8, 0x2e,
+ 0xbc, 0x17, 0xf6, 0x8e, 0xe2, 0x4e, 0x46, 0xcb, 0x44, 0x4d, 0x49, 0x49, 0xcb, 0x55, 0x26, 0xe5,
+ 0x3b, 0x2f, 0xe8, 0x34, 0x95, 0x11, 0x0d, 0x9f, 0x49, 0x9d, 0x11, 0xa8, 0xfc, 0x4e, 0x02, 0x6e,
+ 0x84, 0xe7, 0x56, 0xc7, 0x32, 0x09, 0x3b, 0x3b, 0xbb, 0xe3, 0x21, 0xf1, 0x95, 0x53, 0x58, 0x56,
+ 0x9d, 0xd1, 0x00, 0x7d, 0x08, 0x39, 0xad, 0xab, 0xea, 0x95, 0x6a, 0x9d, 0xee, 0xc1, 0xb8, 0x49,
+ 0xf0, 0x2d, 0x93, 0x18, 0xcc, 0x8f, 0xd8, 0xb1, 0x02, 0xe2, 0xd1, 0x25, 0x45, 0x07, 0xf1, 0x21,
+ 0xe4, 0x1a, 0x07, 0xf5, 0xae, 0xd6, 0xa8, 0xb4, 0xa5, 0xc4, 0x65, 0x0c, 0x83, 0x91, 0x1d, 0x58,
+ 0x03, 0x3c, 0xa4, 0x9d, 0xf8, 0x79, 0x12, 0x0a, 0x31, 0xef, 0x7d, 0xd6, 0xe5, 0x4a, 0x9c, 0x73,
+ 0xb9, 0x5e, 0x87, 0x1c, 0xbb, 0x21, 0x51, 0x2f, 0x80, 0x1f, 0xc5, 0x59, 0xf6, 0xac, 0x99, 0xa8,
+ 0x0d, 0x60, 0xf9, 0xc6, 0xb1, 0x3b, 0x72, 0x4c, 0x62, 0x32, 0x77, 0xb0, 0xb4, 0xb5, 0x39, 0x87,
+ 0xdf, 0xa1, 0xf9, 0x55, 0xce, 0x53, 0xa6, 0x83, 0xd6, 0xf3, 0x56, 0xf8, 0x8c, 0xb6, 0xe0, 0xfa,
+ 0xb9, 0x2b, 0xe5, 0x98, 0x4a, 0x5e, 0x66, 0x92, 0xcf, 0xdd, 0x05, 0xc7, 0x9a, 0x79, 0xce, 0xff,
+ 0x49, 0xbf, 0xbc, 0x5b, 0xfa, 0xb3, 0x2c, 0x14, 0xd9, 0x86, 0x6d, 0xe3, 0xb1, 0xed, 0x62, 0x93,
+ 0xba, 0xf3, 0xa6, 0x6b, 0xf4, 0x9d, 0x2b, 0x3b, 0x9e, 0xa6, 0xbb, 0xe7, 0xa0, 0x3a, 0xc0, 0x10,
+ 0x7b, 0x78, 0x40, 0x02, 0x7a, 0x73, 0xe5, 0x77, 0xf2, 0x0f, 0xe6, 0x71, 0x01, 0x43, 0x26, 0x3d,
+ 0xc6, 0x8f, 0x7e, 0x13, 0x0a, 0x93, 0x39, 0x0e, 0xbd, 0xd4, 0x4f, 0xe7, 0x83, 0x8b, 0x46, 0x56,
+ 0x8e, 0x16, 0x62, 0x18, 0x45, 0xf0, 0x23, 0x02, 0x93, 0x10, 0xd0, 0xf3, 0x93, 0xba, 0xcd, 0xa1,
+ 0xcf, 0xba, 0xb8, 0x04, 0x0a, 0x41, 0xb5, 0x10, 0x49, 0x88, 0x08, 0x54, 0x42, 0x60, 0x0d, 0x88,
+ 0x27, 0x24, 0xa4, 0xaf, 0x26, 0xa1, 0x4b, 0x21, 0xe2, 0x12, 0x82, 0x88, 0x80, 0xde, 0x04, 0xf0,
+ 0x23, 0x23, 0xcc, 0x7c, 0xe3, 0x9c, 0x1e, 0xa3, 0xa0, 0x3b, 0xb0, 0x1e, 0xdb, 0xa7, 0x46, 0xb4,
+ 0xd4, 0xb9, 0xc3, 0x8b, 0x62, 0xef, 0x76, 0xc5, 0xaa, 0xdf, 0x86, 0xeb, 0x1e, 0xf9, 0xf1, 0x88,
+ 0xba, 0x4f, 0x46, 0xdf, 0x72, 0xb0, 0x6d, 0xfd, 0x04, 0xd3, 0xf7, 0x1b, 0x39, 0x06, 0xbe, 0x1e,
+ 0xbe, 0xdc, 0x8b, 0xbd, 0x93, 0x9f, 0xc1, 0xea, 0x8c, 0xa6, 0x2f, 0x70, 0x79, 0xab, 0xd3, 0x97,
+ 0xc6, 0x79, 0x96, 0x46, 0x04, 0x1a, 0x77, 0xae, 0xa9, 0xb0, 0x69, 0xa5, 0xbf, 0x22, 0x61, 0x21,
+ 0xe8, 0x8c, 0xb0, 0x19, 0xfd, 0xbf, 0x1a, 0x61, 0x11, 0x68, 0xdc, 0xf5, 0xff, 0x65, 0x02, 0xf2,
+ 0xd1, 0x6e, 0x40, 0x8f, 0x60, 0x39, 0x18, 0x0f, 0xb9, 0xd1, 0x2a, 0x6d, 0x7d, 0xbc, 0xc8, 0x4e,
+ 0x2a, 0x53, 0xbb, 0xcb, 0xcd, 0x0f, 0xc3, 0x90, 0x3f, 0x83, 0x65, 0x4a, 0x52, 0x74, 0x61, 0x89,
+ 0x57, 0xa1, 0x70, 0xd0, 0xec, 0xb4, 0xd5, 0x5d, 0x6d, 0x4f, 0x53, 0x6b, 0xd2, 0x12, 0x02, 0xc8,
+ 0x70, 0x2f, 0x57, 0x4a, 0xa0, 0x75, 0x90, 0xda, 0x5a, 0x5b, 0xad, 0x53, 0x3f, 0xa1, 0xd5, 0xe6,
+ 0x67, 0x44, 0x12, 0xbd, 0x06, 0x6b, 0xb1, 0x53, 0xc3, 0xa0, 0x4e, 0xc9, 0x63, 0x55, 0x97, 0x52,
+ 0xca, 0xbf, 0xa6, 0x20, 0x1f, 0xe9, 0x0e, 0x79, 0x70, 0x83, 0x7a, 0xb1, 0xc6, 0xc0, 0x35, 0xad,
+ 0xfe, 0xd8, 0xe0, 0xde, 0x5a, 0xec, 0x56, 0xfb, 0xed, 0x39, 0xc6, 0xa1, 0x13, 0x6c, 0x36, 0x18,
+ 0xff, 0x11, 0x65, 0x8f, 0xc0, 0xf7, 0x97, 0xf4, 0x35, 0x6f, 0xe6, 0x1d, 0x95, 0x59, 0x87, 0xdc,
+ 0x31, 0x3e, 0xe1, 0x52, 0x92, 0x73, 0x9b, 0xb0, 0x2a, 0x3e, 0x89, 0x23, 0x67, 0x8f, 0xf1, 0x09,
+ 0x43, 0xfb, 0x1c, 0x4a, 0xdc, 0xed, 0x61, 0x56, 0x9a, 0x62, 0xf2, 0x50, 0xc0, 0xdd, 0xf9, 0x22,
+ 0x11, 0x9c, 0x31, 0x8e, 0xbc, 0x12, 0xc1, 0x85, 0xbd, 0xa5, 0x17, 0x0d, 0x86, 0xbc, 0x3c, 0x77,
+ 0x6f, 0x1b, 0x78, 0x38, 0xd5, 0xdb, 0x01, 0x1e, 0x86, 0x68, 0x3e, 0x09, 0x38, 0x5a, 0x7a, 0x6e,
+ 0xb4, 0x0e, 0x09, 0xa6, 0xd0, 0x7c, 0x12, 0xd0, 0x9f, 0xd5, 0x0c, 0x8f, 0x40, 0x28, 0x77, 0x61,
+ 0xe3, 0xb2, 0x49, 0x98, 0x3a, 0x32, 0x13, 0x53, 0x47, 0xa6, 0x72, 0x0f, 0x8a, 0x71, 0xad, 0xa2,
+ 0xdb, 0x20, 0x85, 0x2e, 0xc3, 0x0c, 0x4b, 0x49, 0xd0, 0x85, 0xd9, 0x51, 0x7e, 0x96, 0x00, 0x74,
+ 0x5e, 0x79, 0xd4, 0x7e, 0xc5, 0x5c, 0xe4, 0x59, 0x10, 0x14, 0x7b, 0x17, 0xda, 0xaf, 0x26, 0x8b,
+ 0x21, 0x31, 0xa7, 0xb5, 0xef, 0x2c, 0xb0, 0x1a, 0xa6, 0x0e, 0xb4, 0xbc, 0x80, 0xd8, 0x73, 0x94,
+ 0x43, 0x28, 0xc6, 0x55, 0x8f, 0x6e, 0x41, 0x91, 0x7a, 0xd7, 0x33, 0x3d, 0x81, 0x67, 0x64, 0x1c,
+ 0xf6, 0xe0, 0x1d, 0x28, 0xb1, 0x2d, 0x6d, 0xcc, 0x38, 0x16, 0x45, 0x46, 0xdd, 0x9d, 0xa8, 0x2a,
+ 0x3e, 0x09, 0x0b, 0xa8, 0xea, 0xcb, 0x04, 0xe4, 0x23, 0xf3, 0x81, 0x3a, 0xfc, 0x8c, 0x31, 0x4c,
+ 0x77, 0x80, 0x2d, 0x47, 0x18, 0x8b, 0xad, 0x39, 0x2d, 0x50, 0x8d, 0x31, 0x71, 0x43, 0xc1, 0x8e,
+ 0x15, 0x4e, 0xa0, 0x43, 0xe0, 0x07, 0xd7, 0xec, 0x10, 0x18, 0x35, 0xec, 0xc8, 0xf7, 0x20, 0x1f,
+ 0xf9, 0x3a, 0xca, 0xf6, 0x65, 0x96, 0x65, 0x05, 0xf2, 0x07, 0xcd, 0x6a, 0xeb, 0xa0, 0x59, 0x53,
+ 0x6b, 0x52, 0x02, 0x15, 0x20, 0x1b, 0x3e, 0x24, 0x95, 0x3f, 0x4f, 0x40, 0x81, 0xae, 0xb3, 0xd0,
+ 0x11, 0x79, 0x08, 0x19, 0xdf, 0x1d, 0x79, 0x3d, 0x72, 0x55, 0x4f, 0x44, 0xb0, 0xcf, 0xf8, 0x6e,
+ 0xc9, 0x97, 0xf7, 0xdd, 0x14, 0x0c, 0xd7, 0x78, 0x78, 0x56, 0x73, 0x82, 0xc8, 0x71, 0xaa, 0x43,
+ 0x5e, 0x84, 0x27, 0xae, 0xee, 0x3c, 0xe5, 0x38, 0xc2, 0x9e, 0xa3, 0xfc, 0x51, 0x02, 0x4a, 0xe2,
+ 0x2a, 0x1b, 0x0a, 0x98, 0x5e, 0xcd, 0x89, 0x97, 0x5d, 0xcd, 0x97, 0xee, 0xa7, 0xe4, 0x65, 0xfb,
+ 0x49, 0xf9, 0xe7, 0x2c, 0x5c, 0xeb, 0x12, 0x3f, 0xe8, 0xb0, 0x60, 0x4a, 0xd8, 0xaf, 0xcb, 0x6d,
+ 0x00, 0xd2, 0x21, 0x43, 0xce, 0x58, 0x00, 0x37, 0x39, 0x77, 0x14, 0xf0, 0x9c, 0x80, 0xb2, 0x4a,
+ 0x21, 0x74, 0x81, 0x84, 0x3a, 0x90, 0x0b, 0x93, 0x72, 0xc2, 0x18, 0x7f, 0x32, 0x07, 0x6a, 0x65,
+ 0x68, 0x75, 0x88, 0x77, 0x66, 0xf5, 0x48, 0x2d, 0xca, 0xca, 0xe9, 0x11, 0x90, 0xfc, 0xb3, 0x34,
+ 0xa4, 0x99, 0x18, 0x74, 0x06, 0xab, 0xcf, 0x71, 0x40, 0xbc, 0x01, 0xf6, 0x9e, 0x19, 0x4c, 0xa4,
+ 0x50, 0xf5, 0xe3, 0xab, 0xf7, 0xbd, 0x5c, 0x31, 0xcf, 0xb0, 0xd3, 0x23, 0x47, 0x21, 0xf0, 0xfe,
+ 0x92, 0x5e, 0x8a, 0xa4, 0x70, 0xb9, 0x5f, 0x26, 0xe0, 0xba, 0xb8, 0x60, 0xd1, 0xb3, 0x86, 0xed,
+ 0x63, 0x2e, 0x9e, 0xdb, 0xad, 0xf6, 0xcb, 0x8b, 0x6f, 0x47, 0xf0, 0x74, 0xbf, 0xd3, 0x03, 0x74,
+ 0x38, 0x45, 0xe1, 0x1d, 0x19, 0xc0, 0x4a, 0x68, 0x7c, 0xb8, 0x7c, 0xae, 0xe4, 0xbd, 0x97, 0x92,
+ 0x6f, 0xaa, 0xe2, 0xa2, 0xbb, 0xbf, 0xa4, 0x17, 0x05, 0x3c, 0x7b, 0x27, 0x6b, 0x20, 0xcd, 0x6a,
+ 0x07, 0xbd, 0x0d, 0x2b, 0x0e, 0x79, 0x6e, 0x44, 0x1a, 0x62, 0x33, 0x90, 0xd2, 0x8b, 0x0e, 0x79,
+ 0x3e, 0x69, 0x24, 0x41, 0x2a, 0xc0, 0x27, 0x62, 0xb5, 0xd2, 0x9f, 0x72, 0x15, 0xae, 0x5f, 0x38,
+ 0x52, 0xf4, 0x4d, 0x90, 0x30, 0x7f, 0x61, 0x98, 0x23, 0x8f, 0xbb, 0xb0, 0x1c, 0x72, 0x55, 0xd0,
+ 0x6b, 0x82, 0x2c, 0xff, 0x6e, 0x02, 0x0a, 0xb1, 0xee, 0xa2, 0x1e, 0xe4, 0xc2, 0x3b, 0xba, 0xc8,
+ 0x5d, 0x3e, 0xbc, 0x92, 0x22, 0xba, 0x61, 0xa6, 0x97, 0x84, 0xd8, 0x7a, 0x04, 0x1c, 0x0e, 0x25,
+ 0x15, 0x0d, 0xa5, 0x9a, 0x85, 0x34, 0x53, 0xbe, 0xfc, 0x03, 0x40, 0xe7, 0x59, 0xd1, 0x7b, 0xb0,
+ 0x4a, 0x1c, 0xba, 0xc9, 0xa2, 0x6b, 0x38, 0x1b, 0x4f, 0x51, 0x2f, 0x09, 0x72, 0xd8, 0xf0, 0x0d,
+ 0xc8, 0x47, 0x39, 0x66, 0xa6, 0xaa, 0x94, 0x3e, 0x21, 0x28, 0xab, 0xb0, 0xc2, 0x26, 0xc1, 0xd7,
+ 0xb9, 0x23, 0xaf, 0xfc, 0x57, 0x0a, 0xae, 0xb1, 0x13, 0x7e, 0xcf, 0xb2, 0x89, 0x1f, 0x6e, 0xf0,
+ 0x5d, 0x58, 0xf6, 0x2d, 0xe7, 0xd9, 0xd5, 0x53, 0x11, 0x96, 0xf3, 0x0c, 0x3d, 0x81, 0xd5, 0xbe,
+ 0xeb, 0x0d, 0x70, 0x60, 0xf4, 0xc5, 0xcb, 0xab, 0x1e, 0xc8, 0x25, 0x8e, 0x13, 0xd2, 0xa8, 0x32,
+ 0xb8, 0xd9, 0x24, 0x26, 0x77, 0x31, 0x7d, 0xa6, 0xc9, 0x9c, 0x5e, 0x0a, 0xc9, 0x6c, 0x48, 0x3e,
+ 0xfa, 0x0e, 0xc8, 0xa2, 0x06, 0xc0, 0xa4, 0x8e, 0xf1, 0xc0, 0x72, 0x88, 0x69, 0xf8, 0xa7, 0xd8,
+ 0x33, 0x2d, 0xe7, 0x84, 0xb9, 0x5f, 0x39, 0x7d, 0x83, 0xb7, 0xa8, 0x45, 0x0d, 0x3a, 0xe2, 0x3d,
+ 0x22, 0xd3, 0x97, 0x50, 0x7e, 0x81, 0xab, 0xcd, 0x93, 0xc9, 0x9b, 0x55, 0xe8, 0x8b, 0x6e, 0xa2,
+ 0xff, 0xaf, 0xd7, 0x27, 0xe5, 0xa7, 0x90, 0x66, 0xb6, 0xfd, 0xd5, 0x64, 0x9b, 0xca, 0xb0, 0x16,
+ 0x65, 0xdc, 0xa2, 0xe3, 0x24, 0xcc, 0x39, 0x5d, 0x8b, 0x5e, 0x89, 0xd3, 0xc4, 0x57, 0xfe, 0x33,
+ 0x0d, 0xa5, 0x30, 0x4a, 0xc4, 0xd3, 0x99, 0xca, 0xbf, 0xa4, 0x85, 0xeb, 0xf0, 0x0e, 0xa4, 0xab,
+ 0x4f, 0xbb, 0x6a, 0x47, 0x5a, 0x92, 0x5f, 0x67, 0xa1, 0x9e, 0x35, 0x16, 0xea, 0x61, 0xa8, 0x3b,
+ 0xc7, 0xe3, 0x80, 0x05, 0x1e, 0xd1, 0x1d, 0x28, 0xd0, 0x5b, 0x48, 0xf3, 0xa1, 0x71, 0xd0, 0xdd,
+ 0xbb, 0x27, 0xc1, 0x54, 0xae, 0x81, 0xb7, 0xa5, 0x97, 0x5a, 0xe7, 0xc4, 0x18, 0x05, 0xfd, 0x7b,
+ 0x94, 0xe3, 0x4d, 0x48, 0x3e, 0x3e, 0x94, 0x12, 0xf2, 0x0d, 0xd6, 0x50, 0x8a, 0x35, 0x7c, 0x76,
+ 0x46, 0xdf, 0x2b, 0xb0, 0x5c, 0x6d, 0xb5, 0xea, 0x52, 0x51, 0xde, 0x60, 0x2d, 0x50, 0x5c, 0xac,
+ 0xeb, 0xda, 0xb4, 0xcd, 0xbb, 0x90, 0x39, 0xac, 0xe8, 0x5a, 0xb3, 0x2b, 0x25, 0x65, 0x99, 0xb5,
+ 0x5a, 0x8f, 0xb5, 0x3a, 0xc3, 0x9e, 0xe5, 0x04, 0xa2, 0x5d, 0xad, 0x75, 0x50, 0xad, 0xab, 0x52,
+ 0xe1, 0x82, 0x76, 0xa6, 0x3b, 0x12, 0x91, 0xad, 0xf7, 0x63, 0xa1, 0xb0, 0xd4, 0x54, 0x36, 0x80,
+ 0xb7, 0x8c, 0x47, 0xc1, 0xde, 0x81, 0x74, 0x57, 0x6b, 0xa8, 0xba, 0xb4, 0x7c, 0x81, 0x5e, 0x98,
+ 0x47, 0xc6, 0xb3, 0x15, 0xab, 0x5a, 0xb3, 0xab, 0xea, 0x87, 0x51, 0x11, 0x87, 0x94, 0x9e, 0x0a,
+ 0xa1, 0x0b, 0x60, 0x27, 0x20, 0xde, 0x19, 0xb6, 0x45, 0xba, 0x82, 0x07, 0xde, 0x57, 0xea, 0x6a,
+ 0xf3, 0x61, 0x77, 0xdf, 0x68, 0xeb, 0xea, 0x9e, 0xf6, 0x44, 0xca, 0x4c, 0x85, 0xda, 0x38, 0x9f,
+ 0x4d, 0x9c, 0x93, 0xe0, 0xd4, 0x18, 0x7a, 0xa4, 0x6f, 0x7d, 0x21, 0xb8, 0xa6, 0x4a, 0x46, 0xa4,
+ 0xec, 0x05, 0x5c, 0x3c, 0x23, 0x10, 0x93, 0xf5, 0x31, 0x94, 0x78, 0xf3, 0x30, 0xf6, 0x2c, 0xe5,
+ 0xa6, 0x32, 0x38, 0x9c, 0x2d, 0xda, 0xdb, 0x7c, 0xd9, 0xb2, 0x60, 0xeb, 0x7a, 0xbb, 0xa2, 0x57,
+ 0x1a, 0xc6, 0x0c, 0x77, 0x69, 0x2a, 0x25, 0xc1, 0xb9, 0x59, 0x14, 0xc9, 0x38, 0x8f, 0x51, 0x83,
+ 0xeb, 0x9d, 0x6e, 0xa5, 0xab, 0x1a, 0x55, 0x7a, 0x75, 0xad, 0x19, 0xd1, 0x04, 0xe4, 0xe5, 0x6f,
+ 0x32, 0x90, 0xb7, 0xa7, 0xd6, 0x10, 0x0e, 0x88, 0x71, 0x8c, 0x7b, 0xcf, 0x88, 0x69, 0xc4, 0x67,
+ 0xe3, 0x16, 0xa4, 0xf4, 0xd6, 0x91, 0xb4, 0x22, 0xbf, 0xc6, 0x78, 0xae, 0xc5, 0x78, 0x3c, 0x36,
+ 0x46, 0xe5, 0xaf, 0x32, 0xa1, 0xaf, 0x18, 0x0b, 0xe5, 0xbd, 0x5a, 0x5f, 0x11, 0x1d, 0x42, 0x91,
+ 0x67, 0x10, 0x68, 0x3f, 0x47, 0xbe, 0x70, 0x71, 0xb7, 0xe7, 0xb9, 0x48, 0x52, 0xb6, 0x0e, 0xe3,
+ 0xe2, 0x4e, 0x6e, 0x61, 0x30, 0xa1, 0xa0, 0x77, 0x43, 0xc3, 0x3a, 0xf1, 0x0d, 0xf9, 0x11, 0xb5,
+ 0xc2, 0xc9, 0xe1, 0x25, 0xa7, 0x06, 0xd9, 0xc0, 0xb3, 0x4e, 0x4e, 0x88, 0x27, 0xee, 0xb0, 0xef,
+ 0xcf, 0x73, 0x44, 0x72, 0x0e, 0x3d, 0x64, 0x45, 0x04, 0xae, 0x45, 0x2e, 0xa7, 0xe5, 0x3a, 0x06,
+ 0x65, 0x61, 0xb7, 0xd8, 0xd2, 0xd6, 0xbd, 0x79, 0x1c, 0xbc, 0x18, 0x6f, 0xc3, 0x35, 0x45, 0xc4,
+ 0x43, 0xc2, 0x33, 0x64, 0x7a, 0x47, 0xe2, 0x59, 0x10, 0xe6, 0x62, 0xb1, 0x30, 0xd9, 0x7c, 0x77,
+ 0x24, 0x9e, 0xc4, 0xa5, 0x87, 0xb1, 0xb8, 0x23, 0xb9, 0x11, 0x01, 0x1d, 0x83, 0xd4, 0xb3, 0x5d,
+ 0xe6, 0xb8, 0x1d, 0x93, 0x53, 0x7c, 0x66, 0xb9, 0x1e, 0x0b, 0xab, 0x95, 0xe6, 0xf2, 0x4d, 0x77,
+ 0x39, 0x6b, 0x55, 0x70, 0x72, 0xf8, 0xd5, 0xde, 0x34, 0x95, 0x39, 0x31, 0xb6, 0xcd, 0x56, 0xb1,
+ 0x8d, 0x03, 0xe2, 0x10, 0xdf, 0x67, 0x71, 0x38, 0xea, 0xc4, 0x70, 0x7a, 0x5d, 0x90, 0xd1, 0xe7,
+ 0x50, 0x6a, 0x39, 0xb4, 0x63, 0x21, 0xf3, 0x46, 0x7e, 0xee, 0xb8, 0xd1, 0x34, 0x23, 0xef, 0xcb,
+ 0x0c, 0x1a, 0xda, 0x84, 0xeb, 0xd8, 0xf7, 0xad, 0x13, 0xc7, 0x37, 0x02, 0xd7, 0x70, 0x9d, 0x30,
+ 0xdf, 0xb9, 0x01, 0xec, 0x0c, 0x45, 0xe2, 0x65, 0xd7, 0x6d, 0x39, 0x84, 0x2f, 0xfe, 0x0b, 0xf2,
+ 0xec, 0x85, 0x0b, 0xf2, 0xec, 0xca, 0x0f, 0xa1, 0x10, 0x5b, 0x93, 0x4a, 0xe3, 0xb2, 0x8b, 0xe4,
+ 0x2a, 0x14, 0x9a, 0xad, 0x26, 0xcb, 0xb9, 0x69, 0xcd, 0x87, 0x52, 0x82, 0x11, 0x54, 0xb5, 0xd6,
+ 0xe1, 0x69, 0x38, 0x29, 0x89, 0x10, 0x94, 0x2a, 0x75, 0x5d, 0xad, 0xd4, 0x44, 0x66, 0xae, 0x26,
+ 0xa5, 0x94, 0x1f, 0x81, 0x34, 0xbb, 0x4c, 0x14, 0xed, 0x32, 0x11, 0x25, 0x80, 0x9a, 0xd6, 0xd9,
+ 0xad, 0xe8, 0x35, 0x2e, 0x41, 0x82, 0x62, 0x94, 0xdc, 0xa3, 0x94, 0x24, 0x6d, 0xa1, 0xab, 0x2c,
+ 0x21, 0x47, 0x9f, 0x53, 0xca, 0xf7, 0x61, 0x75, 0x66, 0x2a, 0x95, 0x07, 0x2f, 0x18, 0x80, 0xda,
+ 0xd0, 0xba, 0x46, 0xa5, 0x7e, 0x54, 0x79, 0xda, 0xe1, 0x81, 0x36, 0x46, 0xd0, 0xf6, 0x8c, 0x66,
+ 0xab, 0xa9, 0x36, 0xda, 0xdd, 0xa7, 0x52, 0x52, 0x69, 0xcf, 0xce, 0xe4, 0x0b, 0x11, 0xf7, 0x34,
+ 0x5d, 0x9d, 0x42, 0x64, 0x84, 0x69, 0xc4, 0x63, 0x80, 0xc9, 0x4a, 0x56, 0xba, 0x97, 0xa1, 0x5d,
+ 0x83, 0x15, 0xb5, 0x59, 0x33, 0x5a, 0x7b, 0x46, 0x14, 0x0a, 0x44, 0x50, 0xaa, 0x57, 0x58, 0xbe,
+ 0x5d, 0x6b, 0x1a, 0xed, 0x4a, 0x93, 0x6a, 0x99, 0xf6, 0xba, 0xa2, 0xd7, 0xb5, 0x38, 0x35, 0xa5,
+ 0xd8, 0x00, 0x93, 0x88, 0x82, 0xf2, 0xf9, 0x0b, 0x34, 0xac, 0x1e, 0xaa, 0xcd, 0x2e, 0x2b, 0x2e,
+ 0x94, 0x12, 0x68, 0x0d, 0x56, 0x45, 0x9a, 0x8a, 0x9e, 0xe8, 0x8c, 0x98, 0x44, 0xb7, 0xe0, 0x8d,
+ 0xce, 0xd3, 0xe6, 0xee, 0xbe, 0xde, 0x6a, 0xb2, 0xd4, 0xd5, 0x6c, 0x8b, 0x94, 0xf2, 0xa7, 0x12,
+ 0x64, 0x85, 0x35, 0x41, 0x3a, 0xe4, 0x71, 0x3f, 0x20, 0x9e, 0x81, 0x6d, 0x5b, 0x18, 0xd6, 0xed,
+ 0xf9, 0x8d, 0x51, 0xb9, 0x42, 0x79, 0x2b, 0xb6, 0xbd, 0xbf, 0xa4, 0xe7, 0xb0, 0xf8, 0x1d, 0xc3,
+ 0x74, 0xc6, 0xc2, 0xe1, 0x5a, 0x1c, 0xd3, 0x19, 0x4f, 0x30, 0x9d, 0x31, 0x3a, 0x00, 0xe0, 0x98,
+ 0x04, 0xf7, 0x4e, 0xc5, 0x0d, 0xeb, 0xa3, 0x45, 0x41, 0x55, 0xdc, 0x3b, 0xdd, 0x5f, 0xd2, 0x79,
+ 0xef, 0xe8, 0x03, 0xb2, 0x61, 0x4d, 0xc0, 0x3a, 0xa6, 0xe1, 0xf6, 0xc3, 0x6d, 0xb9, 0x3c, 0x77,
+ 0xb4, 0x75, 0x1a, 0xdf, 0x31, 0x5b, 0x7d, 0xbe, 0x7f, 0xf7, 0x97, 0x74, 0x09, 0xcf, 0xd0, 0x50,
+ 0x00, 0xd7, 0xb9, 0xb4, 0x99, 0x7b, 0xab, 0x88, 0x3d, 0x3e, 0x58, 0x54, 0xde, 0xf9, 0xfb, 0x29,
+ 0x3e, 0x4f, 0x46, 0x7f, 0x9c, 0x00, 0x85, 0x8b, 0xf5, 0xc7, 0x4e, 0xef, 0xd4, 0x73, 0x1d, 0x96,
+ 0x8e, 0x9c, 0xed, 0x03, 0xaf, 0x0d, 0x7a, 0xb4, 0x68, 0x1f, 0x3a, 0x31, 0xcc, 0x73, 0xfd, 0xb9,
+ 0x89, 0x5f, 0xdc, 0x04, 0x3d, 0x86, 0x0c, 0xb6, 0x9f, 0xe3, 0xb1, 0xbf, 0x51, 0x64, 0xe2, 0x37,
+ 0x17, 0x11, 0xcf, 0x18, 0xf7, 0x97, 0x74, 0x01, 0x81, 0x9a, 0x90, 0x35, 0x49, 0x1f, 0x8f, 0xec,
+ 0x80, 0x9d, 0x25, 0x85, 0xf9, 0x22, 0x79, 0x02, 0xad, 0xc6, 0x39, 0xf7, 0x97, 0xf4, 0x10, 0x04,
+ 0x7d, 0x3e, 0xb9, 0xd8, 0xf7, 0xdc, 0x91, 0x13, 0xb0, 0xd3, 0x63, 0xbe, 0xe8, 0x49, 0x88, 0xaa,
+ 0x86, 0xd1, 0xc7, 0x91, 0x13, 0xc4, 0x6e, 0xf2, 0xec, 0x19, 0xed, 0x43, 0xda, 0x21, 0x67, 0x84,
+ 0x1f, 0x36, 0x85, 0xad, 0x3b, 0x0b, 0xe0, 0x36, 0x29, 0xdf, 0xfe, 0x92, 0xce, 0x01, 0xe8, 0xee,
+ 0x70, 0x3d, 0x9e, 0x71, 0xb2, 0xc7, 0xec, 0x50, 0x59, 0x6c, 0x77, 0xb4, 0xbc, 0x3d, 0xce, 0x4b,
+ 0x77, 0x87, 0x1b, 0x3e, 0xd0, 0xd9, 0xf1, 0xc8, 0x90, 0xe0, 0x80, 0x9d, 0x3d, 0x8b, 0xcd, 0x8e,
+ 0xce, 0x18, 0xe9, 0xec, 0x70, 0x08, 0xf9, 0x09, 0xe4, 0x42, 0x6b, 0x81, 0xea, 0x50, 0x60, 0x15,
+ 0x75, 0xac, 0x69, 0x18, 0x27, 0x58, 0xc4, 0x09, 0x8a, 0xb3, 0x4f, 0x90, 0x9d, 0xf1, 0x2b, 0x46,
+ 0x7e, 0x0a, 0xf9, 0xc8, 0x70, 0xbc, 0x62, 0xe8, 0x5f, 0x24, 0x40, 0x9a, 0x35, 0x1a, 0xa8, 0x05,
+ 0x2b, 0x04, 0x7b, 0xf6, 0xd8, 0xe8, 0x5b, 0xf4, 0x12, 0x16, 0x96, 0x71, 0x2e, 0x22, 0xa4, 0xc8,
+ 0x00, 0xf6, 0x38, 0x3f, 0x6a, 0x40, 0x91, 0xfa, 0x3e, 0x11, 0x5e, 0x72, 0x61, 0xbc, 0x02, 0xe5,
+ 0x17, 0x70, 0xf2, 0x6f, 0xc3, 0xda, 0x05, 0x86, 0x07, 0x9d, 0xc2, 0x7a, 0x14, 0x23, 0x31, 0xce,
+ 0xd5, 0xae, 0xdf, 0x9d, 0x33, 0x9e, 0xce, 0xd8, 0x27, 0xc5, 0xca, 0x6b, 0xc1, 0x39, 0x9a, 0x2f,
+ 0xbf, 0x05, 0x37, 0xbf, 0xc2, 0xea, 0xc8, 0x79, 0xc8, 0x8a, 0xbd, 0x2c, 0x6f, 0x43, 0x31, 0xbe,
+ 0x01, 0xd1, 0xdb, 0xb3, 0x1b, 0x9a, 0xaa, 0x37, 0x3d, 0xbd, 0x2b, 0xe5, 0x2c, 0xa4, 0xd9, 0xee,
+ 0x92, 0x73, 0x90, 0xe1, 0x26, 0x46, 0xfe, 0xc3, 0x04, 0xe4, 0xa3, 0x2d, 0x82, 0x1e, 0xc0, 0x72,
+ 0x94, 0x2d, 0x58, 0x4c, 0x97, 0x8c, 0x8f, 0x7a, 0xff, 0xe1, 0x4e, 0x5d, 0x7c, 0x3a, 0x42, 0x56,
+ 0xb9, 0x0b, 0x19, 0xbe, 0xc5, 0xd0, 0x23, 0x80, 0xc9, 0xc2, 0xba, 0x42, 0xaf, 0x62, 0xdc, 0xd5,
+ 0x7c, 0x74, 0x33, 0x51, 0xfe, 0x21, 0x19, 0x8b, 0xa4, 0x4d, 0xea, 0x70, 0x3b, 0x90, 0x36, 0x89,
+ 0x8d, 0xc7, 0x0b, 0x64, 0x24, 0xcf, 0xa3, 0x94, 0x6b, 0x14, 0x82, 0xda, 0x2f, 0x86, 0x85, 0x3e,
+ 0x83, 0x1c, 0xb6, 0xad, 0x13, 0xc7, 0x08, 0x5c, 0xa1, 0x93, 0xef, 0x5e, 0x0d, 0xb7, 0x42, 0x51,
+ 0xba, 0x2e, 0xb5, 0xe2, 0x98, 0xff, 0x94, 0xdf, 0x87, 0x34, 0x93, 0x86, 0xde, 0x82, 0x22, 0x93,
+ 0x66, 0x0c, 0x2c, 0xdb, 0xb6, 0x7c, 0x11, 0xd0, 0x2c, 0x30, 0x5a, 0x83, 0x91, 0xe4, 0xfb, 0x90,
+ 0x15, 0x08, 0xe8, 0x06, 0x64, 0x86, 0xc4, 0xb3, 0x5c, 0x7e, 0x85, 0x4b, 0xe9, 0xe2, 0x89, 0xd2,
+ 0xdd, 0x7e, 0xdf, 0x27, 0x01, 0x73, 0x12, 0x52, 0xba, 0x78, 0xaa, 0x5e, 0x87, 0xb5, 0x0b, 0xf6,
+ 0x80, 0xf2, 0x7b, 0x49, 0xc8, 0x47, 0x91, 0x24, 0x74, 0x08, 0x25, 0xdc, 0x63, 0x25, 0x41, 0x43,
+ 0x1c, 0x04, 0xc4, 0xbb, 0xf2, 0x5d, 0x76, 0x85, 0xc3, 0xb4, 0x39, 0x0a, 0xda, 0x87, 0xec, 0x99,
+ 0x45, 0x9e, 0xbf, 0x44, 0xd2, 0x2e, 0x43, 0xf9, 0xf7, 0x1c, 0xf4, 0x03, 0xb8, 0x26, 0xae, 0xb0,
+ 0x03, 0x3c, 0x1c, 0x52, 0xe7, 0xa0, 0xef, 0x08, 0x77, 0x6b, 0x61, 0x4c, 0x71, 0x19, 0x6e, 0x70,
+ 0xa0, 0x3d, 0x47, 0x79, 0x00, 0xd2, 0x91, 0xe5, 0xf1, 0x2c, 0x5e, 0x87, 0x04, 0x81, 0xe5, 0x9c,
+ 0x20, 0x09, 0x52, 0x23, 0xa1, 0x87, 0xbc, 0x4e, 0x7f, 0xa2, 0x0d, 0xc8, 0x0e, 0x79, 0xdc, 0x8f,
+ 0x0d, 0xa6, 0xa8, 0x87, 0x8f, 0xca, 0xa7, 0x50, 0x88, 0x55, 0xc2, 0x87, 0xac, 0xc9, 0x0b, 0x59,
+ 0x53, 0x53, 0xac, 0x8f, 0x96, 0x73, 0x09, 0x29, 0xa9, 0xfc, 0x49, 0x02, 0xd6, 0xc3, 0x00, 0x5a,
+ 0xbc, 0x54, 0x5f, 0xf9, 0x32, 0x01, 0xc5, 0x38, 0x01, 0xbd, 0x03, 0x99, 0x5a, 0x8b, 0xe5, 0xe8,
+ 0x97, 0xa6, 0x02, 0x5b, 0xc4, 0x39, 0xdb, 0x31, 0xdd, 0xde, 0x33, 0x1e, 0x36, 0x7a, 0x17, 0xb2,
+ 0xc2, 0xbd, 0x96, 0x12, 0x53, 0xe1, 0x25, 0xda, 0x4c, 0x38, 0x58, 0xb4, 0xdd, 0x6d, 0xc8, 0xa9,
+ 0x4f, 0xba, 0xaa, 0xde, 0xac, 0xd4, 0x67, 0x42, 0x60, 0xb4, 0x21, 0xf9, 0x82, 0x4e, 0x22, 0xb6,
+ 0x77, 0xce, 0x36, 0x95, 0x7b, 0xb0, 0x52, 0x63, 0xf0, 0x61, 0x2c, 0xf9, 0x3d, 0x58, 0xed, 0xb9,
+ 0x4e, 0x80, 0x2d, 0x87, 0x78, 0x86, 0x35, 0xc0, 0x27, 0x61, 0x21, 0x56, 0x29, 0x22, 0x6b, 0x94,
+ 0xaa, 0xfc, 0x5b, 0x02, 0x4a, 0xc2, 0x14, 0x86, 0xbc, 0x25, 0x48, 0xba, 0xbe, 0x68, 0x9e, 0x74,
+ 0x7d, 0x84, 0x60, 0x19, 0x7b, 0xbd, 0x53, 0xa1, 0x31, 0xf6, 0x9b, 0xaa, 0xac, 0xe7, 0x0e, 0x06,
+ 0xd8, 0x09, 0x63, 0x15, 0xe1, 0x23, 0xaa, 0x43, 0x8a, 0x38, 0x67, 0x8b, 0x94, 0xa3, 0x4f, 0x49,
+ 0x2f, 0xab, 0xce, 0x19, 0x8f, 0xd6, 0x52, 0x18, 0xf9, 0x63, 0xc8, 0x85, 0x84, 0x85, 0x2a, 0xba,
+ 0xff, 0x27, 0x01, 0xab, 0xaa, 0x50, 0x50, 0x38, 0xae, 0x78, 0x46, 0x2b, 0xf1, 0x8a, 0x32, 0x5a,
+ 0xe8, 0x10, 0x32, 0x2c, 0xf6, 0x15, 0xa6, 0xde, 0xe6, 0xf1, 0xc6, 0x67, 0x3a, 0xc6, 0x6b, 0x48,
+ 0xc2, 0xca, 0x7e, 0x8e, 0x26, 0xdf, 0x87, 0x42, 0x8c, 0xbc, 0xd0, 0xd8, 0x77, 0xa0, 0x18, 0xdf,
+ 0x50, 0x2f, 0xde, 0x2b, 0xd3, 0x0b, 0x5e, 0xf9, 0xf5, 0x32, 0x14, 0x62, 0x45, 0x6c, 0x48, 0x83,
+ 0xb4, 0x15, 0x90, 0xe8, 0x50, 0xde, 0x5e, 0xac, 0x06, 0xae, 0xac, 0x05, 0x64, 0xa0, 0x73, 0x04,
+ 0xb9, 0x0f, 0xa0, 0x99, 0xc4, 0x09, 0xac, 0xbe, 0x45, 0x3c, 0x6a, 0x56, 0xe3, 0xdf, 0x88, 0x88,
+ 0xde, 0x15, 0x82, 0xc9, 0xe7, 0x21, 0xf4, 0xdc, 0x9d, 0x34, 0x99, 0x6c, 0xd9, 0x09, 0xdf, 0x81,
+ 0xe7, 0x84, 0x8a, 0x49, 0x45, 0x8a, 0x91, 0x7f, 0x99, 0x84, 0x65, 0x2a, 0x17, 0x69, 0x90, 0x14,
+ 0xc0, 0xf3, 0x7d, 0x6b, 0x31, 0xd5, 0xf1, 0xa8, 0xa7, 0x7a, 0xd2, 0xa2, 0x8b, 0x9a, 0xd7, 0x05,
+ 0x25, 0xe7, 0x8e, 0x93, 0xc5, 0xc1, 0x66, 0x2a, 0x83, 0xd0, 0xfb, 0xe1, 0xd4, 0x71, 0x0b, 0xb9,
+ 0x5e, 0xe6, 0xdf, 0x2b, 0x96, 0xc3, 0xef, 0x15, 0xcb, 0x15, 0x27, 0xfc, 0x0c, 0x09, 0xdd, 0x85,
+ 0x82, 0x7f, 0xea, 0x7a, 0x01, 0x0f, 0xa9, 0x8a, 0x2b, 0xe6, 0xc5, 0x1c, 0xc0, 0x1a, 0x1e, 0x32,
+ 0xb6, 0x75, 0x48, 0xdb, 0xf8, 0x98, 0xd8, 0xe2, 0x8b, 0x17, 0xfe, 0x80, 0x5e, 0x87, 0x9c, 0x6d,
+ 0x39, 0xcf, 0x8c, 0x91, 0x67, 0xb3, 0x8b, 0x5b, 0x5e, 0xcf, 0xd2, 0xe7, 0x03, 0xcf, 0x96, 0x7f,
+ 0x4b, 0x54, 0x2b, 0x8d, 0x5e, 0x50, 0xad, 0xc4, 0x73, 0x00, 0xbc, 0xa0, 0x40, 0x6b, 0x76, 0xd5,
+ 0x87, 0xaa, 0x2e, 0x25, 0x51, 0x1e, 0xd2, 0x7b, 0xf5, 0x56, 0xa5, 0x2b, 0xa5, 0x78, 0xa1, 0x41,
+ 0xab, 0xae, 0x56, 0x9a, 0xd2, 0x32, 0x5a, 0x81, 0x7c, 0xf4, 0x35, 0xa3, 0x94, 0x46, 0x45, 0xc8,
+ 0xd5, 0x0e, 0xf4, 0x0a, 0xab, 0x23, 0xce, 0xa0, 0x12, 0xc0, 0xa3, 0xca, 0x61, 0xc5, 0xd8, 0xad,
+ 0x57, 0x3a, 0x1d, 0x29, 0xab, 0xfc, 0x7b, 0x16, 0xae, 0x37, 0x88, 0xef, 0xe3, 0x13, 0x72, 0x64,
+ 0x05, 0xa7, 0xb1, 0xb2, 0xe6, 0x57, 0xfc, 0x81, 0xd2, 0xf7, 0x20, 0xcd, 0xa2, 0xac, 0x8b, 0x7e,
+ 0xb1, 0x45, 0xbd, 0x0e, 0xc6, 0x88, 0x7e, 0x48, 0x4d, 0xab, 0xa8, 0xfb, 0x8e, 0x6d, 0xa2, 0xf9,
+ 0xee, 0x39, 0xd3, 0xb5, 0x06, 0xfb, 0x4b, 0xba, 0xa8, 0x7b, 0x8a, 0xaa, 0x0f, 0x0e, 0x61, 0x25,
+ 0x4c, 0xdc, 0x2d, 0x5a, 0xae, 0x14, 0xdf, 0xf4, 0xf4, 0xd6, 0xd8, 0x8f, 0x1b, 0x81, 0x23, 0x28,
+ 0x0d, 0xb1, 0x67, 0x98, 0x6e, 0xd4, 0xe9, 0xcc, 0xdc, 0xc0, 0xf1, 0xd2, 0x48, 0x0a, 0x3c, 0x8c,
+ 0x17, 0xb2, 0xb6, 0x00, 0x86, 0xd1, 0x8e, 0x14, 0x37, 0xe8, 0xc5, 0x3e, 0x30, 0xdc, 0x5f, 0xd2,
+ 0x63, 0x10, 0x48, 0x87, 0x42, 0xec, 0xa3, 0x50, 0x71, 0x7b, 0x5e, 0xf0, 0x13, 0xc2, 0xfd, 0x25,
+ 0x3d, 0x0e, 0x82, 0x3a, 0x50, 0x64, 0x15, 0x72, 0xe1, 0xd8, 0xf3, 0x73, 0x83, 0xc6, 0x4a, 0x65,
+ 0x28, 0xa8, 0x17, 0xab, 0x9c, 0x69, 0x00, 0x4c, 0x32, 0x95, 0xe2, 0xae, 0xbb, 0x50, 0x8a, 0x90,
+ 0x5e, 0x9b, 0xa3, 0x94, 0x24, 0xea, 0xc3, 0x5a, 0xec, 0xbb, 0x9b, 0xa8, 0xab, 0xc5, 0x05, 0x3f,
+ 0x65, 0x8c, 0xd5, 0xca, 0xec, 0x2f, 0xe9, 0xc2, 0x2d, 0x8b, 0x17, 0xd0, 0x10, 0x40, 0xe7, 0x2b,
+ 0xa2, 0x37, 0x56, 0xae, 0xfe, 0xc5, 0xe4, 0x44, 0x4c, 0x8c, 0x58, 0xcd, 0xc0, 0xb2, 0xe7, 0xba,
+ 0x81, 0xf2, 0x65, 0x16, 0x6e, 0xa8, 0x5f, 0x90, 0xde, 0x88, 0x55, 0xc7, 0x76, 0x02, 0x7c, 0x12,
+ 0xad, 0xf5, 0x36, 0x14, 0x62, 0x61, 0x69, 0xb1, 0xb7, 0x17, 0xfd, 0xb0, 0x31, 0x0e, 0x81, 0x30,
+ 0x1d, 0x9b, 0x17, 0x96, 0x81, 0xf9, 0xdc, 0x59, 0x14, 0xb3, 0xbd, 0x3d, 0xd7, 0xd8, 0xa6, 0xfd,
+ 0x4c, 0x5d, 0x7a, 0x3e, 0xeb, 0x79, 0xae, 0x43, 0x9a, 0x4f, 0xb8, 0x38, 0x77, 0x2d, 0x31, 0x79,
+ 0x17, 0x94, 0x4e, 0xab, 0x73, 0xf9, 0x02, 0x17, 0xa9, 0x66, 0xb2, 0x46, 0x34, 0x73, 0xaa, 0x80,
+ 0xfa, 0xcd, 0xa9, 0xaf, 0xbc, 0x97, 0x59, 0xca, 0x37, 0xfe, 0x99, 0xf6, 0xc6, 0xe4, 0x83, 0xc0,
+ 0x34, 0x7b, 0x19, 0x7d, 0xd4, 0x37, 0x6d, 0x47, 0x33, 0x2f, 0x6b, 0x47, 0xfb, 0x50, 0x18, 0xf9,
+ 0x54, 0xc7, 0x01, 0x0e, 0x88, 0xbf, 0x91, 0x7d, 0xd9, 0x01, 0x1f, 0xf8, 0xc4, 0x63, 0x95, 0x79,
+ 0x74, 0xc0, 0xa3, 0xf0, 0xc1, 0x47, 0x4f, 0x21, 0xc3, 0xd2, 0xad, 0xfe, 0x46, 0x8e, 0x89, 0xa8,
+ 0x5c, 0x5d, 0x04, 0x2b, 0xe0, 0xd3, 0x4c, 0x5d, 0x00, 0xca, 0x2d, 0x28, 0xc4, 0xd4, 0x3c, 0x8f,
+ 0x47, 0xf2, 0x75, 0x00, 0xdb, 0xed, 0x61, 0x9b, 0x7f, 0xd9, 0xc0, 0x17, 0x40, 0x9e, 0x51, 0x9a,
+ 0x78, 0x40, 0x28, 0x60, 0x6c, 0x18, 0xaf, 0x00, 0xf0, 0x31, 0x64, 0x45, 0xa7, 0x5f, 0x1e, 0x6c,
+ 0xeb, 0x0f, 0x12, 0xf1, 0xaa, 0x32, 0xe1, 0xd3, 0xa2, 0x9f, 0x40, 0x86, 0xd7, 0xa6, 0xa0, 0x79,
+ 0x02, 0x89, 0x53, 0x65, 0x2c, 0xf2, 0x4b, 0x94, 0x99, 0x29, 0x4b, 0x77, 0x12, 0x3b, 0x9f, 0x42,
+ 0x8e, 0xfd, 0x63, 0x04, 0x75, 0x49, 0xdf, 0x3a, 0xe7, 0xd2, 0x50, 0x37, 0x84, 0x39, 0x33, 0xad,
+ 0x21, 0xff, 0xcb, 0x81, 0x7f, 0xfc, 0xb3, 0xbf, 0x7e, 0xc2, 0x9d, 0x16, 0xca, 0x75, 0xe0, 0x39,
+ 0x3b, 0x1a, 0xac, 0x30, 0x80, 0x9e, 0xf8, 0xe7, 0x86, 0x79, 0x50, 0xfe, 0x29, 0x44, 0x29, 0x1e,
+ 0xc7, 0xfe, 0x01, 0xa2, 0xfa, 0x6d, 0xf8, 0xea, 0xbf, 0xa9, 0xa8, 0xe6, 0x75, 0x56, 0xb5, 0x52,
+ 0x19, 0x5a, 0x9f, 0x15, 0x42, 0xba, 0x71, 0xb6, 0x79, 0x9c, 0x61, 0xe2, 0xb6, 0xff, 0x2f, 0x00,
+ 0x00, 0xff, 0xff, 0x74, 0x65, 0x10, 0xa9, 0x01, 0x43, 0x00, 0x00,
}
// Reference imports to suppress errors if they are not otherwise used.
diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go
index 67cd387..91e588f 100644
--- a/sdks/java/container/boot.go
+++ b/sdks/java/container/boot.go
@@ -97,6 +97,10 @@
os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pb.ApiServiceDescriptor{Url: *loggingEndpoint}))
os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pb.ApiServiceDescriptor{Url: *controlEndpoint}))
+ if info.GetStatusEndpoint() != nil {
+ os.Setenv("STATUS_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(info.GetStatusEndpoint()))
+ }
+
const jarsDir = "/opt/apache/beam/jars"
cp := []string{
filepath.Join(jarsDir, "slf4j-api.jar"),
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueSetter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueSetter.java
index 5d9e82b..db7caaa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueSetter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueSetter.java
@@ -26,7 +26,7 @@
*
* <p>An interface to set a field of a class.
*
- * <p>Implementations of this interface are generated at runtime to map Row fields back into objet
+ * <p>Implementations of this interface are generated at runtime to map Row fields back into object
* fields.
*/
@Internal
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
index a6ecc45..33ed888 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
@@ -22,7 +22,10 @@
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
import org.apache.beam.sdk.schemas.utils.ReflectUtils;
import org.apache.beam.sdk.values.TypeDescriptor;
@@ -47,6 +50,8 @@
@Nullable
public abstract Method getMethod();
+ public abstract Map<String, FieldValueTypeInformation> getOneOfTypes();
+
/** If the field is a container type, returns the element type. */
@Nullable
public abstract FieldValueTypeInformation getElementType();
@@ -62,7 +67,7 @@
abstract Builder toBuilder();
@AutoValue.Builder
- abstract static class Builder {
+ public abstract static class Builder {
public abstract Builder setName(String name);
public abstract Builder setNullable(boolean nullable);
@@ -75,6 +80,8 @@
public abstract Builder setMethod(@Nullable Method method);
+ public abstract Builder setOneOfTypes(Map<String, FieldValueTypeInformation> oneOfTypes);
+
public abstract Builder setElementType(@Nullable FieldValueTypeInformation elementType);
public abstract Builder setMapKeyType(@Nullable FieldValueTypeInformation mapKeyType);
@@ -84,6 +91,22 @@
abstract FieldValueTypeInformation build();
}
+ public static FieldValueTypeInformation forOneOf(
+ String name, boolean nullable, Map<String, FieldValueTypeInformation> oneOfTypes) {
+ final TypeDescriptor<OneOfType.Value> typeDescriptor = TypeDescriptor.of(OneOfType.Value.class);
+ return new AutoValue_FieldValueTypeInformation.Builder()
+ .setName(name)
+ .setNullable(nullable)
+ .setType(typeDescriptor)
+ .setRawType(typeDescriptor.getRawType())
+ .setField(null)
+ .setElementType(null)
+ .setMapKeyType(null)
+ .setMapValueType(null)
+ .setOneOfTypes(oneOfTypes)
+ .build();
+ }
+
public static FieldValueTypeInformation forField(Field field) {
TypeDescriptor type = TypeDescriptor.of(field.getGenericType());
return new AutoValue_FieldValueTypeInformation.Builder()
@@ -95,6 +118,7 @@
.setElementType(getIterableComponentType(field))
.setMapKeyType(getMapKeyType(field))
.setMapValueType(getMapValueType(field))
+ .setOneOfTypes(Collections.emptyMap())
.build();
}
@@ -119,6 +143,7 @@
.setElementType(getIterableComponentType(type))
.setMapKeyType(getMapKeyType(type))
.setMapValueType(getMapValueType(type))
+ .setOneOfTypes(Collections.emptyMap())
.build();
}
@@ -148,6 +173,7 @@
.setElementType(getIterableComponentType(type))
.setMapKeyType(getMapKeyType(type))
.setMapValueType(getMapValueType(type))
+ .setOneOfTypes(Collections.emptyMap())
.build();
}
@@ -175,6 +201,7 @@
.setElementType(getIterableComponentType(componentType))
.setMapKeyType(getMapKeyType(componentType))
.setMapValueType(getMapValueType(componentType))
+ .setOneOfTypes(Collections.emptyMap())
.build();
}
@@ -217,6 +244,7 @@
.setElementType(getIterableComponentType(mapType))
.setMapKeyType(getMapKeyType(mapType))
.setMapValueType(getMapValueType(mapType))
+ .setOneOfTypes(Collections.emptyMap())
.build();
}
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
index 61c0d05..499991f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
@@ -28,6 +28,7 @@
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.RowWithGetters;
@@ -80,13 +81,7 @@
FieldValueTypeInformation typeInformation = checkNotNull(typeInformations.get(i));
params[i] =
fromValue(
- type,
- row.getValue(i),
- typeInformation.getRawType(),
- typeInformation.getElementType(),
- typeInformation.getMapKeyType(),
- typeInformation.getMapValueType(),
- typeFactory);
+ type, row.getValue(i), typeInformation.getRawType(), typeInformation, typeFactory);
}
SchemaUserTypeCreator creator = schemaTypeCreatorFactory.create(clazz, schema);
@@ -99,10 +94,11 @@
FieldType type,
ValueT value,
Type fieldType,
- FieldValueTypeInformation elementType,
- FieldValueTypeInformation keyType,
- FieldValueTypeInformation valueType,
+ FieldValueTypeInformation fieldValueTypeInformation,
Factory<List<FieldValueTypeInformation>> typeFactory) {
+ FieldValueTypeInformation elementType = fieldValueTypeInformation.getElementType();
+ FieldValueTypeInformation keyType = fieldValueTypeInformation.getMapKeyType();
+ FieldValueTypeInformation valueType = fieldValueTypeInformation.getMapValueType();
if (value == null) {
return null;
}
@@ -127,6 +123,22 @@
valueType,
typeFactory);
} else {
+ if (type.getTypeName().isLogicalType()
+ && OneOfType.IDENTIFIER.equals(type.getLogicalType().getIdentifier())) {
+ OneOfType oneOfType = type.getLogicalType(OneOfType.class);
+ OneOfType.Value oneOfValue = oneOfType.toInputType((Row) value);
+ FieldValueTypeInformation oneOfFieldValueTypeInformation =
+ checkNotNull(
+ fieldValueTypeInformation.getOneOfTypes().get(oneOfValue.getCaseType().toString()));
+ Object fromValue =
+ fromValue(
+ oneOfValue.getFieldType(),
+ oneOfValue.getValue(),
+ oneOfFieldValueTypeInformation.getRawType(),
+ oneOfFieldValueTypeInformation,
+ typeFactory);
+ return (ValueT) oneOfType.createValue(oneOfValue.getCaseType(), fromValue);
+ }
return value;
}
}
@@ -156,9 +168,7 @@
elementType,
element,
elementTypeInformation.getType().getType(),
- elementTypeInformation.getElementType(),
- elementTypeInformation.getMapKeyType(),
- elementTypeInformation.getMapValueType(),
+ elementTypeInformation,
typeFactory));
}
@@ -175,9 +185,7 @@
elementType,
element,
elementTypeInformation.getType().getType(),
- elementTypeInformation.getElementType(),
- elementTypeInformation.getMapKeyType(),
- elementTypeInformation.getMapValueType(),
+ elementTypeInformation,
typeFactory));
}
@@ -196,18 +204,14 @@
keyType,
entry.getKey(),
keyTypeInformation.getType().getType(),
- keyTypeInformation.getElementType(),
- keyTypeInformation.getMapKeyType(),
- keyTypeInformation.getMapValueType(),
+ keyTypeInformation,
typeFactory);
Object value =
fromValue(
valueType,
entry.getValue(),
valueTypeInformation.getType().getType(),
- valueTypeInformation.getElementType(),
- valueTypeInformation.getMapKeyType(),
- valueTypeInformation.getMapValueType(),
+ valueTypeInformation,
typeFactory);
newMap.put(key, value);
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
index c998037..fc6eb5e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
@@ -559,8 +559,14 @@
abstract FieldType.Builder toBuilder();
+ public boolean isLogicalType(String logicalTypeIdentifier) {
+ return getTypeName().isLogicalType()
+ && getLogicalType().getIdentifier().equals(logicalTypeIdentifier);
+ }
+
/** Helper function for retrieving the concrete logical type subclass. */
- public <LogicalTypeT> LogicalTypeT getLogicalType(Class<LogicalTypeT> logicalTypeClass) {
+ public <LogicalTypeT extends LogicalType> LogicalTypeT getLogicalType(
+ Class<LogicalTypeT> logicalTypeClass) {
return logicalTypeClass.cast(getLogicalType());
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
index cea324a..6784712 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
@@ -20,12 +20,19 @@
import java.util.Map;
import java.util.UUID;
import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.model.pipeline.v1.SchemaApi.ArrayTypeValue;
+import org.apache.beam.model.pipeline.v1.SchemaApi.FieldValue;
+import org.apache.beam.model.pipeline.v1.SchemaApi.IterableTypeValue;
+import org.apache.beam.model.pipeline.v1.SchemaApi.MapTypeEntry;
+import org.apache.beam.model.pipeline.v1.SchemaApi.MapTypeValue;
import org.apache.beam.sdk.schemas.Schema.Field;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.Schema.LogicalType;
import org.apache.beam.sdk.schemas.Schema.TypeName;
import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
/** Utility methods for translating schemas. */
@@ -35,7 +42,7 @@
private static final String URN_BEAM_LOGICAL_DECIMAL = "beam:logical_type:decimal:v1";
private static final String URN_BEAM_LOGICAL_JAVASDK = "beam:logical_type:javasdk:v1";
- public static SchemaApi.Schema schemaToProto(Schema schema) {
+ public static SchemaApi.Schema schemaToProto(Schema schema, boolean serializeLogicalType) {
String uuid = schema.getUUID() != null ? schema.getUUID().toString() : "";
SchemaApi.Schema.Builder builder = SchemaApi.Schema.newBuilder().setId(uuid);
for (Field field : schema.getFields()) {
@@ -43,62 +50,76 @@
fieldToProto(
field,
schema.indexOf(field.getName()),
- schema.getEncodingPositions().get(field.getName()));
+ schema.getEncodingPositions().get(field.getName()),
+ serializeLogicalType);
builder.addFields(protoField);
}
return builder.build();
}
- private static SchemaApi.Field fieldToProto(Field field, int fieldId, int position) {
+ private static SchemaApi.Field fieldToProto(
+ Field field, int fieldId, int position, boolean serializeLogicalType) {
return SchemaApi.Field.newBuilder()
.setName(field.getName())
.setDescription(field.getDescription())
- .setType(fieldTypeToProto(field.getType()))
+ .setType(fieldTypeToProto(field.getType(), serializeLogicalType))
.setId(fieldId)
.setEncodingPosition(position)
.build();
}
- private static SchemaApi.FieldType fieldTypeToProto(FieldType fieldType) {
+ private static SchemaApi.FieldType fieldTypeToProto(
+ FieldType fieldType, boolean serializeLogicalType) {
SchemaApi.FieldType.Builder builder = SchemaApi.FieldType.newBuilder();
switch (fieldType.getTypeName()) {
case ROW:
builder.setRowType(
- SchemaApi.RowType.newBuilder().setSchema(schemaToProto(fieldType.getRowSchema())));
+ SchemaApi.RowType.newBuilder()
+ .setSchema(schemaToProto(fieldType.getRowSchema(), serializeLogicalType)));
break;
case ARRAY:
builder.setArrayType(
SchemaApi.ArrayType.newBuilder()
- .setElementType(fieldTypeToProto(fieldType.getCollectionElementType())));
+ .setElementType(
+ fieldTypeToProto(fieldType.getCollectionElementType(), serializeLogicalType)));
break;
case ITERABLE:
builder.setIterableType(
SchemaApi.IterableType.newBuilder()
- .setElementType(fieldTypeToProto(fieldType.getCollectionElementType())));
+ .setElementType(
+ fieldTypeToProto(fieldType.getCollectionElementType(), serializeLogicalType)));
break;
case MAP:
builder.setMapType(
SchemaApi.MapType.newBuilder()
- .setKeyType(fieldTypeToProto(fieldType.getMapKeyType()))
- .setValueType(fieldTypeToProto(fieldType.getMapValueType()))
+ .setKeyType(fieldTypeToProto(fieldType.getMapKeyType(), serializeLogicalType))
+ .setValueType(fieldTypeToProto(fieldType.getMapValueType(), serializeLogicalType))
.build());
break;
case LOGICAL_TYPE:
LogicalType logicalType = fieldType.getLogicalType();
- builder.setLogicalType(
+ SchemaApi.LogicalType.Builder logicalTypeBuilder =
SchemaApi.LogicalType.newBuilder()
+ .setArgumentType(
+ fieldTypeToProto(logicalType.getArgumentType(), serializeLogicalType))
+ .setArgument(
+ rowFieldToProto(logicalType.getArgumentType(), logicalType.getArgument()))
+ .setRepresentation(
+ fieldTypeToProto(logicalType.getBaseType(), serializeLogicalType))
// TODO(BEAM-7855): "javasdk" types should only be a last resort. Types defined in
// Beam should have their own URN, and there should be a mechanism for users to
// register their own types by URN.
- .setUrn(URN_BEAM_LOGICAL_JAVASDK)
- .setPayload(
- ByteString.copyFrom(SerializableUtils.serializeToByteArray(logicalType)))
- .setRepresentation(fieldTypeToProto(logicalType.getBaseType()))
- .build());
+ .setUrn(URN_BEAM_LOGICAL_JAVASDK);
+ if (serializeLogicalType) {
+ logicalTypeBuilder =
+ logicalTypeBuilder.setPayload(
+ ByteString.copyFrom(SerializableUtils.serializeToByteArray(logicalType)));
+ }
+ builder.setLogicalType(logicalTypeBuilder.build());
break;
// Special-case for DATETIME and DECIMAL which are logical types in portable representation,
// but not yet in Java. (BEAM-7554)
@@ -106,14 +127,14 @@
builder.setLogicalType(
SchemaApi.LogicalType.newBuilder()
.setUrn(URN_BEAM_LOGICAL_DATETIME)
- .setRepresentation(fieldTypeToProto(FieldType.INT64))
+ .setRepresentation(fieldTypeToProto(FieldType.INT64, serializeLogicalType))
.build());
break;
case DECIMAL:
builder.setLogicalType(
SchemaApi.LogicalType.newBuilder()
.setUrn(URN_BEAM_LOGICAL_DECIMAL)
- .setRepresentation(fieldTypeToProto(FieldType.BYTES))
+ .setRepresentation(fieldTypeToProto(FieldType.BYTES, serializeLogicalType))
.build());
break;
case BYTE:
@@ -240,4 +261,94 @@
"Unexpected type_info: " + protoFieldType.getTypeInfoCase());
}
}
+
+ public static SchemaApi.Row rowToProto(Row row) {
+ SchemaApi.Row.Builder builder = SchemaApi.Row.newBuilder();
+ for (int i = 0; i < row.getFieldCount(); ++i) {
+ builder.addValues(rowFieldToProto(row.getSchema().getField(i).getType(), row.getValue(i)));
+ }
+ return builder.build();
+ }
+
+ private static SchemaApi.FieldValue rowFieldToProto(FieldType fieldType, Object value) {
+ FieldValue.Builder builder = FieldValue.newBuilder();
+ switch (fieldType.getTypeName()) {
+ case ARRAY:
+ return builder
+ .setArrayValue(
+ arrayValueToProto(fieldType.getCollectionElementType(), (Iterable) value))
+ .build();
+ case ITERABLE:
+ return builder
+ .setIterableValue(
+ iterableValueToProto(fieldType.getCollectionElementType(), (Iterable) value))
+ .build();
+ case MAP:
+ return builder
+ .setMapValue(
+ mapToProto(fieldType.getMapKeyType(), fieldType.getMapValueType(), (Map) value))
+ .build();
+ case ROW:
+ return builder.setRowValue(rowToProto((Row) value)).build();
+ case LOGICAL_TYPE:
+ default:
+ return builder.setAtomicValue(primitiveRowFieldToProto(fieldType, value)).build();
+ }
+ }
+
+ private static SchemaApi.ArrayTypeValue arrayValueToProto(
+ FieldType elementType, Iterable values) {
+ return ArrayTypeValue.newBuilder()
+ .addAllElement(Iterables.transform(values, e -> rowFieldToProto(elementType, e)))
+ .build();
+ }
+
+ private static SchemaApi.IterableTypeValue iterableValueToProto(
+ FieldType elementType, Iterable values) {
+ return IterableTypeValue.newBuilder()
+ .addAllElement(Iterables.transform(values, e -> rowFieldToProto(elementType, e)))
+ .build();
+ }
+
+ private static SchemaApi.MapTypeValue mapToProto(
+ FieldType keyType, FieldType valueType, Map<Object, Object> map) {
+ MapTypeValue.Builder builder = MapTypeValue.newBuilder();
+ for (Map.Entry entry : map.entrySet()) {
+ MapTypeEntry mapProtoEntry =
+ MapTypeEntry.newBuilder()
+ .setKey(rowFieldToProto(keyType, entry.getKey()))
+ .setValue(rowFieldToProto(valueType, entry.getValue()))
+ .build();
+ builder.addEntries(mapProtoEntry);
+ }
+ return builder.build();
+ }
+
+ private static SchemaApi.AtomicTypeValue primitiveRowFieldToProto(
+ FieldType fieldType, Object value) {
+ switch (fieldType.getTypeName()) {
+ case BYTE:
+ return SchemaApi.AtomicTypeValue.newBuilder().setByte((int) value).build();
+ case INT16:
+ return SchemaApi.AtomicTypeValue.newBuilder().setInt16((int) value).build();
+ case INT32:
+ return SchemaApi.AtomicTypeValue.newBuilder().setInt32((int) value).build();
+ case INT64:
+ return SchemaApi.AtomicTypeValue.newBuilder().setInt64((long) value).build();
+ case FLOAT:
+ return SchemaApi.AtomicTypeValue.newBuilder().setFloat((float) value).build();
+ case DOUBLE:
+ return SchemaApi.AtomicTypeValue.newBuilder().setDouble((double) value).build();
+ case STRING:
+ return SchemaApi.AtomicTypeValue.newBuilder().setString((String) value).build();
+ case BOOLEAN:
+ return SchemaApi.AtomicTypeValue.newBuilder().setBoolean((boolean) value).build();
+ case BYTES:
+ return SchemaApi.AtomicTypeValue.newBuilder()
+ .setBytes(ByteString.copyFrom((byte[]) value))
+ .build();
+ default:
+ throw new RuntimeException("FieldType unexpected " + fieldType.getTypeName());
+ }
+ }
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java
index 214eeb5..dc9d9b0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java
@@ -22,7 +22,9 @@
import java.util.Arrays;
import java.util.List;
+import java.util.Map;
import java.util.stream.Collectors;
+import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.Schema.Field;
import org.apache.beam.sdk.schemas.Schema.FieldType;
@@ -44,15 +46,24 @@
private final byte[] schemaProtoRepresentation;
private OneOfType(List<Field> fields) {
+ this(fields, null);
+ }
+
+ private OneOfType(List<Field> fields, @Nullable Map<String, Integer> enumMap) {
List<Field> nullableFields =
fields.stream()
.map(f -> Field.nullable(f.getName(), f.getType()))
.collect(Collectors.toList());
- List<String> enumValues =
- nullableFields.stream().map(Field::getName).collect(Collectors.toList());
+ if (enumMap != null) {
+ nullableFields.stream().forEach(f -> checkArgument(enumMap.containsKey(f.getName())));
+ enumerationType = EnumerationType.create(enumMap);
+ } else {
+ List<String> enumValues =
+ nullableFields.stream().map(Field::getName).collect(Collectors.toList());
+ enumerationType = EnumerationType.create(enumValues);
+ }
oneOfSchema = Schema.builder().addFields(nullableFields).build();
- enumerationType = EnumerationType.create(enumValues);
- schemaProtoRepresentation = SchemaTranslation.schemaToProto(oneOfSchema).toByteArray();
+ schemaProtoRepresentation = SchemaTranslation.schemaToProto(oneOfSchema, false).toByteArray();
}
/** Create an {@link OneOfType} logical type. */
@@ -65,6 +76,14 @@
return new OneOfType(fields);
}
+ /**
+ * Create an {@link OneOfType} logical type. This method allows control over the integer values in
+ * the generated enum.
+ */
+ public static OneOfType create(List<Field> fields, Map<String, Integer> enumValues) {
+ return new OneOfType(fields, enumValues);
+ }
+
/** Returns the schema of the underlying {@link Row} that is used to represent the union. */
public Schema getOneOfSchema() {
return oneOfSchema;
@@ -158,15 +177,24 @@
return caseType;
}
- /** Returns the current value of the OneOf. */
- @SuppressWarnings("TypeParameterUnusedInFormals")
- public <T> T getValue() {
+ /** Returns the current value of the OneOf as the destination type. */
+ public <T> T getValue(Class<T> clazz) {
return (T) value;
}
+ /** Returns the current value of the OneOf. */
+ public Object getValue() {
+ return value;
+ }
+
/** Return the type of this union field. */
public FieldType getFieldType() {
return fieldType;
}
+
+ @Override
+ public String toString() {
+ return "caseType: " + caseType + " Value: " + value;
+ }
}
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java
index 5176def..5233b0c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java
@@ -40,7 +40,7 @@
* Convert a {@link PCollection}{@literal <InputT>} into a {@link PCollection}{@literal <Row>}.
*
* <p>The input {@link PCollection} must have a schema attached. The output collection will have
- * the same schema as the iput.
+ * the same schema as the input.
*/
public static <InputT> PTransform<PCollection<InputT>, PCollection<Row>> toRows() {
return to(Row.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java
index fd7f601..436da6c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java
@@ -54,7 +54,7 @@
static <T extends SpecificRecord> SchemaUserTypeCreator getCreator(
Class<T> clazz, Schema schema) {
return CACHED_CREATORS.computeIfAbsent(
- new ClassWithSchema(clazz, schema), c -> createCreator(clazz, schema));
+ ClassWithSchema.create(clazz, schema), c -> createCreator(clazz, schema));
}
private static <T> SchemaUserTypeCreator createCreator(Class<T> clazz, Schema schema) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java
index 791dafb..c00d5d0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java
@@ -94,7 +94,6 @@
new ForLoadedType(ReadableInstant.class);
private static final ForLoadedType READABLE_PARTIAL_TYPE =
new ForLoadedType(ReadablePartial.class);
- private static final ForLoadedType OBJECT_TYPE = new ForLoadedType(Object.class);
private static final ForLoadedType INTEGER_TYPE = new ForLoadedType(Integer.class);
private static final ForLoadedType ENUM_TYPE = new ForLoadedType(Enum.class);
private static final ForLoadedType BYTE_BUDDY_UTILS_TYPE =
@@ -134,7 +133,7 @@
// Create a new FieldValueGetter subclass.
@SuppressWarnings("unchecked")
- static DynamicType.Builder<FieldValueGetter> subclassGetterInterface(
+ public static DynamicType.Builder<FieldValueGetter> subclassGetterInterface(
ByteBuddy byteBuddy, Type objectType, Type fieldType) {
TypeDescription.Generic getterGenericType =
TypeDescription.Generic.Builder.parameterizedType(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java
index 759d77d..e25342b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java
@@ -102,7 +102,7 @@
public static List<FieldValueTypeInformation> getFieldTypes(
Class<?> clazz, Schema schema, FieldValueTypeSupplier fieldValueTypeSupplier) {
return CACHED_FIELD_TYPES.computeIfAbsent(
- new ClassWithSchema(clazz, schema), c -> fieldValueTypeSupplier.get(clazz, schema));
+ ClassWithSchema.create(clazz, schema), c -> fieldValueTypeSupplier.get(clazz, schema));
}
// The list of getters for a class is cached, so we only create the classes the first time
@@ -121,7 +121,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_GETTERS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return types.stream()
@@ -130,7 +130,7 @@
});
}
- private static <T> FieldValueGetter createGetter(
+ public static <T> FieldValueGetter createGetter(
FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) {
DynamicType.Builder<FieldValueGetter> builder =
ByteBuddyUtils.subclassGetterInterface(
@@ -184,7 +184,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_SETTERS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return types.stream()
@@ -193,14 +193,14 @@
});
}
- private static FieldValueSetter createSetter(
+ public static FieldValueSetter createSetter(
FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) {
DynamicType.Builder<FieldValueSetter> builder =
ByteBuddyUtils.subclassSetterInterface(
BYTE_BUDDY,
typeInformation.getMethod().getDeclaringClass(),
typeConversionsFactory.createTypeConversion(false).convert(typeInformation.getType()));
- builder = implementSetterMethods(builder, typeInformation.getMethod(), typeConversionsFactory);
+ builder = implementSetterMethods(builder, typeInformation, typeConversionsFactory);
try {
return builder
.make()
@@ -222,14 +222,13 @@
private static DynamicType.Builder<FieldValueSetter> implementSetterMethods(
DynamicType.Builder<FieldValueSetter> builder,
- Method method,
+ FieldValueTypeInformation fieldValueTypeInformation,
TypeConversionsFactory typeConversionsFactory) {
- FieldValueTypeInformation javaTypeInformation = FieldValueTypeInformation.forSetter(method);
return builder
.method(ElementMatchers.named("name"))
- .intercept(FixedValue.reference(javaTypeInformation.getName()))
+ .intercept(FixedValue.reference(fieldValueTypeInformation.getName()))
.method(ElementMatchers.named("set"))
- .intercept(new InvokeSetterInstruction(method, typeConversionsFactory));
+ .intercept(new InvokeSetterInstruction(fieldValueTypeInformation, typeConversionsFactory));
}
// The list of constructors for a class is cached, so we only create the classes the first time
@@ -244,7 +243,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_CREATORS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return createConstructorCreator(
@@ -291,7 +290,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_CREATORS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return createStaticCreator(clazz, creator, schema, types, typeConversionsFactory);
@@ -377,11 +376,13 @@
// Implements a method to write a public set out on an object.
private static class InvokeSetterInstruction implements Implementation {
// Setter method that will be invoked
- private Method method;
+ private FieldValueTypeInformation fieldValueTypeInformation;
private final TypeConversionsFactory typeConversionsFactory;
- InvokeSetterInstruction(Method method, TypeConversionsFactory typeConversionsFactory) {
- this.method = method;
+ InvokeSetterInstruction(
+ FieldValueTypeInformation fieldValueTypeInformation,
+ TypeConversionsFactory typeConversionsFactory) {
+ this.fieldValueTypeInformation = fieldValueTypeInformation;
this.typeConversionsFactory = typeConversionsFactory;
}
@@ -393,13 +394,13 @@
@Override
public ByteCodeAppender appender(final Target implementationTarget) {
return (methodVisitor, implementationContext, instrumentedMethod) -> {
- FieldValueTypeInformation javaTypeInformation = FieldValueTypeInformation.forSetter(method);
// this + method parameters.
int numLocals = 1 + instrumentedMethod.getParameters().size();
// The instruction to read the field.
StackManipulation readField = MethodVariableAccess.REFERENCE.loadFrom(2);
+ Method method = fieldValueTypeInformation.getMethod();
boolean setterMethodReturnsVoid = method.getReturnType().equals(Void.TYPE);
// Read the object onto the stack.
StackManipulation stackManipulation =
@@ -409,7 +410,7 @@
// Do any conversions necessary.
typeConversionsFactory
.createSetterConversions(readField)
- .convert(javaTypeInformation.getType()),
+ .convert(fieldValueTypeInformation.getType()),
// Now update the field and return void.
MethodInvocation.invoke(new ForLoadedMethod(method)));
if (!setterMethodReturnsVoid) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java
index a58ddf8..aa968b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java
@@ -81,7 +81,7 @@
public static List<FieldValueTypeInformation> getFieldTypes(
Class<?> clazz, Schema schema, FieldValueTypeSupplier fieldValueTypeSupplier) {
return CACHED_FIELD_TYPES.computeIfAbsent(
- new ClassWithSchema(clazz, schema), c -> fieldValueTypeSupplier.get(clazz, schema));
+ ClassWithSchema.create(clazz, schema), c -> fieldValueTypeSupplier.get(clazz, schema));
}
// The list of getters for a class is cached, so we only create the classes the first time
@@ -96,7 +96,7 @@
TypeConversionsFactory typeConversionsFactory) {
// Return the getters ordered by their position in the schema.
return CACHED_GETTERS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
List<FieldValueGetter> getters =
@@ -122,7 +122,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_CREATORS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return createSetFieldCreator(clazz, schema, types, typeConversionsFactory);
@@ -169,7 +169,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_CREATORS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return createConstructorCreator(
@@ -217,7 +217,7 @@
FieldValueTypeSupplier fieldValueTypeSupplier,
TypeConversionsFactory typeConversionsFactory) {
return CACHED_CREATORS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return createStaticCreator(clazz, creator, schema, types, typeConversionsFactory);
@@ -323,7 +323,7 @@
TypeConversionsFactory typeConversionsFactory) {
// Return the setters, ordered by their position in the schema.
return CACHED_SETTERS.computeIfAbsent(
- new ClassWithSchema(clazz, schema),
+ ClassWithSchema.create(clazz, schema),
c -> {
List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
return types.stream()
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java
index d56f0bd..08c494c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java
@@ -19,6 +19,7 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import com.google.auto.value.AutoValue;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
@@ -31,7 +32,6 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.Schema;
@@ -39,35 +39,21 @@
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimaps;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Primitives;
/** A set of reflection helper methods. */
public class ReflectUtils {
/** Represents a class and a schema. */
- public static class ClassWithSchema {
- private final Class clazz;
- private final Schema schema;
+ @AutoValue
+ public abstract static class ClassWithSchema {
+ public abstract Class getClazz();
- public ClassWithSchema(Class clazz, Schema schema) {
- this.clazz = clazz;
- this.schema = schema;
- }
+ public abstract Schema getSchema();
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- ClassWithSchema that = (ClassWithSchema) o;
- return Objects.equals(clazz, that.clazz) && Objects.equals(schema, that.schema);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(clazz, schema);
+ public static ClassWithSchema create(Class clazz, Schema schema) {
+ return new AutoValue_ReflectUtils_ClassWithSchema(clazz, schema);
}
}
@@ -94,6 +80,10 @@
});
}
+ public static Multimap<String, Method> getMethodsMap(Class clazz) {
+ return Multimaps.index(getMethods(clazz), Method::getName);
+ }
+
@Nullable
public static Constructor getAnnotatedConstructor(Class clazz) {
return Arrays.stream(clazz.getDeclaredConstructors())
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
index 84b1253..55c1601 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
@@ -20,8 +20,12 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
import java.io.OutputStream;
import java.util.Arrays;
import java.util.Collection;
@@ -30,6 +34,7 @@
import java.util.List;
import java.util.Objects;
import java.util.Set;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.CollectionCoder;
@@ -505,6 +510,11 @@
return ValueOnlyWindowedValueCoder.of(valueCoder);
}
+ /** Returns the {@code ParamWindowedValueCoder} from the given valueCoder. */
+ public static <T> ParamWindowedValueCoder<T> getParamWindowedValueCoder(Coder<T> valueCoder) {
+ return ParamWindowedValueCoder.of(valueCoder);
+ }
+
/** Abstract class for {@code WindowedValue} coder. */
public abstract static class WindowedValueCoder<T> extends StructuredCoder<WindowedValue<T>> {
final Coder<T> valueCoder;
@@ -637,7 +647,11 @@
*
* <p>A {@code ValueOnlyWindowedValueCoder} only encodes and decodes the value. It drops timestamp
* and windows for encoding, and uses defaults timestamp, and windows for decoding.
+ *
+ * @deprecated Use ParamWindowedValueCoder instead, it is a general purpose implementation of the
+ * same concept but makes timestamp, windows and pane info configurable.
*/
+ @Deprecated
public static class ValueOnlyWindowedValueCoder<T> extends WindowedValueCoder<T> {
public static <T> ValueOnlyWindowedValueCoder<T> of(Coder<T> valueCoder) {
return new ValueOnlyWindowedValueCoder<>(valueCoder);
@@ -693,4 +707,181 @@
return Collections.singletonList(valueCoder);
}
}
+
+ /**
+ * A parameterized coder for {@code WindowedValue}.
+ *
+ * <p>A {@code ParamWindowedValueCoder} only encodes and decodes the value. It drops timestamp,
+ * windows, and pane info during encoding, and uses the supplied parameterized timestamp, windows
+ * and pane info values during decoding when reconstructing the windowed value.
+ */
+ public static class ParamWindowedValueCoder<T> extends FullWindowedValueCoder<T> {
+
+ private static final long serialVersionUID = 1L;
+
+ private transient Instant timestamp;
+ private transient Collection<? extends BoundedWindow> windows;
+ private transient PaneInfo pane;
+
+ private static final byte[] EMPTY_BYTES = new byte[0];
+
+ /**
+ * Returns the {@link ParamWindowedValueCoder} for the given valueCoder and windowCoder using
+ * the supplied parameterized timestamp, windows and pane info for {@link WindowedValue}.
+ */
+ public static <T> ParamWindowedValueCoder<T> of(
+ Coder<T> valueCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ Instant timestamp,
+ Collection<? extends BoundedWindow> windows,
+ PaneInfo pane) {
+ return new ParamWindowedValueCoder<>(valueCoder, windowCoder, timestamp, windows, pane);
+ }
+
+ /**
+ * Returns the {@link ParamWindowedValueCoder} for the given valueCoder and windowCoder using
+ * {@link BoundedWindow#TIMESTAMP_MIN_VALUE} as the timestamp, {@link #GLOBAL_WINDOWS} as the
+ * window and {@link PaneInfo#NO_FIRING} as the pane info for parameters.
+ */
+ public static <T> ParamWindowedValueCoder<T> of(
+ Coder<T> valueCoder, Coder<? extends BoundedWindow> windowCoder) {
+ return ParamWindowedValueCoder.of(
+ valueCoder,
+ windowCoder,
+ BoundedWindow.TIMESTAMP_MIN_VALUE,
+ GLOBAL_WINDOWS,
+ PaneInfo.NO_FIRING);
+ }
+
+ /**
+ * Returns the {@link ParamWindowedValueCoder} for the given valueCoder and {@link
+ * GlobalWindow.Coder#INSTANCE} using {@link BoundedWindow#TIMESTAMP_MIN_VALUE} as the
+ * timestamp, {@link #GLOBAL_WINDOWS} as the window and {@link PaneInfo#NO_FIRING} as the pane
+ * info for parameters.
+ */
+ public static <T> ParamWindowedValueCoder<T> of(Coder<T> valueCoder) {
+ return ParamWindowedValueCoder.of(valueCoder, GlobalWindow.Coder.INSTANCE);
+ }
+
+ ParamWindowedValueCoder(
+ Coder<T> valueCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ Instant timestamp,
+ Collection<? extends BoundedWindow> windows,
+ PaneInfo pane) {
+ super(valueCoder, windowCoder);
+ this.timestamp = timestamp;
+ this.windows = windows;
+ this.pane = pane;
+ }
+
+ @Override
+ public <NewT> WindowedValueCoder<NewT> withValueCoder(Coder<NewT> valueCoder) {
+ return new ParamWindowedValueCoder<>(valueCoder, getWindowCoder(), timestamp, windows, pane);
+ }
+
+ @Override
+ public void encode(WindowedValue<T> windowedElem, OutputStream outStream)
+ throws CoderException, IOException {
+ encode(windowedElem, outStream, Context.NESTED);
+ }
+
+ @Override
+ public void encode(WindowedValue<T> windowedElem, OutputStream outStream, Context context)
+ throws CoderException, IOException {
+ valueCoder.encode(windowedElem.getValue(), outStream, context);
+ }
+
+ @Override
+ public WindowedValue<T> decode(InputStream inStream) throws CoderException, IOException {
+ return decode(inStream, Context.NESTED);
+ }
+
+ @Override
+ public WindowedValue<T> decode(InputStream inStream, Context context)
+ throws CoderException, IOException {
+ T value = valueCoder.decode(inStream, context);
+ return WindowedValue.of(value, this.timestamp, this.windows, this.pane);
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ verifyDeterministic(
+ this, "ParamWindowedValueCoder requires a deterministic valueCoder", valueCoder);
+ }
+
+ @Override
+ public void registerByteSizeObserver(WindowedValue<T> value, ElementByteSizeObserver observer)
+ throws Exception {
+ valueCoder.registerByteSizeObserver(value.getValue(), observer);
+ }
+
+ public Instant getTimestamp() {
+ return timestamp;
+ }
+
+ public Collection<? extends BoundedWindow> getWindows() {
+ return windows;
+ }
+
+ public PaneInfo getPane() {
+ return pane;
+ }
+
+ /** Returns the serialized payload that will be provided when deserializing this coder. */
+ public static byte[] getPayload(ParamWindowedValueCoder<?> from) {
+ // Use FullWindowedValueCoder to encode the constant members(timestamp, window, pane) in
+ // ParamWindowedValueCoder
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ WindowedValue<byte[]> windowedValue =
+ WindowedValue.of(EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPane());
+ WindowedValue.FullWindowedValueCoder<byte[]> windowedValueCoder =
+ WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), from.getWindowCoder());
+ try {
+ windowedValueCoder.encode(windowedValue, baos);
+ } catch (IOException e) {
+ throw new RuntimeException(
+ "Unable to encode constant members of ParamWindowedValueCoder: ", e);
+ }
+ return baos.toByteArray();
+ }
+
+ /** Create a {@link Coder} from its component {@link Coder coders}. */
+ public static WindowedValue.ParamWindowedValueCoder<?> fromComponents(
+ List<Coder<?>> components, byte[] payload) {
+ Coder<? extends BoundedWindow> windowCoder =
+ (Coder<? extends BoundedWindow>) components.get(1);
+ WindowedValue.FullWindowedValueCoder<byte[]> windowedValueCoder =
+ WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder);
+
+ try {
+ ByteArrayInputStream bais = new ByteArrayInputStream(payload);
+ WindowedValue<byte[]> windowedValue = windowedValueCoder.decode(bais);
+ return WindowedValue.ParamWindowedValueCoder.of(
+ components.get(0),
+ windowCoder,
+ windowedValue.getTimestamp(),
+ windowedValue.getWindows(),
+ windowedValue.getPane());
+ } catch (IOException e) {
+ throw new RuntimeException(
+ "Unable to decode constant members from payload for ParamWindowedValueCoder: ", e);
+ }
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ out.defaultWriteObject();
+ out.writeObject(getPayload(this));
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ byte[] payload = (byte[]) in.readObject();
+ ParamWindowedValueCoder<?> paramWindowedValueCoder =
+ fromComponents(Arrays.asList(valueCoder, getWindowCoder()), payload);
+ this.timestamp = paramWindowedValueCoder.timestamp;
+ this.windows = paramWindowedValueCoder.windows;
+ this.pane = paramWindowedValueCoder.pane;
+ }
+ }
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
index 3945ba1..0cc74cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
@@ -214,9 +214,8 @@
* schema doesn't match.
*/
@Nullable
- @SuppressWarnings("TypeParameterUnusedInFormals")
- public <T> T getLogicalTypeValue(String fieldName) {
- return getLogicalTypeValue(getSchema().indexOf(fieldName));
+ public <T> T getLogicalTypeValue(String fieldName, Class<T> clazz) {
+ return getLogicalTypeValue(getSchema().indexOf(fieldName), clazz);
}
/**
@@ -360,8 +359,7 @@
* schema doesn't match.
*/
@Nullable
- @SuppressWarnings("TypeParameterUnusedInFormals")
- public <T> T getLogicalTypeValue(int idx) {
+ public <T> T getLogicalTypeValue(int idx, Class<T> clazz) {
LogicalType logicalType = checkNotNull(getSchema().getField(idx).getType().getLogicalType());
return (T) logicalType.toInputType(getValue(idx));
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java
index ebf59b9..0d68709 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java
@@ -29,6 +29,7 @@
import org.apache.beam.sdk.schemas.Schema.Field;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Collections2;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
@@ -122,6 +123,15 @@
cacheKey, i -> getMapValue(type.getMapKeyType(), type.getMapValueType(), map))
: (T) getMapValue(type.getMapKeyType(), type.getMapValueType(), map);
} else {
+ if (type.isLogicalType(OneOfType.IDENTIFIER)) {
+ OneOfType oneOfType = type.getLogicalType(OneOfType.class);
+ OneOfType.Value oneOfValue = (OneOfType.Value) fieldValue;
+ Object convertedOneOfField =
+ getValue(oneOfValue.getFieldType(), oneOfValue.getValue(), null);
+ return (T)
+ oneOfType.toBaseType(
+ oneOfType.createValue(oneOfValue.getCaseType(), convertedOneOfField));
+ }
return (T) fieldValue;
}
}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java
index 657af04..95a05a3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java
@@ -46,12 +46,12 @@
Row row1 = Row.withSchema(schema).addValue(enumeration.valueOf(1)).build();
Row row2 = Row.withSchema(schema).addValue(enumeration.valueOf("FIRST")).build();
assertEquals(row1, row2);
- assertEquals(1, row1.<EnumerationType.Value>getLogicalTypeValue(0).getValue());
+ assertEquals(1, row1.getLogicalTypeValue(0, EnumerationType.Value.class).getValue());
Row row3 = Row.withSchema(schema).addValue(enumeration.valueOf(2)).build();
Row row4 = Row.withSchema(schema).addValue(enumeration.valueOf("SECOND")).build();
assertEquals(row3, row4);
- assertEquals(2, row3.<EnumerationType.Value>getLogicalTypeValue(0).getValue());
+ assertEquals(2, row3.getLogicalTypeValue(0, EnumerationType.Value.class).getValue());
}
@Test
@@ -62,12 +62,12 @@
Row stringOneOf =
Row.withSchema(schema).addValue(oneOf.createValue("string", "stringValue")).build();
- Value union = stringOneOf.getLogicalTypeValue(0);
+ Value union = stringOneOf.getLogicalTypeValue(0, OneOfType.Value.class);
assertEquals("string", union.getCaseType().toString());
assertEquals("stringValue", union.getValue());
Row intOneOf = Row.withSchema(schema).addValue(oneOf.createValue("int32", 42)).build();
- union = intOneOf.getLogicalTypeValue(0);
+ union = intOneOf.getLogicalTypeValue(0, OneOfType.Value.class);
assertEquals("int32", union.getCaseType().toString());
assertEquals(42, (int) union.getValue());
}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
index e566ebf..a681fbe 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java
@@ -17,8 +17,8 @@
*/
package org.apache.beam.sdk.util;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
index 206b548..821c216 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java
@@ -78,6 +78,12 @@
}
@Test
+ public void testParamWindowedValueCoderIsSerializableWithWellKnownCoderType() {
+ CoderProperties.coderSerializable(
+ WindowedValue.getParamWindowedValueCoder(GlobalWindow.Coder.INSTANCE));
+ }
+
+ @Test
public void testValueOnlyWindowedValueCoderIsSerializableWithWellKnownCoderType() {
CoderProperties.coderSerializable(WindowedValue.getValueOnlyCoder(GlobalWindow.Coder.INSTANCE));
}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java
index 3547411..da62796 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java
@@ -20,7 +20,7 @@
import static org.hamcrest.Matchers.contains;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.when;
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java
index 551f7bc..13a9fd7 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java
@@ -21,9 +21,9 @@
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java
new file mode 100644
index 0000000..6827fb1
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java
@@ -0,0 +1,663 @@
+/*
+ * 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.extensions.protobuf;
+
+import static org.apache.beam.sdk.extensions.protobuf.ProtoSchemaTranslator.getFieldNumber;
+
+import com.google.protobuf.BoolValue;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.Duration;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int32Value;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.ProtocolMessageEnum;
+import com.google.protobuf.StringValue;
+import com.google.protobuf.Timestamp;
+import com.google.protobuf.UInt32Value;
+import com.google.protobuf.UInt64Value;
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.DurationNanos;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.TimestampNanos;
+import org.apache.beam.sdk.schemas.FieldValueGetter;
+import org.apache.beam.sdk.schemas.FieldValueSetter;
+import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.SchemaUserTypeCreator;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType.Value;
+import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertType;
+import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertValueForGetter;
+import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertValueForSetter;
+import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.InjectPackageStrategy;
+import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversion;
+import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory;
+import org.apache.beam.sdk.schemas.utils.FieldValueTypeSupplier;
+import org.apache.beam.sdk.schemas.utils.JavaBeanUtils;
+import org.apache.beam.sdk.schemas.utils.ReflectUtils;
+import org.apache.beam.sdk.schemas.utils.ReflectUtils.ClassWithSchema;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.ByteBuddy;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.description.method.MethodDescription;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.description.type.TypeDescription;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.description.type.TypeDescription.ForLoadedType;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.dynamic.DynamicType;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.dynamic.scaffold.InstrumentedType;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.Implementation;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.ByteCodeAppender;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.StackManipulation;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.StackManipulation.Compound;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.assign.Assigner;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.assign.Assigner.Typing;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.assign.TypeCasting;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.member.MethodInvocation;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.member.MethodReturn;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.matcher.ElementMatchers;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.CaseFormat;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+
+public class ProtoByteBuddyUtils {
+ private static final ByteBuddy BYTE_BUDDY = new ByteBuddy();
+ private static final TypeDescriptor<ByteString> BYTE_STRING_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(ByteString.class);
+ private static final TypeDescriptor<Timestamp> PROTO_TIMESTAMP_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(Timestamp.class);
+ private static final TypeDescriptor<Duration> PROTO_DURATION_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(Duration.class);
+ private static final TypeDescriptor<Int32Value> PROTO_INT32_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(Int32Value.class);
+ private static final TypeDescriptor<Int64Value> PROTO_INT64_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(Int64Value.class);
+ private static final TypeDescriptor<UInt32Value> PROTO_UINT32_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(UInt32Value.class);
+ private static final TypeDescriptor<UInt64Value> PROTO_UINT64_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(UInt64Value.class);
+ private static final TypeDescriptor<FloatValue> PROTO_FLOAT_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(FloatValue.class);
+ private static final TypeDescriptor<DoubleValue> PROTO_DOUBLE_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(DoubleValue.class);
+ private static final TypeDescriptor<BoolValue> PROTO_BOOL_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(BoolValue.class);
+ private static final TypeDescriptor<StringValue> PROTO_STRING_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(StringValue.class);
+ private static final TypeDescriptor<BytesValue> PROTO_BYTES_VALUE_TYPE_DESCRIPTOR =
+ TypeDescriptor.of(BytesValue.class);
+
+ private static final ForLoadedType BYTE_STRING_TYPE = new ForLoadedType(ByteString.class);
+ private static final ForLoadedType BYTE_ARRAY_TYPE = new ForLoadedType(byte[].class);
+ private static final ForLoadedType PROTO_ENUM_TYPE = new ForLoadedType(ProtocolMessageEnum.class);
+ private static final ForLoadedType INTEGER_TYPE = new ForLoadedType(Integer.class);
+
+ private static final Map<TypeDescriptor<?>, ForLoadedType> WRAPPER_LOADED_TYPES =
+ ImmutableMap.<TypeDescriptor<?>, ForLoadedType>builder()
+ .put(PROTO_INT32_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(Int32Value.class))
+ .put(PROTO_INT64_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(Int64Value.class))
+ .put(PROTO_UINT32_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(UInt32Value.class))
+ .put(PROTO_UINT64_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(UInt64Value.class))
+ .put(PROTO_FLOAT_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(FloatValue.class))
+ .put(PROTO_DOUBLE_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(DoubleValue.class))
+ .put(PROTO_BOOL_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(BoolValue.class))
+ .put(PROTO_STRING_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(StringValue.class))
+ .put(PROTO_BYTES_VALUE_TYPE_DESCRIPTOR, new ForLoadedType(BytesValue.class))
+ .build();
+
+ private static final Map<TypeName, String> PROTO_GETTER_SUFFIX =
+ ImmutableMap.of(
+ TypeName.ARRAY, "List",
+ TypeName.ITERABLE, "List",
+ TypeName.MAP, "Map");
+
+ private static final Map<TypeName, String> PROTO_SETTER_PREFIX =
+ ImmutableMap.of(
+ TypeName.ARRAY, "addAll",
+ TypeName.ITERABLE, "addAll",
+ TypeName.MAP, "putAll");
+ private static final String DEFAULT_PROTO_GETTER_PREFIX = "get";
+ private static final String DEFAULT_PROTO_SETTER_PREFIX = "set";
+
+ static String protoGetterName(String name, FieldType fieldType) {
+ final String camel = CaseFormat.LOWER_UNDERSCORE.to(CaseFormat.UPPER_CAMEL, name);
+ return DEFAULT_PROTO_GETTER_PREFIX
+ + camel
+ + PROTO_GETTER_SUFFIX.getOrDefault(fieldType.getTypeName(), "");
+ }
+
+ static String protoSetterName(String name, FieldType fieldType) {
+ final String camel = CaseFormat.LOWER_UNDERSCORE.to(CaseFormat.UPPER_CAMEL, name);
+ return protoSetterPrefix(fieldType) + camel;
+ }
+
+ static String protoSetterPrefix(FieldType fieldType) {
+ return PROTO_SETTER_PREFIX.getOrDefault(fieldType.getTypeName(), DEFAULT_PROTO_SETTER_PREFIX);
+ }
+
+ static class ProtoConvertType extends ConvertType {
+ ProtoConvertType(boolean returnRawValues) {
+ super(returnRawValues);
+ }
+
+ private static final Map<TypeDescriptor<?>, Class<?>> TYPE_OVERRIDES =
+ ImmutableMap.<TypeDescriptor<?>, Class<?>>builder()
+ .put(PROTO_TIMESTAMP_TYPE_DESCRIPTOR, Row.class)
+ .put(PROTO_DURATION_TYPE_DESCRIPTOR, Row.class)
+ .put(PROTO_INT32_VALUE_TYPE_DESCRIPTOR, Integer.class)
+ .put(PROTO_INT64_VALUE_TYPE_DESCRIPTOR, Long.class)
+ .put(PROTO_UINT32_VALUE_TYPE_DESCRIPTOR, Integer.class)
+ .put(PROTO_UINT64_VALUE_TYPE_DESCRIPTOR, Long.class)
+ .put(PROTO_FLOAT_VALUE_TYPE_DESCRIPTOR, Float.class)
+ .put(PROTO_DOUBLE_VALUE_TYPE_DESCRIPTOR, Double.class)
+ .put(PROTO_BOOL_VALUE_TYPE_DESCRIPTOR, Boolean.class)
+ .put(PROTO_STRING_VALUE_TYPE_DESCRIPTOR, String.class)
+ .put(PROTO_BYTES_VALUE_TYPE_DESCRIPTOR, byte[].class)
+ .build();
+
+ @Override
+ public Type convert(TypeDescriptor typeDescriptor) {
+ if (typeDescriptor.equals(BYTE_STRING_TYPE_DESCRIPTOR)
+ || typeDescriptor.isSubtypeOf(BYTE_STRING_TYPE_DESCRIPTOR)) {
+ return byte[].class;
+ } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(ProtocolMessageEnum.class))) {
+ return Integer.class;
+ } else if (typeDescriptor.equals(PROTO_TIMESTAMP_TYPE_DESCRIPTOR)
+ || typeDescriptor.equals(PROTO_DURATION_TYPE_DESCRIPTOR)) {
+ return Row.class;
+ } else {
+ Type type = TYPE_OVERRIDES.get(typeDescriptor);
+ return (type != null) ? type : super.convert(typeDescriptor);
+ }
+ }
+ }
+
+ static class ProtoConvertValueForGetter extends ConvertValueForGetter {
+ ProtoConvertValueForGetter(StackManipulation readValue) {
+ super(readValue);
+ }
+
+ @Override
+ protected ProtoTypeConversionsFactory getFactory() {
+ return new ProtoTypeConversionsFactory();
+ }
+
+ @Override
+ public StackManipulation convert(TypeDescriptor type) {
+ if (type.equals(BYTE_STRING_TYPE_DESCRIPTOR)
+ || type.isSubtypeOf(BYTE_STRING_TYPE_DESCRIPTOR)) {
+ return new Compound(
+ readValue,
+ MethodInvocation.invoke(
+ BYTE_STRING_TYPE
+ .getDeclaredMethods()
+ .filter(ElementMatchers.named("toByteArray"))
+ .getOnly()));
+ } else if (type.isSubtypeOf(TypeDescriptor.of(ProtocolMessageEnum.class))) {
+ return new Compound(
+ readValue,
+ MethodInvocation.invoke(
+ PROTO_ENUM_TYPE
+ .getDeclaredMethods()
+ .filter(
+ ElementMatchers.named("getNumber").and(ElementMatchers.takesArguments(0)))
+ .getOnly()),
+ Assigner.DEFAULT.assign(
+ INTEGER_TYPE.asUnboxed().asGenericType(),
+ INTEGER_TYPE.asGenericType(),
+ Typing.STATIC));
+ } else if (type.equals(PROTO_TIMESTAMP_TYPE_DESCRIPTOR)) {
+ return new Compound(
+ readValue,
+ MethodInvocation.invoke(
+ new ForLoadedType(TimestampNanos.class)
+ .getDeclaredMethods()
+ .filter(ElementMatchers.named("toRow"))
+ .getOnly()));
+ } else if (type.equals(PROTO_DURATION_TYPE_DESCRIPTOR)) {
+ return new Compound(
+ readValue,
+ MethodInvocation.invoke(
+ new ForLoadedType(DurationNanos.class)
+ .getDeclaredMethods()
+ .filter(ElementMatchers.named("toRow"))
+ .getOnly()));
+ } else {
+ ForLoadedType wrapperType = WRAPPER_LOADED_TYPES.get(type);
+ if (wrapperType != null) {
+ MethodDescription.InDefinedShape getValueMethod =
+ wrapperType.getDeclaredMethods().filter(ElementMatchers.named("getValue")).getOnly();
+ TypeDescription.Generic returnType = getValueMethod.getReturnType();
+ StackManipulation stackManipulation =
+ new Compound(
+ readValue,
+ MethodInvocation.invoke(getValueMethod),
+ Assigner.DEFAULT.assign(
+ returnType, returnType.asErasure().asBoxed().asGenericType(), Typing.STATIC));
+ if (type.equals(PROTO_BYTES_VALUE_TYPE_DESCRIPTOR)) {
+ stackManipulation =
+ getFactory()
+ .createGetterConversions(stackManipulation)
+ .convert(BYTE_STRING_TYPE_DESCRIPTOR);
+ }
+ return stackManipulation;
+ }
+ return super.convert(type);
+ }
+ }
+ }
+
+ static class ProtoConvertValueForSetter extends ConvertValueForSetter {
+ ProtoConvertValueForSetter(StackManipulation readValue) {
+ super(readValue);
+ }
+
+ @Override
+ protected ProtoTypeConversionsFactory getFactory() {
+ return new ProtoTypeConversionsFactory();
+ }
+
+ @Override
+ public StackManipulation convert(TypeDescriptor type) {
+ if (type.isSubtypeOf(TypeDescriptor.of(ByteString.class))) {
+ return new Compound(
+ readValue,
+ TypeCasting.to(BYTE_ARRAY_TYPE),
+ MethodInvocation.invoke(
+ BYTE_STRING_TYPE
+ .getDeclaredMethods()
+ .filter(
+ ElementMatchers.named("copyFrom")
+ .and(ElementMatchers.takesArguments(BYTE_ARRAY_TYPE)))
+ .getOnly()));
+ } else if (type.isSubtypeOf(TypeDescriptor.of(ProtocolMessageEnum.class))) {
+ ForLoadedType loadedType = new ForLoadedType(type.getRawType());
+ // Convert the stored number back to the enum constant.
+ return new Compound(
+ readValue,
+ Assigner.DEFAULT.assign(
+ INTEGER_TYPE.asBoxed().asGenericType(),
+ INTEGER_TYPE.asUnboxed().asGenericType(),
+ Typing.STATIC),
+ MethodInvocation.invoke(
+ loadedType
+ .getDeclaredMethods()
+ .filter(
+ ElementMatchers.named("forNumber")
+ .and(ElementMatchers.isStatic().and(ElementMatchers.takesArguments(1))))
+ .getOnly()));
+ } else if (type.equals(PROTO_TIMESTAMP_TYPE_DESCRIPTOR)) {
+ return new Compound(
+ readValue,
+ MethodInvocation.invoke(
+ new ForLoadedType(TimestampNanos.class)
+ .getDeclaredMethods()
+ .filter(ElementMatchers.named("toTimestamp"))
+ .getOnly()));
+ } else if (type.equals(PROTO_DURATION_TYPE_DESCRIPTOR)) {
+ return new Compound(
+ readValue,
+ MethodInvocation.invoke(
+ new ForLoadedType(DurationNanos.class)
+ .getDeclaredMethods()
+ .filter(ElementMatchers.named("toDuration"))
+ .getOnly()));
+ } else {
+ ForLoadedType wrapperType = WRAPPER_LOADED_TYPES.get(type);
+ if (wrapperType != null) {
+ if (type.equals(PROTO_BYTES_VALUE_TYPE_DESCRIPTOR)) {
+ readValue =
+ getFactory()
+ .createSetterConversions(readValue)
+ .convert(TypeDescriptor.of(ByteString.class));
+ }
+ MethodDescription.InDefinedShape ofMethod =
+ wrapperType.getDeclaredMethods().filter(ElementMatchers.named("of")).getOnly();
+ TypeDescription.Generic argumentType = ofMethod.getParameters().get(0).getType();
+ return new Compound(
+ readValue,
+ Assigner.DEFAULT.assign(
+ argumentType.asErasure().asBoxed().asGenericType(), argumentType, Typing.STATIC),
+ MethodInvocation.invoke(ofMethod));
+ } else {
+ return super.convert(type);
+ }
+ }
+ }
+ }
+
+ static class ProtoTypeConversionsFactory implements TypeConversionsFactory {
+ @Override
+ public TypeConversion<Type> createTypeConversion(boolean returnRawTypes) {
+ return new ProtoConvertType(returnRawTypes);
+ }
+
+ @Override
+ public TypeConversion<StackManipulation> createGetterConversions(StackManipulation readValue) {
+ return new ProtoConvertValueForGetter(readValue);
+ }
+
+ @Override
+ public TypeConversion<StackManipulation> createSetterConversions(StackManipulation readValue) {
+ return new ProtoConvertValueForSetter(readValue);
+ }
+ }
+
+ // The list of getters for a class is cached, so we only create the classes the first time
+ // getSetters is called.
+ private static final Map<ClassWithSchema, List<FieldValueGetter>> CACHED_GETTERS =
+ Maps.newConcurrentMap();
+
+ /**
+ * Return the list of {@link FieldValueGetter}s for a Java Bean class
+ *
+ * <p>The returned list is ordered by the order of fields in the schema.
+ */
+ public static List<FieldValueGetter> getGetters(
+ Class<?> clazz,
+ Schema schema,
+ FieldValueTypeSupplier fieldValueTypeSupplier,
+ TypeConversionsFactory typeConversionsFactory) {
+ Multimap<String, Method> methods = ReflectUtils.getMethodsMap(clazz);
+ return CACHED_GETTERS.computeIfAbsent(
+ ClassWithSchema.create(clazz, schema),
+ c -> {
+ List<FieldValueTypeInformation> types = fieldValueTypeSupplier.get(clazz, schema);
+ return types.stream()
+ .map(
+ t ->
+ createGetter(
+ t,
+ typeConversionsFactory,
+ clazz,
+ methods,
+ schema.getField(t.getName()),
+ fieldValueTypeSupplier))
+ .collect(Collectors.toList());
+ });
+ }
+
+ static class OneOfFieldValueGetter<ProtoT extends MessageLite>
+ implements FieldValueGetter<ProtoT, OneOfType.Value> {
+ private final String name;
+ private final Supplier<Method> getCaseMethod;
+ private final Map<Integer, FieldValueGetter<ProtoT, ?>> getterMethodMap;
+ private final OneOfType oneOfType;
+
+ public OneOfFieldValueGetter(
+ String name,
+ Supplier<Method> getCaseMethod,
+ Map<Integer, FieldValueGetter<ProtoT, ?>> getterMethodMap,
+ OneOfType oneOfType) {
+ this.name = name;
+ this.getCaseMethod = getCaseMethod;
+ this.getterMethodMap = getterMethodMap;
+ this.oneOfType = oneOfType;
+ }
+
+ @Nullable
+ @Override
+ public Value get(ProtoT object) {
+ try {
+ EnumLite caseValue = (EnumLite) getCaseMethod.get().invoke(object);
+ if (caseValue.getNumber() == 0) {
+ return null;
+ } else {
+ Object value = getterMethodMap.get(caseValue.getNumber()).get(object);
+ return oneOfType.createValue(
+ oneOfType.getCaseEnumType().valueOf(caseValue.getNumber()), value);
+ }
+ } catch (IllegalAccessException | InvocationTargetException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public String name() {
+ return name;
+ }
+ }
+
+ private static FieldValueGetter createGetter(
+ FieldValueTypeInformation fieldValueTypeInformation,
+ TypeConversionsFactory typeConversionsFactory,
+ Class clazz,
+ Multimap<String, Method> methods,
+ Field field,
+ FieldValueTypeSupplier fieldValueTypeSupplier) {
+ if (field.getType().isLogicalType(OneOfType.IDENTIFIER)) {
+ OneOfType oneOfType = field.getType().getLogicalType(OneOfType.class);
+
+ // The case accessor method in the proto is named getOneOfNameCase.
+ Method caseMethod =
+ getProtoGetter(
+ methods,
+ field.getName() + "_case",
+ FieldType.logicalType(oneOfType.getCaseEnumType()));
+ Map<Integer, FieldValueGetter> oneOfGetters = Maps.newHashMap();
+ Map<String, FieldValueTypeInformation> oneOfFieldTypes =
+ fieldValueTypeSupplier.get(clazz, oneOfType.getOneOfSchema()).stream()
+ .collect(Collectors.toMap(FieldValueTypeInformation::getName, f -> f));
+ for (Field oneOfField : oneOfType.getOneOfSchema().getFields()) {
+ int protoFieldIndex = getFieldNumber(oneOfField.getType());
+ FieldValueGetter oneOfFieldGetter =
+ createGetter(
+ oneOfFieldTypes.get(oneOfField.getName()),
+ typeConversionsFactory,
+ clazz,
+ methods,
+ oneOfField,
+ fieldValueTypeSupplier);
+ oneOfGetters.put(protoFieldIndex, oneOfFieldGetter);
+ }
+ return new OneOfFieldValueGetter(
+ field.getName(),
+ (Supplier<Method> & Serializable) () -> caseMethod,
+ oneOfGetters,
+ oneOfType);
+ } else {
+ return JavaBeanUtils.createGetter(fieldValueTypeInformation, typeConversionsFactory);
+ }
+ }
+
+ private static Class getProtoGeneratedBuilder(Class<?> clazz) {
+ String builderClassName = clazz.getName() + "$Builder";
+ try {
+ return Class.forName(builderClassName);
+ } catch (ClassNotFoundException e) {
+ return null;
+ }
+ }
+
+ static Method getProtoSetter(Multimap<String, Method> methods, String name, FieldType fieldType) {
+ final TypeDescriptor<MessageLite.Builder> builderDescriptor =
+ TypeDescriptor.of(MessageLite.Builder.class);
+ return methods.get(protoSetterName(name, fieldType)).stream()
+ // Setter methods take only a single parameter.
+ .filter(m -> m.getParameterCount() == 1)
+ // For nested types, we don't use the version that takes a builder.
+ .filter(
+ m -> !TypeDescriptor.of(m.getGenericParameterTypes()[0]).isSubtypeOf(builderDescriptor))
+ .findAny()
+ .orElseThrow(IllegalArgumentException::new);
+ }
+
+ static Method getProtoGetter(Multimap<String, Method> methods, String name, FieldType fieldType) {
+ return methods.get(protoGetterName(name, fieldType)).stream()
+ .filter(m -> m.getParameterCount() == 0)
+ .findAny()
+ .orElseThrow(IllegalArgumentException::new);
+ }
+
+ @Nullable
+ public static SchemaUserTypeCreator getBuilderCreator(
+ Class<?> protoClass, Schema schema, FieldValueTypeSupplier fieldValueTypeSupplier) {
+ Class<?> builderClass = getProtoGeneratedBuilder(protoClass);
+ if (builderClass == null) {
+ return null;
+ }
+
+ List<FieldValueSetter> setters = Lists.newArrayListWithCapacity(schema.getFieldCount());
+ Multimap<String, Method> methods = ReflectUtils.getMethodsMap(builderClass);
+ for (Field field : schema.getFields()) {
+ if (field.getType().isLogicalType(OneOfType.IDENTIFIER)) {
+ OneOfType oneOfType = field.getType().getLogicalType(OneOfType.class);
+ Map<Integer, Method> oneOfMethods = Maps.newHashMap();
+ for (Field oneOfField : oneOfType.getOneOfSchema().getFields()) {
+ Method method = getProtoSetter(methods, oneOfField.getName(), oneOfField.getType());
+ oneOfMethods.put(getFieldNumber(oneOfField.getType()), method);
+ }
+ setters.add(
+ new ProtoOneOfSetter(
+ (Function<Integer, Method> & Serializable) oneOfMethods::get, field.getName()));
+ } else {
+ Method method = getProtoSetter(methods, field.getName(), field.getType());
+ setters.add(
+ JavaBeanUtils.createSetter(
+ FieldValueTypeInformation.forSetter(method, protoSetterPrefix(field.getType())),
+ new ProtoTypeConversionsFactory()));
+ }
+ }
+ return createBuilderCreator(protoClass, builderClass, setters, schema);
+ }
+
+ static class ProtoOneOfSetter<BuilderT extends MessageLite.Builder>
+ implements FieldValueSetter<BuilderT, OneOfType.Value> {
+ private final Function<Integer, Method> methods;
+ private final String name;
+
+ ProtoOneOfSetter(Function<Integer, Method> methods, String name) {
+ this.methods = methods;
+ this.name = name;
+ }
+
+ @Override
+ public void set(BuilderT builder, OneOfType.Value oneOfValue) {
+ Method method = methods.apply(oneOfValue.getCaseType().getValue());
+ try {
+ method.invoke(builder, oneOfValue.getValue());
+ } catch (IllegalAccessException | InvocationTargetException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public String name() {
+ return name;
+ }
+ }
+
+ static SchemaUserTypeCreator createBuilderCreator(
+ Class<?> protoClass, Class<?> builderClass, List<FieldValueSetter> setters, Schema schema) {
+ try {
+ DynamicType.Builder<Supplier> builder =
+ BYTE_BUDDY
+ .with(new InjectPackageStrategy(builderClass))
+ .subclass(Supplier.class)
+ .method(ElementMatchers.named("get"))
+ .intercept(new BuilderSupplier(protoClass));
+ Supplier supplier =
+ builder
+ .make()
+ .load(ReflectHelpers.findClassLoader(), ClassLoadingStrategy.Default.INJECTION)
+ .getLoaded()
+ .getDeclaredConstructor()
+ .newInstance();
+ return new ProtoCreatorFactory(supplier, setters);
+ } catch (InstantiationException
+ | IllegalAccessException
+ | NoSuchMethodException
+ | InvocationTargetException e) {
+ throw new RuntimeException(
+ "Unable to generate a creator for class " + builderClass + " with schema " + schema);
+ }
+ }
+
+ static class ProtoCreatorFactory implements SchemaUserTypeCreator {
+ private final Supplier<? extends MessageLite.Builder> builderCreator;
+ private final List<FieldValueSetter> setters;
+
+ public ProtoCreatorFactory(
+ Supplier<? extends MessageLite.Builder> builderCreator, List<FieldValueSetter> setters) {
+ this.builderCreator = builderCreator;
+ this.setters = setters;
+ }
+
+ @Override
+ public Object create(Object... params) {
+ MessageLite.Builder builder = builderCreator.get();
+ for (int i = 0; i < params.length; ++i) {
+ setters.get(i).set(builder, params[i]);
+ }
+ return builder.build();
+ }
+ }
+
+ static class BuilderSupplier implements Implementation {
+ private final Class<?> protoClass;
+
+ public BuilderSupplier(Class<?> protoClass) {
+ this.protoClass = protoClass;
+ }
+
+ @Override
+ public InstrumentedType prepare(InstrumentedType instrumentedType) {
+ return instrumentedType;
+ }
+
+ @Override
+ public ByteCodeAppender appender(final Target implementationTarget) {
+ ForLoadedType loadedProto = new ForLoadedType(protoClass);
+ return (methodVisitor, implementationContext, instrumentedMethod) -> {
+ // this + method parameters.
+ int numLocals = 1 + instrumentedMethod.getParameters().size();
+
+ // Create the builder object by calling ProtoClass.newBuilder().
+ StackManipulation stackManipulation =
+ new StackManipulation.Compound(
+ MethodInvocation.invoke(
+ loadedProto
+ .getDeclaredMethods()
+ .filter(
+ ElementMatchers.named("newBuilder")
+ .and(ElementMatchers.takesArguments(0)))
+ .getOnly()),
+ MethodReturn.REFERENCE);
+ StackManipulation.Size size = stackManipulation.apply(methodVisitor, implementationContext);
+ return new ByteCodeAppender.Size(size.getMaximalSize(), numLocals);
+ };
+ }
+ }
+}
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java
new file mode 100644
index 0000000..47a928c
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java
@@ -0,0 +1,125 @@
+/*
+ * 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.extensions.protobuf;
+
+import static org.apache.beam.sdk.extensions.protobuf.ProtoByteBuddyUtils.getProtoGetter;
+
+import com.google.protobuf.DynamicMessage;
+import com.google.protobuf.Message;
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.extensions.protobuf.ProtoByteBuddyUtils.ProtoTypeConversionsFactory;
+import org.apache.beam.sdk.schemas.FieldValueGetter;
+import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
+import org.apache.beam.sdk.schemas.GetterBasedSchemaProvider;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.SchemaUserTypeCreator;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
+import org.apache.beam.sdk.schemas.utils.FieldValueTypeSupplier;
+import org.apache.beam.sdk.schemas.utils.JavaBeanUtils;
+import org.apache.beam.sdk.schemas.utils.ReflectUtils;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+
+@Experimental(Kind.SCHEMAS)
+public class ProtoMessageSchema extends GetterBasedSchemaProvider {
+
+ private static final class ProtoClassFieldValueTypeSupplier implements FieldValueTypeSupplier {
+ @Override
+ public List<FieldValueTypeInformation> get(Class<?> clazz) {
+ throw new RuntimeException("Unexpected call.");
+ }
+
+ @Override
+ public List<FieldValueTypeInformation> get(Class<?> clazz, Schema schema) {
+ Multimap<String, Method> methods = ReflectUtils.getMethodsMap(clazz);
+ List<FieldValueTypeInformation> types =
+ Lists.newArrayListWithCapacity(schema.getFieldCount());
+ for (Field field : schema.getFields()) {
+ if (field.getType().isLogicalType(OneOfType.IDENTIFIER)) {
+ // This is a OneOf. Look for the getters for each OneOf option.
+ OneOfType oneOfType = field.getType().getLogicalType(OneOfType.class);
+ Map<String, FieldValueTypeInformation> oneOfTypes = Maps.newHashMap();
+ for (Field oneOfField : oneOfType.getOneOfSchema().getFields()) {
+ Method method = getProtoGetter(methods, oneOfField.getName(), oneOfField.getType());
+ oneOfTypes.put(
+ oneOfField.getName(),
+ FieldValueTypeInformation.forGetter(method).withName(field.getName()));
+ }
+ // Add an entry that encapsulates information about all possible getters.
+ types.add(
+ FieldValueTypeInformation.forOneOf(
+ field.getName(), field.getType().getNullable(), oneOfTypes)
+ .withName(field.getName()));
+ } else {
+ // This is a simple field. Add the getter.
+ Method method = getProtoGetter(methods, field.getName(), field.getType());
+ types.add(FieldValueTypeInformation.forGetter(method).withName(field.getName()));
+ }
+ }
+ return types;
+ }
+ }
+
+ @Nullable
+ @Override
+ public <T> Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
+ checkForDynamicType(typeDescriptor);
+ return ProtoSchemaTranslator.getSchema((Class<Message>) typeDescriptor.getRawType());
+ }
+
+ @Override
+ public List<FieldValueGetter> fieldValueGetters(Class<?> targetClass, Schema schema) {
+ return ProtoByteBuddyUtils.getGetters(
+ targetClass,
+ schema,
+ new ProtoClassFieldValueTypeSupplier(),
+ new ProtoTypeConversionsFactory());
+ }
+
+ @Override
+ public List<FieldValueTypeInformation> fieldValueTypeInformations(
+ Class<?> targetClass, Schema schema) {
+ return JavaBeanUtils.getFieldTypes(targetClass, schema, new ProtoClassFieldValueTypeSupplier());
+ }
+
+ @Override
+ public SchemaUserTypeCreator schemaTypeCreator(Class<?> targetClass, Schema schema) {
+ SchemaUserTypeCreator creator =
+ ProtoByteBuddyUtils.getBuilderCreator(
+ targetClass, schema, new ProtoClassFieldValueTypeSupplier());
+ if (creator == null) {
+ throw new RuntimeException("Cannot create creator for " + targetClass);
+ }
+ return creator;
+ }
+
+ private <T> void checkForDynamicType(TypeDescriptor<T> typeDescriptor) {
+ if (typeDescriptor.getRawType().equals(DynamicMessage.class)) {
+ throw new RuntimeException(
+ "DynamicMessage is not allowed for the standard ProtoSchemaProvider, use ProtoDynamicMessageSchema instead.");
+ }
+ }
+}
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java
new file mode 100644
index 0000000..0d4a5a6
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java
@@ -0,0 +1,186 @@
+/*
+ * 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.extensions.protobuf;
+
+import com.google.protobuf.Duration;
+import com.google.protobuf.Timestamp;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.LogicalType;
+import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
+import org.apache.beam.sdk.values.Row;
+
+/** A set of {@link LogicalType} classes to represent protocol buffer types. */
+public class ProtoSchemaLogicalTypes {
+ /** Base class for types representing timestamps or durations as nanoseconds. */
+ public abstract static class NanosType<T> implements LogicalType<T, Row> {
+ private final String identifier;
+
+ private static final Schema SCHEMA =
+ Schema.builder().addInt64Field("seconds").addInt32Field("nanos").build();
+
+ protected NanosType(String identifier) {
+ this.identifier = identifier;
+ }
+
+ @Override
+ public String getIdentifier() {
+ return identifier;
+ }
+
+ @Override
+ public FieldType getArgumentType() {
+ return FieldType.STRING;
+ }
+
+ @Override
+ public FieldType getBaseType() {
+ return FieldType.row(SCHEMA);
+ }
+ }
+
+ /** A timestamp represented as nanoseconds since the epoch. */
+ public static class TimestampNanos extends NanosType<Timestamp> {
+ public static final String IDENTIFIER = "ProtoTimestamp";
+
+ public TimestampNanos() {
+ super(IDENTIFIER);
+ }
+
+ @Override
+ public Row toBaseType(Timestamp input) {
+ return toRow(input);
+ }
+
+ @Override
+ public Timestamp toInputType(Row base) {
+ return toTimestamp(base);
+ }
+
+ public static Row toRow(Timestamp input) {
+ return Row.withSchema(NanosType.SCHEMA)
+ .addValues(input.getSeconds(), input.getNanos())
+ .build();
+ }
+
+ public static Timestamp toTimestamp(Row row) {
+ return Timestamp.newBuilder().setSeconds(row.getInt64(0)).setNanos(row.getInt32(1)).build();
+ }
+ }
+
+ /** A duration represented in nanoseconds. */
+ public static class DurationNanos extends NanosType<Duration> {
+ public static final String IDENTIFIER = "ProtoTimestamp";
+
+ public DurationNanos() {
+ super(IDENTIFIER);
+ }
+
+ @Override
+ public Row toBaseType(Duration input) {
+ return toRow(input);
+ }
+
+ @Override
+ public Duration toInputType(Row base) {
+ return toDuration(base);
+ }
+
+ public static Row toRow(Duration input) {
+ return Row.withSchema(NanosType.SCHEMA)
+ .addValues(input.getSeconds(), input.getNanos())
+ .build();
+ }
+
+ public static Duration toDuration(Row row) {
+ return Duration.newBuilder().setSeconds(row.getInt64(0)).setNanos(row.getInt32(1)).build();
+ }
+ }
+
+ /** A UInt32 type. */
+ public static class UInt32 extends PassThroughLogicalType<Integer> {
+ public static final String IDENTIFIER = "Uint32";
+
+ UInt32() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT32);
+ }
+ }
+
+ /** A SInt32 type. */
+ public static class SInt32 extends PassThroughLogicalType<Integer> {
+ public static final String IDENTIFIER = "Sint32";
+
+ SInt32() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT32);
+ }
+ }
+
+ /** A Fixed32 type. */
+ public static class Fixed32 extends PassThroughLogicalType<Integer> {
+ public static final String IDENTIFIER = "Fixed32";
+
+ Fixed32() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT32);
+ }
+ }
+
+ /** A SFixed32 type. */
+ public static class SFixed32 extends PassThroughLogicalType<Integer> {
+ public static final String IDENTIFIER = "SFixed32";
+
+ SFixed32() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT32);
+ }
+ }
+
+ /** A UIn64 type. */
+ public static class UInt64 extends PassThroughLogicalType<Long> {
+ public static final String IDENTIFIER = "Uint64";
+
+ UInt64() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT64);
+ }
+ }
+
+ /** A SIn64 type. */
+ public static class SInt64 extends PassThroughLogicalType<Long> {
+ public static final String IDENTIFIER = "Sint64";
+
+ SInt64() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT64);
+ }
+ }
+
+ /** A Fixed64 type. */
+ public static class Fixed64 extends PassThroughLogicalType<Long> {
+ public static final String IDENTIFIER = "Fixed64";
+
+ Fixed64() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT64);
+ }
+ }
+
+ /** An SFixed64 type. */
+ public static class SFixed64 extends PassThroughLogicalType<Long> {
+ public static final String IDENTIFIER = "SFixed64";
+
+ SFixed64() {
+ super(IDENTIFIER, FieldType.STRING, "", FieldType.INT64);
+ }
+ }
+}
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java
new file mode 100644
index 0000000..d27f480
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslator.java
@@ -0,0 +1,280 @@
+/*
+ * 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.extensions.protobuf;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.EnumValueDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.OneofDescriptor;
+import com.google.protobuf.Message;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.DurationNanos;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.Fixed32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.Fixed64;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SFixed32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SFixed64;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SInt32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SInt64;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.TimestampNanos;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.UInt32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.UInt64;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
+
+/**
+ * This class provides utilities for inferring a Beam schema from a protocol buffer.
+ *
+ * <p>The following proto primitive types map to the following Beam types:
+ *
+ * <ul>
+ * <li>INT32 maps to FieldType.INT32
+ * <li>INT64 maps to FieldType.INT64
+ * <li>FLOAT maps to FieldType.FLOAT
+ * <li>DOUBLE maps to FieldType.DOUBLE
+ * <li>BOOL maps to FieldType.BOOLEAN
+ * <li>STRING maps to FieldType.STRING
+ * <li>BYTES maps to FieldType.BYTES
+ * </ul>
+ *
+ * <p>The following proto numeric types do not have have native Beam primitive types. LogicalType
+ * objects were created to represent these types. Normal numeric types are used as the base type of
+ * each of these logical types, so SQL queries should work as normal.
+ *
+ * <ul>
+ * <li>UINT32 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.UInt32()))
+ * <li>SINT32 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.SInt32()))
+ * <li>FIXED32 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.Fixed32()))
+ * <li>SFIXED32 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.SFixed32()))
+ * <li>UINT64 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.UInt64()))
+ * <li>SINT64 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.SInt64()))
+ * <li>FIXED64 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.Fixed64()))
+ * <li>SFIXED64 maps to FieldType.logicalType(new ProtoSchemaLogicalTypes.SFixed64()))
+ * </ul>
+ *
+ * <p>Protobuf maps are mapped to Beam FieldType.MAP types. Protobuf repeated fields are mapped to
+ * Beam FieldType.ARRAY types.
+ *
+ * <p>Beam schemas include the EnumerationType logical type to represent enumerations, and protobuf
+ * enumerations are translated to this logical type. The base representation type for this logical
+ * type is an INT32.
+ *
+ * <p>Beam schemas include the OneOfType logical type to represent unions, and protobuf oneOfs are
+ * translated to this logical type. The base representation type for this logical type is a subrow
+ * containing an optional field for each oneof option.
+ *
+ * <p>google.com.protobuf.Timestamp messages cannot be translated to FieldType.DATETIME types, as
+ * the proto type represents nanonseconds and Beam's native type does not currently support that. a
+ * new TimestampNanos logical type has been introduced to allow representing nanosecond timestamp,
+ * as well as a DurationNanos logical type to represent google.com.protobuf.Duration types.
+ *
+ * <p>Protobuf wrapper classes are translated to nullable types, as follows.
+ *
+ * <ul>
+ * <li>google.protobuf.Int32Value maps to a nullable FieldType.INT32
+ * <li>google.protobuf.Int64Value maps to a nullable FieldType.INT64
+ * <li>google.protobuf.UInt32Value maps to a nullable FieldType.logicalType(new UInt32())
+ * <li>google.protobuf.UInt64Value maps to a nullable Field.logicalType(new UInt64())
+ * <li>google.protobuf.FloatValue maps to a nullable FieldType.FLOAT
+ * <li>google.protobuf.DoubleValue maps to a nullable FieldType.DOUBLE
+ * <li>google.protobuf.BoolValue maps to a nullable FieldType.BOOLEAN
+ * <li>google.protobuf.StringValue maps to a nullable FieldType.STRING
+ * <li>google.protobuf.BytesValue maps to a nullable FieldType.BYTES
+ * </ul>
+ */
+@Experimental(Experimental.Kind.SCHEMAS)
+public class ProtoSchemaTranslator {
+ /** This METADATA tag is used to store the field number of a proto tag. */
+ public static final String PROTO_NUMBER_METADATA_TAG = "PROTO_NUMBER";
+
+ /** Attach a proto field number to a type. */
+ public static FieldType withFieldNumber(FieldType fieldType, int index) {
+ return fieldType.withMetadata(PROTO_NUMBER_METADATA_TAG, Long.toString(index));
+ }
+
+ /** Return the proto field number for a type. */
+ public static int getFieldNumber(FieldType fieldType) {
+ return Integer.parseInt(fieldType.getMetadataString(PROTO_NUMBER_METADATA_TAG));
+ }
+
+ /** Return a Beam scheam representing a proto class. */
+ public static Schema getSchema(Class<? extends Message> clazz) {
+ return getSchema(ProtobufUtil.getDescriptorForClass(clazz));
+ }
+
+ private static Schema getSchema(Descriptors.Descriptor descriptor) {
+ Set<Integer> oneOfFields = Sets.newHashSet();
+ List<Field> fields = Lists.newArrayListWithCapacity(descriptor.getFields().size());
+ for (OneofDescriptor oneofDescriptor : descriptor.getOneofs()) {
+ List<Field> subFields = Lists.newArrayListWithCapacity(oneofDescriptor.getFieldCount());
+ Map<String, Integer> enumIds = Maps.newHashMap();
+ for (FieldDescriptor fieldDescriptor : oneofDescriptor.getFields()) {
+ oneOfFields.add(fieldDescriptor.getNumber());
+ // Store proto field number in metadata.
+ FieldType fieldType =
+ withFieldNumber(
+ beamFieldTypeFromProtoField(fieldDescriptor), fieldDescriptor.getNumber());
+ subFields.add(Field.nullable(fieldDescriptor.getName(), fieldType));
+ checkArgument(
+ enumIds.putIfAbsent(fieldDescriptor.getName(), fieldDescriptor.getNumber()) == null);
+ }
+ FieldType oneOfType = FieldType.logicalType(OneOfType.create(subFields, enumIds));
+ fields.add(Field.of(oneofDescriptor.getName(), oneOfType));
+ }
+
+ for (Descriptors.FieldDescriptor fieldDescriptor : descriptor.getFields()) {
+ if (!oneOfFields.contains(fieldDescriptor.getNumber())) {
+ // Store proto field number in metadata.
+ FieldType fieldType =
+ withFieldNumber(
+ beamFieldTypeFromProtoField(fieldDescriptor), fieldDescriptor.getNumber());
+ fields.add(Field.of(fieldDescriptor.getName(), fieldType));
+ }
+ }
+ return Schema.builder().addFields(fields).build();
+ }
+
+ private static FieldType beamFieldTypeFromProtoField(
+ Descriptors.FieldDescriptor protoFieldDescriptor) {
+ FieldType fieldType = null;
+ if (protoFieldDescriptor.isMapField()) {
+ FieldDescriptor keyFieldDescriptor =
+ protoFieldDescriptor.getMessageType().findFieldByName("key");
+ FieldDescriptor valueFieldDescriptor =
+ protoFieldDescriptor.getMessageType().findFieldByName("value");
+ fieldType =
+ FieldType.map(
+ beamFieldTypeFromProtoField(keyFieldDescriptor),
+ beamFieldTypeFromProtoField(valueFieldDescriptor));
+ } else if (protoFieldDescriptor.isRepeated()) {
+ fieldType = FieldType.array(beamFieldTypeFromSingularProtoField(protoFieldDescriptor));
+ } else {
+ fieldType = beamFieldTypeFromSingularProtoField(protoFieldDescriptor);
+ }
+ return fieldType;
+ }
+
+ private static FieldType beamFieldTypeFromSingularProtoField(
+ Descriptors.FieldDescriptor protoFieldDescriptor) {
+ Descriptors.FieldDescriptor.Type fieldDescriptor = protoFieldDescriptor.getType();
+ FieldType fieldType;
+ switch (fieldDescriptor) {
+ case INT32:
+ fieldType = FieldType.INT32;
+ break;
+ case INT64:
+ fieldType = FieldType.INT64;
+ break;
+ case FLOAT:
+ fieldType = FieldType.FLOAT;
+ break;
+ case DOUBLE:
+ fieldType = FieldType.DOUBLE;
+ break;
+ case BOOL:
+ fieldType = FieldType.BOOLEAN;
+ break;
+ case STRING:
+ fieldType = FieldType.STRING;
+ break;
+ case BYTES:
+ fieldType = FieldType.BYTES;
+ break;
+ case UINT32:
+ fieldType = FieldType.logicalType(new UInt32());
+ break;
+ case SINT32:
+ fieldType = FieldType.logicalType(new SInt32());
+ break;
+ case FIXED32:
+ fieldType = FieldType.logicalType(new Fixed32());
+ break;
+ case SFIXED32:
+ fieldType = FieldType.logicalType(new SFixed32());
+ break;
+ case UINT64:
+ fieldType = FieldType.logicalType(new UInt64());
+ break;
+ case SINT64:
+ fieldType = FieldType.logicalType(new SInt64());
+ break;
+ case FIXED64:
+ fieldType = FieldType.logicalType(new Fixed64());
+ break;
+ case SFIXED64:
+ fieldType = FieldType.logicalType(new SFixed64());
+ break;
+
+ case ENUM:
+ Map<String, Integer> enumValues = Maps.newHashMap();
+ for (EnumValueDescriptor enumValue : protoFieldDescriptor.getEnumType().getValues()) {
+ if (enumValues.putIfAbsent(enumValue.getName(), enumValue.getNumber()) != null) {
+ throw new RuntimeException("Aliased enumerations not currently supported.");
+ }
+ }
+ fieldType = FieldType.logicalType(EnumerationType.create(enumValues));
+ break;
+ case MESSAGE:
+ case GROUP:
+ String fullName = protoFieldDescriptor.getMessageType().getFullName();
+ switch (fullName) {
+ case "google.protobuf.Timestamp":
+ fieldType = FieldType.logicalType(new TimestampNanos());
+ break;
+ case "google.protobuf.Int32Value":
+ case "google.protobuf.UInt32Value":
+ case "google.protobuf.Int64Value":
+ case "google.protobuf.UInt64Value":
+ case "google.protobuf.FloatValue":
+ case "google.protobuf.DoubleValue":
+ case "google.protobuf.StringValue":
+ case "google.protobuf.BoolValue":
+ case "google.protobuf.BytesValue":
+ fieldType =
+ beamFieldTypeFromSingularProtoField(
+ protoFieldDescriptor.getMessageType().findFieldByNumber(1))
+ .withNullable(true);
+ break;
+ case "google.protobuf.Duration":
+ fieldType = FieldType.logicalType(new DurationNanos());
+ break;
+ case "google.protobuf.Any":
+ throw new RuntimeException("Any not yet supported");
+ default:
+ fieldType = FieldType.row(getSchema(protoFieldDescriptor.getMessageType()));
+ }
+ break;
+ default:
+ throw new RuntimeException("Field type not matched.");
+ }
+ if (protoFieldDescriptor.isOptional()) {
+ fieldType = fieldType.withNullable(true);
+ }
+ return fieldType;
+ }
+}
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java
new file mode 100644
index 0000000..8952f9a
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java
@@ -0,0 +1,249 @@
+/*
+ * 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.extensions.protobuf;
+
+import static org.apache.beam.sdk.extensions.protobuf.ProtoSchemaTranslator.withFieldNumber;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.MAP_PRIMITIVE_PROTO;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.MAP_PRIMITIVE_ROW;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.MAP_PRIMITIVE_SCHEMA;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.NESTED_PROTO;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.NESTED_ROW;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.NESTED_SCHEMA;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_PROTO_BOOL;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_PROTO_INT32;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_PROTO_PRIMITIVE;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_PROTO_STRING;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_ROW_BOOL;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_ROW_INT32;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_ROW_PRIMITIVE;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_ROW_STRING;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.ONEOF_SCHEMA;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.OUTER_ONEOF_PROTO;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.OUTER_ONEOF_ROW;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.OUTER_ONEOF_SCHEMA;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.PRIMITIVE_PROTO;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.PRIMITIVE_ROW;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.PRIMITIVE_SCHEMA;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.REPEATED_PROTO;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.REPEATED_ROW;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.REPEATED_SCHEMA;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.WKT_MESSAGE_PROTO;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.WKT_MESSAGE_ROW;
+import static org.apache.beam.sdk.extensions.protobuf.TestProtoSchemas.WKT_MESSAGE_SCHEMA;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.EnumMessage;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.EnumMessage.Enum;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.MapPrimitive;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Nested;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.OneOf;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.OuterOneOf;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Primitive;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.RepeatPrimitive;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.WktMessage;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class ProtoMessageSchemaTest {
+
+ @Test
+ public void testPrimitiveSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(Primitive.class));
+ assertEquals(PRIMITIVE_SCHEMA, schema);
+ }
+
+ @Test
+ public void testPrimitiveProtoToRow() {
+ SerializableFunction<Primitive, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(Primitive.class));
+ assertEquals(PRIMITIVE_ROW, toRow.apply(PRIMITIVE_PROTO));
+ }
+
+ @Test
+ public void testPrimitiveRowToProto() {
+ SerializableFunction<Row, Primitive> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(Primitive.class));
+ assertEquals(PRIMITIVE_PROTO, fromRow.apply(PRIMITIVE_ROW));
+ }
+
+ @Test
+ public void testRepeatedSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(RepeatPrimitive.class));
+ assertEquals(REPEATED_SCHEMA, schema);
+ }
+
+ @Test
+ public void testRepeatedProtoToRow() {
+ SerializableFunction<RepeatPrimitive, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(RepeatPrimitive.class));
+ assertEquals(REPEATED_ROW, toRow.apply(REPEATED_PROTO));
+ }
+
+ @Test
+ public void testRepeatedRowToProto() {
+ SerializableFunction<Row, RepeatPrimitive> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(RepeatPrimitive.class));
+ assertEquals(REPEATED_PROTO, fromRow.apply(REPEATED_ROW));
+ }
+
+ // Test map type
+ @Test
+ public void testMapSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(MapPrimitive.class));
+ assertEquals(MAP_PRIMITIVE_SCHEMA, schema);
+ }
+
+ @Test
+ public void testMapProtoToRow() {
+ SerializableFunction<MapPrimitive, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(MapPrimitive.class));
+ assertEquals(MAP_PRIMITIVE_ROW, toRow.apply(MAP_PRIMITIVE_PROTO));
+ }
+
+ @Test
+ public void testMapRowToProto() {
+ SerializableFunction<Row, MapPrimitive> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(MapPrimitive.class));
+ assertEquals(MAP_PRIMITIVE_PROTO, fromRow.apply(MAP_PRIMITIVE_ROW));
+ }
+
+ @Test
+ public void testNestedSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(Nested.class));
+ assertEquals(NESTED_SCHEMA, schema);
+ }
+
+ @Test
+ public void testNestedProtoToRow() {
+ SerializableFunction<Nested, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(Nested.class));
+ assertEquals(NESTED_ROW, toRow.apply(NESTED_PROTO));
+ }
+
+ @Test
+ public void testNestedRowToProto() {
+ SerializableFunction<Row, Nested> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(Nested.class));
+ assertEquals(NESTED_PROTO, fromRow.apply(NESTED_ROW));
+ }
+
+ @Test
+ public void testOneOfSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(OneOf.class));
+ assertEquals(ONEOF_SCHEMA, schema);
+ }
+
+ @Test
+ public void testOneOfProtoToRow() {
+ SerializableFunction<OneOf, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(OneOf.class));
+ assertEquals(ONEOF_ROW_INT32, toRow.apply(ONEOF_PROTO_INT32));
+ assertEquals(ONEOF_ROW_BOOL, toRow.apply(ONEOF_PROTO_BOOL));
+ assertEquals(ONEOF_ROW_STRING, toRow.apply(ONEOF_PROTO_STRING));
+ assertEquals(ONEOF_ROW_PRIMITIVE, toRow.apply(ONEOF_PROTO_PRIMITIVE));
+ }
+
+ @Test
+ public void testOneOfRowToProto() {
+ SerializableFunction<Row, OneOf> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(OneOf.class));
+ assertEquals(ONEOF_PROTO_INT32, fromRow.apply(ONEOF_ROW_INT32));
+ assertEquals(ONEOF_PROTO_BOOL, fromRow.apply(ONEOF_ROW_BOOL));
+ assertEquals(ONEOF_PROTO_STRING, fromRow.apply(ONEOF_ROW_STRING));
+ assertEquals(ONEOF_PROTO_PRIMITIVE, fromRow.apply(ONEOF_ROW_PRIMITIVE));
+ }
+
+ @Test
+ public void testOuterOneOfSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(OuterOneOf.class));
+ assertEquals(OUTER_ONEOF_SCHEMA, schema);
+ }
+
+ @Test
+ public void testOuterOneOfProtoToRow() {
+ SerializableFunction<OuterOneOf, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(OuterOneOf.class));
+ assertEquals(OUTER_ONEOF_ROW, toRow.apply(OUTER_ONEOF_PROTO));
+ }
+
+ @Test
+ public void testOuterOneOfRowToProto() {
+ SerializableFunction<Row, OuterOneOf> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(OuterOneOf.class));
+ assertEquals(OUTER_ONEOF_PROTO, fromRow.apply(OUTER_ONEOF_ROW));
+ }
+
+ private static final EnumerationType ENUM_TYPE =
+ EnumerationType.create(ImmutableMap.of("ZERO", 0, "TWO", 2, "THREE", 3));
+ private static final Schema ENUM_SCHEMA =
+ Schema.builder()
+ .addField("enum", withFieldNumber(FieldType.logicalType(ENUM_TYPE).withNullable(true), 1))
+ .build();
+ private static final Row ENUM_ROW =
+ Row.withSchema(ENUM_SCHEMA).addValues(ENUM_TYPE.valueOf("TWO")).build();
+ private static final EnumMessage ENUM_PROTO = EnumMessage.newBuilder().setEnum(Enum.TWO).build();
+
+ @Test
+ public void testEnumSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(EnumMessage.class));
+ assertEquals(ENUM_SCHEMA, schema);
+ }
+
+ @Test
+ public void testEnumProtoToRow() {
+ SerializableFunction<EnumMessage, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(EnumMessage.class));
+ assertEquals(ENUM_ROW, toRow.apply(ENUM_PROTO));
+ }
+
+ @Test
+ public void testEnumRowToProto() {
+ SerializableFunction<Row, EnumMessage> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(EnumMessage.class));
+ assertEquals(ENUM_PROTO, fromRow.apply(ENUM_ROW));
+ }
+
+ @Test
+ public void testWktMessageSchema() {
+ Schema schema = new ProtoMessageSchema().schemaFor(TypeDescriptor.of(WktMessage.class));
+ assertEquals(WKT_MESSAGE_SCHEMA, schema);
+ }
+
+ @Test
+ public void testWktProtoToRow() {
+ SerializableFunction<WktMessage, Row> toRow =
+ new ProtoMessageSchema().toRowFunction(TypeDescriptor.of(WktMessage.class));
+ assertEquals(WKT_MESSAGE_ROW, toRow.apply(WKT_MESSAGE_PROTO));
+ }
+
+ @Test
+ public void testWktRowToProto() {
+ SerializableFunction<Row, WktMessage> fromRow =
+ new ProtoMessageSchema().fromRowFunction(TypeDescriptor.of(WktMessage.class));
+ assertEquals(WKT_MESSAGE_PROTO, fromRow.apply(WKT_MESSAGE_ROW));
+ }
+}
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java
new file mode 100644
index 0000000..34ceb10
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.extensions.protobuf;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Collection of standard tests for Protobuf Schema support. */
+@RunWith(JUnit4.class)
+public class ProtoSchemaTranslatorTest {
+ @Test
+ public void testPrimitiveSchema() {
+ assertEquals(
+ TestProtoSchemas.PRIMITIVE_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.Primitive.class));
+ }
+
+ @Test
+ public void testRepeatedSchema() {
+ assertEquals(
+ TestProtoSchemas.REPEATED_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.RepeatPrimitive.class));
+ }
+
+ @Test
+ public void testMapPrimitiveSchema() {
+ assertEquals(
+ TestProtoSchemas.MAP_PRIMITIVE_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.MapPrimitive.class));
+ }
+
+ @Test
+ public void testNestedSchema() {
+ assertEquals(
+ TestProtoSchemas.NESTED_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.Nested.class));
+ }
+
+ @Test
+ public void testOneOfSchema() {
+ assertEquals(
+ TestProtoSchemas.ONEOF_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.OneOf.class));
+ }
+
+ @Test
+ public void testNestedOneOfSchema() {
+ assertEquals(
+ TestProtoSchemas.OUTER_ONEOF_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.OuterOneOf.class));
+ }
+
+ @Test
+ public void testWrapperMessagesSchema() {
+ assertEquals(
+ TestProtoSchemas.WKT_MESSAGE_SCHEMA,
+ ProtoSchemaTranslator.getSchema(Proto3SchemaMessages.WktMessage.class));
+ }
+}
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/TestProtoSchemas.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/TestProtoSchemas.java
new file mode 100644
index 0000000..88892d8
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/TestProtoSchemas.java
@@ -0,0 +1,397 @@
+/*
+ * 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.extensions.protobuf;
+
+import static org.apache.beam.sdk.extensions.protobuf.ProtoSchemaTranslator.getFieldNumber;
+import static org.apache.beam.sdk.extensions.protobuf.ProtoSchemaTranslator.withFieldNumber;
+
+import com.google.protobuf.BoolValue;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.Duration;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int32Value;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.StringValue;
+import com.google.protobuf.Timestamp;
+import com.google.protobuf.UInt32Value;
+import com.google.protobuf.UInt64Value;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.MapPrimitive;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Nested;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.OneOf;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.OuterOneOf;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.Primitive;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.RepeatPrimitive;
+import org.apache.beam.sdk.extensions.protobuf.Proto3SchemaMessages.WktMessage;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.DurationNanos;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.Fixed32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.Fixed64;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SFixed32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SFixed64;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SInt32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.SInt64;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.TimestampNanos;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.UInt32;
+import org.apache.beam.sdk.extensions.protobuf.ProtoSchemaLogicalTypes.UInt64;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+class TestProtoSchemas {
+ // The schema we expect from the Primitive proto.
+ static final Schema PRIMITIVE_SCHEMA =
+ Schema.builder()
+ .addNullableField("primitive_double", withFieldNumber(FieldType.DOUBLE, 1))
+ .addNullableField("primitive_float", withFieldNumber(FieldType.FLOAT, 2))
+ .addNullableField("primitive_int32", withFieldNumber(FieldType.INT32, 3))
+ .addNullableField("primitive_int64", withFieldNumber(FieldType.INT64, 4))
+ .addNullableField(
+ "primitive_uint32", withFieldNumber(FieldType.logicalType(new UInt32()), 5))
+ .addNullableField(
+ "primitive_uint64", withFieldNumber(FieldType.logicalType(new UInt64()), 6))
+ .addNullableField(
+ "primitive_sint32", withFieldNumber(FieldType.logicalType(new SInt32()), 7))
+ .addNullableField(
+ "primitive_sint64", withFieldNumber(FieldType.logicalType(new SInt64()), 8))
+ .addNullableField(
+ "primitive_fixed32", withFieldNumber(FieldType.logicalType(new Fixed32()), 9))
+ .addNullableField(
+ "primitive_fixed64", withFieldNumber(FieldType.logicalType(new Fixed64()), 10))
+ .addNullableField(
+ "primitive_sfixed32", withFieldNumber(FieldType.logicalType(new SFixed32()), 11))
+ .addNullableField(
+ "primitive_sfixed64", withFieldNumber(FieldType.logicalType(new SFixed64()), 12))
+ .addNullableField("primitive_bool", withFieldNumber(FieldType.BOOLEAN, 13))
+ .addNullableField("primitive_string", withFieldNumber(FieldType.STRING, 14))
+ .addNullableField("primitive_bytes", withFieldNumber(FieldType.BYTES, 15))
+ .build();
+
+ // A sample instance of the row.
+ private static final byte[] BYTE_ARRAY = new byte[] {1, 2, 3, 4};
+ static final Row PRIMITIVE_ROW =
+ Row.withSchema(PRIMITIVE_SCHEMA)
+ .addValues(
+ 1.1, 2.2F, 32, 64L, 33, 65L, 123, 124L, 30, 62L, 31, 63L, true, "horsey", BYTE_ARRAY)
+ .build();
+
+ // A sample instance of the proto.
+ static final Primitive PRIMITIVE_PROTO =
+ Primitive.newBuilder()
+ .setPrimitiveDouble(1.1)
+ .setPrimitiveFloat(2.2F)
+ .setPrimitiveInt32(32)
+ .setPrimitiveInt64(64)
+ .setPrimitiveUint32(33)
+ .setPrimitiveUint64(65)
+ .setPrimitiveSint32(123)
+ .setPrimitiveSint64(124)
+ .setPrimitiveFixed32(30)
+ .setPrimitiveFixed64(62)
+ .setPrimitiveSfixed32(31)
+ .setPrimitiveSfixed64(63)
+ .setPrimitiveBool(true)
+ .setPrimitiveString("horsey")
+ .setPrimitiveBytes(ByteString.copyFrom(BYTE_ARRAY))
+ .build();
+
+ // The schema for the RepeatedPrimitive proto.
+ static final Schema REPEATED_SCHEMA =
+ Schema.builder()
+ .addField("repeated_double", withFieldNumber(FieldType.array(FieldType.DOUBLE), 1))
+ .addField("repeated_float", withFieldNumber(FieldType.array(FieldType.FLOAT), 2))
+ .addField("repeated_int32", withFieldNumber(FieldType.array(FieldType.INT32), 3))
+ .addField("repeated_int64", withFieldNumber(FieldType.array(FieldType.INT64), 4))
+ .addField(
+ "repeated_uint32",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new UInt32())), 5))
+ .addField(
+ "repeated_uint64",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new UInt64())), 6))
+ .addField(
+ "repeated_sint32",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new SInt32())), 7))
+ .addField(
+ "repeated_sint64",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new SInt64())), 8))
+ .addField(
+ "repeated_fixed32",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new Fixed32())), 9))
+ .addField(
+ "repeated_fixed64",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new Fixed64())), 10))
+ .addField(
+ "repeated_sfixed32",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new SFixed32())), 11))
+ .addField(
+ "repeated_sfixed64",
+ withFieldNumber(FieldType.array(FieldType.logicalType(new SFixed64())), 12))
+ .addField("repeated_bool", withFieldNumber(FieldType.array(FieldType.BOOLEAN), 13))
+ .addField("repeated_string", withFieldNumber(FieldType.array(FieldType.STRING), 14))
+ .addField("repeated_bytes", withFieldNumber(FieldType.array(FieldType.BYTES), 15))
+ .build();
+
+ // A sample instance of the row.
+ static final Row REPEATED_ROW =
+ Row.withSchema(REPEATED_SCHEMA)
+ .addArray(1.1, 1.1)
+ .addArray(2.2F, 2.2F)
+ .addArray(32, 32)
+ .addArray(64L, 64L)
+ .addArray(33, 33)
+ .addArray(65L, 65L)
+ .addArray(123, 123)
+ .addArray(124L, 124L)
+ .addArray(30, 30)
+ .addArray(62L, 62L)
+ .addArray(31, 31)
+ .addArray(63L, 63L)
+ .addArray(true, true)
+ .addArray("horsey", "horsey")
+ .addArray(BYTE_ARRAY, BYTE_ARRAY)
+ .build();
+
+ // A sample instance of the proto.
+ static final RepeatPrimitive REPEATED_PROTO =
+ RepeatPrimitive.newBuilder()
+ .addAllRepeatedDouble(ImmutableList.of(1.1, 1.1))
+ .addAllRepeatedFloat(ImmutableList.of(2.2F, 2.2F))
+ .addAllRepeatedInt32(ImmutableList.of(32, 32))
+ .addAllRepeatedInt64(ImmutableList.of(64L, 64L))
+ .addAllRepeatedUint32(ImmutableList.of(33, 33))
+ .addAllRepeatedUint64(ImmutableList.of(65L, 65L))
+ .addAllRepeatedSint32(ImmutableList.of(123, 123))
+ .addAllRepeatedSint64(ImmutableList.of(124L, 124L))
+ .addAllRepeatedFixed32(ImmutableList.of(30, 30))
+ .addAllRepeatedFixed64(ImmutableList.of(62L, 62L))
+ .addAllRepeatedSfixed32(ImmutableList.of(31, 31))
+ .addAllRepeatedSfixed64(ImmutableList.of(63L, 63L))
+ .addAllRepeatedBool(ImmutableList.of(true, true))
+ .addAllRepeatedString(ImmutableList.of("horsey", "horsey"))
+ .addAllRepeatedBytes(
+ ImmutableList.of(ByteString.copyFrom(BYTE_ARRAY), ByteString.copyFrom(BYTE_ARRAY)))
+ .build();
+
+ // The schema for the MapPrimitive proto.
+ static final Schema MAP_PRIMITIVE_SCHEMA =
+ Schema.builder()
+ .addField(
+ "string_string_map",
+ withFieldNumber(
+ FieldType.map(
+ FieldType.STRING.withNullable(true), FieldType.STRING.withNullable(true)),
+ 1))
+ .addField(
+ "string_int_map",
+ withFieldNumber(
+ FieldType.map(
+ FieldType.STRING.withNullable(true), FieldType.INT32.withNullable(true)),
+ 2))
+ .addField(
+ "int_string_map",
+ withFieldNumber(
+ FieldType.map(
+ FieldType.INT32.withNullable(true), FieldType.STRING.withNullable(true)),
+ 3))
+ .addField(
+ "string_bytes_map",
+ withFieldNumber(
+ FieldType.map(
+ FieldType.STRING.withNullable(true), FieldType.BYTES.withNullable(true)),
+ 4))
+ .build();
+
+ // A sample instance of the row.
+ static final Row MAP_PRIMITIVE_ROW =
+ Row.withSchema(MAP_PRIMITIVE_SCHEMA)
+ .addValue(ImmutableMap.of("k1", "v1", "k2", "v2"))
+ .addValue(ImmutableMap.of("k1", 1, "k2", 2))
+ .addValue(ImmutableMap.of(1, "v1", 2, "v2"))
+ .addValue(ImmutableMap.of("k1", BYTE_ARRAY, "k2", BYTE_ARRAY))
+ .build();
+
+ // A sample instance of the proto.
+ static final MapPrimitive MAP_PRIMITIVE_PROTO =
+ MapPrimitive.newBuilder()
+ .putAllStringStringMap(ImmutableMap.of("k1", "v1", "k2", "v2"))
+ .putAllStringIntMap(ImmutableMap.of("k1", 1, "k2", 2))
+ .putAllIntStringMap(ImmutableMap.of(1, "v1", 2, "v2"))
+ .putAllStringBytesMap(
+ ImmutableMap.of(
+ "k1", ByteString.copyFrom(BYTE_ARRAY), "k2", ByteString.copyFrom(BYTE_ARRAY)))
+ .build();
+
+ // The schema for the Nested proto.
+ static final Schema NESTED_SCHEMA =
+ Schema.builder()
+ .addField(
+ "nested", withFieldNumber(FieldType.row(PRIMITIVE_SCHEMA).withNullable(true), 1))
+ .addField(
+ "nested_list", withFieldNumber(FieldType.array(FieldType.row(PRIMITIVE_SCHEMA)), 2))
+ .addField(
+ "nested_map",
+ withFieldNumber(
+ FieldType.map(
+ FieldType.STRING.withNullable(true),
+ FieldType.row(PRIMITIVE_SCHEMA).withNullable(true)),
+ 3))
+ .build();
+
+ // A sample instance of the row.
+ static final Row NESTED_ROW =
+ Row.withSchema(NESTED_SCHEMA)
+ .addValue(PRIMITIVE_ROW)
+ .addArray(ImmutableList.of(PRIMITIVE_ROW, PRIMITIVE_ROW))
+ .addValue(ImmutableMap.of("k1", PRIMITIVE_ROW, "k2", PRIMITIVE_ROW))
+ .build();
+
+ // A sample instance of the proto.
+ static final Nested NESTED_PROTO =
+ Nested.newBuilder()
+ .setNested(PRIMITIVE_PROTO)
+ .addAllNestedList(ImmutableList.of(PRIMITIVE_PROTO, PRIMITIVE_PROTO))
+ .putAllNestedMap(ImmutableMap.of("k1", PRIMITIVE_PROTO, "k2", PRIMITIVE_PROTO))
+ .build();
+
+ // The schema for the OneOf proto.
+ private static final List<Field> ONEOF_FIELDS =
+ ImmutableList.of(
+ Field.of("oneof_int32", withFieldNumber(FieldType.INT32, 2)),
+ Field.of("oneof_bool", withFieldNumber(FieldType.BOOLEAN, 3)),
+ Field.of("oneof_string", withFieldNumber(FieldType.STRING, 4)),
+ Field.of("oneof_primitive", withFieldNumber(FieldType.row(PRIMITIVE_SCHEMA), 5)));
+ private static final Map<String, Integer> ONE_OF_ENUM_MAP =
+ ONEOF_FIELDS.stream()
+ .collect(Collectors.toMap(Field::getName, f -> getFieldNumber(f.getType())));
+ static final OneOfType ONE_OF_TYPE = OneOfType.create(ONEOF_FIELDS, ONE_OF_ENUM_MAP);
+ static final Schema ONEOF_SCHEMA =
+ Schema.builder()
+ .addField("special_oneof", FieldType.logicalType(ONE_OF_TYPE))
+ .addField("place1", withFieldNumber(FieldType.STRING.withNullable(true), 1))
+ .addField("place2", withFieldNumber(FieldType.INT32.withNullable(true), 6))
+ .build();
+
+ // Sample row instances for each OneOf case.
+ static final Row ONEOF_ROW_INT32 =
+ Row.withSchema(ONEOF_SCHEMA)
+ .addValues(ONE_OF_TYPE.createValue("oneof_int32", 1), "foo", 0)
+ .build();
+ static final Row ONEOF_ROW_BOOL =
+ Row.withSchema(ONEOF_SCHEMA)
+ .addValues(ONE_OF_TYPE.createValue("oneof_bool", true), "foo", 0)
+ .build();
+ static final Row ONEOF_ROW_STRING =
+ Row.withSchema(ONEOF_SCHEMA)
+ .addValues(ONE_OF_TYPE.createValue("oneof_string", "foo"), "foo", 0)
+ .build();
+ static final Row ONEOF_ROW_PRIMITIVE =
+ Row.withSchema(ONEOF_SCHEMA)
+ .addValues(ONE_OF_TYPE.createValue("oneof_primitive", PRIMITIVE_ROW), "foo", 0)
+ .build();
+
+ // Sample proto instances for each oneof case.
+ static final OneOf ONEOF_PROTO_INT32 =
+ OneOf.newBuilder().setOneofInt32(1).setPlace1("foo").setPlace2(0).build();
+ static final OneOf ONEOF_PROTO_BOOL =
+ OneOf.newBuilder().setOneofBool(true).setPlace1("foo").setPlace2(0).build();
+ static final OneOf ONEOF_PROTO_STRING =
+ OneOf.newBuilder().setOneofString("foo").setPlace1("foo").setPlace2(0).build();
+ static final OneOf ONEOF_PROTO_PRIMITIVE =
+ OneOf.newBuilder().setOneofPrimitive(PRIMITIVE_PROTO).setPlace1("foo").setPlace2(0).build();
+
+ // The schema for the OuterOneOf proto.
+ private static final List<Field> OUTER_ONEOF_FIELDS =
+ ImmutableList.of(
+ Field.of("oneof_oneof", withFieldNumber(FieldType.row(ONEOF_SCHEMA), 1)),
+ Field.of("oneof_int32", withFieldNumber(FieldType.INT32, 2)));
+ private static final Map<String, Integer> OUTER_ONE_OF_ENUM_MAP =
+ OUTER_ONEOF_FIELDS.stream()
+ .collect(Collectors.toMap(Field::getName, f -> getFieldNumber(f.getType())));
+ static final OneOfType OUTER_ONEOF_TYPE =
+ OneOfType.create(OUTER_ONEOF_FIELDS, OUTER_ONE_OF_ENUM_MAP);
+ static final Schema OUTER_ONEOF_SCHEMA =
+ Schema.builder().addField("outer_oneof", FieldType.logicalType(OUTER_ONEOF_TYPE)).build();
+
+ // A sample instance of the Row.
+ static final Row OUTER_ONEOF_ROW =
+ Row.withSchema(OUTER_ONEOF_SCHEMA)
+ .addValues(OUTER_ONEOF_TYPE.createValue("oneof_oneof", ONEOF_ROW_PRIMITIVE))
+ .build();
+
+ // A sample instance of the proto.
+ static final OuterOneOf OUTER_ONEOF_PROTO =
+ OuterOneOf.newBuilder().setOneofOneof(ONEOF_PROTO_PRIMITIVE).build();
+
+ static final Schema WKT_MESSAGE_SCHEMA =
+ Schema.builder()
+ .addNullableField("double", withFieldNumber(FieldType.DOUBLE, 1))
+ .addNullableField("float", withFieldNumber(FieldType.FLOAT, 2))
+ .addNullableField("int32", withFieldNumber(FieldType.INT32, 3))
+ .addNullableField("int64", withFieldNumber(FieldType.INT64, 4))
+ .addNullableField("uint32", withFieldNumber(FieldType.logicalType(new UInt32()), 5))
+ .addNullableField("uint64", withFieldNumber(FieldType.logicalType(new UInt64()), 6))
+ .addNullableField("bool", withFieldNumber(FieldType.BOOLEAN, 13))
+ .addNullableField("string", withFieldNumber(FieldType.STRING, 14))
+ .addNullableField("bytes", withFieldNumber(FieldType.BYTES, 15))
+ .addNullableField(
+ "timestamp", withFieldNumber(FieldType.logicalType(new TimestampNanos()), 16))
+ .addNullableField(
+ "duration", withFieldNumber(FieldType.logicalType(new DurationNanos()), 17))
+ .build();
+ // A sample instance of the row.
+ static final Instant JAVA_NOW = Instant.now();
+ static final Timestamp PROTO_NOW =
+ Timestamp.newBuilder()
+ .setSeconds(JAVA_NOW.getEpochSecond())
+ .setNanos(JAVA_NOW.getNano())
+ .build();
+ static final Duration PROTO_DURATION =
+ Duration.newBuilder()
+ .setSeconds(JAVA_NOW.getEpochSecond())
+ .setNanos(JAVA_NOW.getNano())
+ .build();
+ static final Row WKT_MESSAGE_ROW =
+ Row.withSchema(WKT_MESSAGE_SCHEMA)
+ .addValues(
+ 1.1, 2.2F, 32, 64L, 33, 65L, true, "horsey", BYTE_ARRAY, PROTO_NOW, PROTO_DURATION)
+ .build();
+
+ // A sample instance of the proto.
+ static final WktMessage WKT_MESSAGE_PROTO =
+ WktMessage.newBuilder()
+ .setDouble(DoubleValue.of(1.1))
+ .setFloat(FloatValue.of(2.2F))
+ .setInt32(Int32Value.of(32))
+ .setInt64(Int64Value.of(64))
+ .setUint32(UInt32Value.of(33))
+ .setUint64(UInt64Value.of(65))
+ .setBool(BoolValue.of(true))
+ .setString(StringValue.of("horsey"))
+ .setBytes(BytesValue.of(ByteString.copyFrom(BYTE_ARRAY)))
+ .setTimestamp(PROTO_NOW)
+ .setDuration(PROTO_DURATION)
+ .build();
+}
diff --git a/sdks/java/extensions/protobuf/src/test/proto/proto3_schema_messages.proto b/sdks/java/extensions/protobuf/src/test/proto/proto3_schema_messages.proto
new file mode 100644
index 0000000..6f6ec44
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/proto/proto3_schema_messages.proto
@@ -0,0 +1,123 @@
+/*
+ * 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.
+ */
+
+/*
+ * Protocol Buffer messages used for testing Proto3 Schema implementation.
+ */
+
+syntax = "proto3";
+
+package proto3_schema_messages;
+
+import "google/protobuf/duration.proto";
+import "google/protobuf/timestamp.proto";
+import "google/protobuf/wrappers.proto";
+import "google/protobuf/descriptor.proto";
+
+option java_package = "org.apache.beam.sdk.extensions.protobuf";
+
+message Primitive {
+ double primitive_double = 1;
+ float primitive_float = 2;
+ int32 primitive_int32 = 3;
+ int64 primitive_int64 = 4;
+ uint32 primitive_uint32 = 5;
+ uint64 primitive_uint64 = 6;
+ sint32 primitive_sint32 = 7;
+ sint64 primitive_sint64 = 8;
+ fixed32 primitive_fixed32 = 9;
+ fixed64 primitive_fixed64 = 10;
+ sfixed32 primitive_sfixed32 = 11;
+ sfixed64 primitive_sfixed64 = 12;
+ bool primitive_bool = 13;
+ string primitive_string = 14;
+ bytes primitive_bytes = 15;
+}
+
+message RepeatPrimitive {
+ repeated double repeated_double = 1;
+ repeated float repeated_float = 2;
+ repeated int32 repeated_int32 = 3;
+ repeated int64 repeated_int64 = 4;
+ repeated uint32 repeated_uint32 = 5;
+ repeated uint64 repeated_uint64 = 6;
+ repeated sint32 repeated_sint32 = 7;
+ repeated sint64 repeated_sint64 = 8;
+ repeated fixed32 repeated_fixed32 = 9;
+ repeated fixed64 repeated_fixed64 = 10;
+ repeated sfixed32 repeated_sfixed32 = 11;
+ repeated sfixed64 repeated_sfixed64 = 12;
+ repeated bool repeated_bool = 13;
+ repeated string repeated_string = 14;
+ repeated bytes repeated_bytes = 15;
+}
+
+message MapPrimitive {
+ map<string, string> string_string_map = 1;
+ map<string, int32> string_int_map = 2;
+ map<int32, string> int_string_map = 3;
+ map<string, bytes> string_bytes_map = 4;
+}
+
+message Nested {
+ Primitive nested = 1;
+ repeated Primitive nested_list = 2;
+ map<string, Primitive> nested_map = 3;
+}
+
+message OneOf {
+ string place1 = 1;
+ oneof special_oneof {
+ int32 oneof_int32 = 2;
+ bool oneof_bool = 3;
+ string oneof_string = 4;
+ Primitive oneof_primitive = 5;
+ }
+ int32 place2 = 6;
+}
+
+message OuterOneOf {
+ oneof outer_oneof {
+ OneOf oneof_oneof = 1;
+ int32 oneof_int32 = 2;
+ }
+}
+
+message EnumMessage {
+ enum Enum {
+ ZERO = 0;
+ TWO = 2;
+ THREE = 3;
+ }
+ Enum enum = 1;
+}
+
+message WktMessage {
+ google.protobuf.DoubleValue double = 1;
+ google.protobuf.FloatValue float = 2;
+ google.protobuf.Int32Value int32 = 3;
+ google.protobuf.Int64Value int64 = 4;
+ google.protobuf.UInt32Value uint32 = 5;
+ google.protobuf.UInt64Value uint64 = 6;
+ google.protobuf.BoolValue bool = 13;
+ google.protobuf.StringValue string = 14;
+ google.protobuf.BytesValue bytes = 15;
+ google.protobuf.Timestamp timestamp = 16;
+ google.protobuf.Duration duration = 17;
+}
+
diff --git a/sdks/java/extensions/protobuf/src/test/resources/README.md b/sdks/java/extensions/protobuf/src/test/resources/README.md
new file mode 100644
index 0000000..79083f5
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/resources/README.md
@@ -0,0 +1,34 @@
+<!--
+ 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.
+-->
+
+This recreates the proto descriptor set included in this resource directory.
+
+```bash
+export PROTO_INCLUDE=<proto_include_dir>
+```
+Execute the following command to create the pb files, in the beam root folder:
+
+```bash
+protoc \
+ -Isdks/java/extensions/protobuf/src/test/resources/ \
+ -I$PROTO_INCLUDE \
+ --descriptor_set_out=sdks/java/extensions/protobuf/src/test/resources/org/apache/beam/sdk/extensions/protobuf/test_option_v1.pb \
+ --include_imports \
+ sdks/java/extensions/protobuf/src/test/resources/test/option/v1/simple.proto
+```
diff --git a/sdks/java/extensions/protobuf/src/test/resources/org/apache/beam/sdk/extensions/protobuf/test_option_v1.pb b/sdks/java/extensions/protobuf/src/test/resources/org/apache/beam/sdk/extensions/protobuf/test_option_v1.pb
new file mode 100644
index 0000000..4e97ad0
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/resources/org/apache/beam/sdk/extensions/protobuf/test_option_v1.pb
Binary files differ
diff --git a/sdks/java/extensions/protobuf/src/test/resources/test/option/v1/option.proto b/sdks/java/extensions/protobuf/src/test/resources/test/option/v1/option.proto
new file mode 100644
index 0000000..ca40119
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/resources/test/option/v1/option.proto
@@ -0,0 +1,137 @@
+/*
+ * 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.
+ */
+
+syntax = "proto3";
+
+package test.option.v1;
+
+import "google/protobuf/descriptor.proto";
+
+extend google.protobuf.FileOptions {
+ double fileoption_double = 66666700;
+ float fileoption_float = 66666701;
+ int32 fileoption_int32 = 66666702;
+ int64 fileoption_int64 = 66666703;
+ uint32 fileoption_uint32 = 66666704;
+ uint64 fileoption_uint64 = 66666705;
+ sint32 fileoption_sint32 = 66666706;
+ sint64 fileoption_sint64 = 66666707;
+ fixed32 fileoption_fixed32 = 66666708;
+ fixed64 fileoption_fixed64 = 66666709;
+ sfixed32 fileoption_sfixed32 = 66666710;
+ sfixed64 fileoption_sfixed64 = 66666711;
+ bool fileoption_bool = 66666712;
+ string fileoption_string = 66666713;
+ bytes fileoption_bytes = 66666714;
+ OptionMessage fileoption_message = 66666715;
+ OptionEnum fileoption_enum = 66666716;
+}
+
+extend google.protobuf.MessageOptions {
+ double messageoption_double = 66666700;
+ float messageoption_float = 66666701;
+ int32 messageoption_int32 = 66666702;
+ int64 messageoption_int64 = 66666703;
+ uint32 messageoption_uint32 = 66666704;
+ uint64 messageoption_uint64 = 66666705;
+ sint32 messageoption_sint32 = 66666706;
+ sint64 messageoption_sint64 = 66666707;
+ fixed32 messageoption_fixed32 = 66666708;
+ fixed64 messageoption_fixed64 = 66666709;
+ sfixed32 messageoption_sfixed32 = 66666710;
+ sfixed64 messageoption_sfixed64 = 66666711;
+ bool messageoption_bool = 66666712;
+ string messageoption_string = 66666713;
+ bytes messageoption_bytes = 66666714;
+ OptionMessage messageoption_message = 66666715;
+ OptionEnum messageoption_enum = 66666716;
+
+ repeated double messageoption_repeated_double = 66666800;
+ repeated float messageoption_repeated_float = 66666801;
+ repeated int32 messageoption_repeated_int32 = 66666802;
+ repeated int64 messageoption_repeated_int64 = 66666803;
+ repeated uint32 messageoption_repeated_uint32 = 66666804;
+ repeated uint64 messageoption_repeated_uint64 = 66666805;
+ repeated sint32 messageoption_repeated_sint32 = 66666806;
+ repeated sint64 messageoption_repeated_sint64 = 66666807;
+ repeated fixed32 messageoption_repeated_fixed32 = 66666808;
+ repeated fixed64 messageoption_repeated_fixed64 = 66666809;
+ repeated sfixed32 messageoption_repeated_sfixed32 = 66666810;
+ repeated sfixed64 messageoption_repeated_sfixed64 = 66666811;
+ repeated bool messageoption_repeated_bool = 66666812;
+ repeated string messageoption_repeated_string = 66666813;
+ repeated bytes messageoption_repeated_bytes = 66666814;
+ repeated OptionMessage messageoption_repeated_message = 66666815;
+ repeated OptionEnum messageoption_repeated_enum = 66666816;
+}
+
+extend google.protobuf.FieldOptions {
+ double fieldoption_double = 66666700;
+ float fieldoption_float = 66666701;
+ int32 fieldoption_int32 = 66666702;
+ int64 fieldoption_int64 = 66666703;
+ uint32 fieldoption_uint32 = 66666704;
+ uint64 fieldoption_uint64 = 66666705;
+ sint32 fieldoption_sint32 = 66666706;
+ sint64 fieldoption_sint64 = 66666707;
+ fixed32 fieldoption_fixed32 = 66666708;
+ fixed64 fieldoption_fixed64 = 66666709;
+ sfixed32 fieldoption_sfixed32 = 66666710;
+ sfixed64 fieldoption_sfixed64 = 66666711;
+ bool fieldoption_bool = 66666712;
+ string fieldoption_string = 66666713;
+ bytes fieldoption_bytes = 66666714;
+ OptionMessage fieldoption_message = 66666715;
+ OptionEnum fieldoption_enum = 66666716;
+
+ repeated double fieldoption_repeated_double = 66666800;
+ repeated float fieldoption_repeated_float = 66666801;
+ repeated int32 fieldoption_repeated_int32 = 66666802;
+ repeated int64 fieldoption_repeated_int64 = 66666803;
+ repeated uint32 fieldoption_repeated_uint32 = 66666804;
+ repeated uint64 fieldoption_repeated_uint64 = 66666805;
+ repeated sint32 fieldoption_repeated_sint32 = 66666806;
+ repeated sint64 fieldoption_repeated_sint64 = 66666807;
+ repeated fixed32 fieldoption_repeated_fixed32 = 66666808;
+ repeated fixed64 fieldoption_repeated_fixed64 = 66666809;
+ repeated sfixed32 fieldoption_repeated_sfixed32 = 66666810;
+ repeated sfixed64 fieldoption_repeated_sfixed64 = 66666811;
+ repeated bool fieldoption_repeated_bool = 66666812;
+ repeated string fieldoption_repeated_string = 66666813;
+ repeated bytes fieldoption_repeated_bytes = 66666814;
+ repeated OptionMessage fieldoption_repeated_message = 66666815;
+ repeated OptionEnum fieldoption_repeated_enum = 66666816;
+}
+
+enum OptionEnum {
+ DEFAULT = 0;
+ ENUM1 = 1;
+ ENUM2 = 2;
+}
+
+message OptionMessage {
+ string string = 1;
+ repeated string repeated_string = 2;
+
+ int32 int32 = 3;
+ repeated int32 repeated_int32 = 4;
+
+ int64 int64 = 5;
+
+ OptionEnum test_enum = 6;
+}
\ No newline at end of file
diff --git a/sdks/java/extensions/protobuf/src/test/resources/test/option/v1/simple.proto b/sdks/java/extensions/protobuf/src/test/resources/test/option/v1/simple.proto
new file mode 100644
index 0000000..1750ddf
--- /dev/null
+++ b/sdks/java/extensions/protobuf/src/test/resources/test/option/v1/simple.proto
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+
+syntax = "proto3";
+
+import "test/option/v1/option.proto";
+
+package test.option.v1;
+
+message MessageWithOptions {
+ string test_name = 1;
+ int32 test_index = 2;
+
+ int32 field_with_fieldoption_double = 700 [(test.option.v1.fieldoption_double) = 100.1];
+ int32 field_with_fieldoption_float = 701 [(test.option.v1.fieldoption_float) = 101.2];
+ int32 field_with_fieldoption_int32 = 702 [(test.option.v1.fieldoption_int32) = 102];
+ int32 field_with_fieldoption_int64 = 703 [(test.option.v1.fieldoption_int64) = 103];
+ int32 field_with_fieldoption_uint32 = 704 [(test.option.v1.fieldoption_uint32) = 104];
+ int32 field_with_fieldoption_uint64 = 705 [(test.option.v1.fieldoption_uint64) = 105];
+ int32 field_with_fieldoption_sint32 = 706 [(test.option.v1.fieldoption_sint32) = 106];
+ int32 field_with_fieldoption_sint64 = 707 [(test.option.v1.fieldoption_sint64) = 107];
+ int32 field_with_fieldoption_fixed32 = 708;
+ int32 field_with_fieldoption_fixed64 = 709;
+ int32 field_with_fieldoption_sfixed32 = 710;
+ int32 field_with_fieldoption_sfixed64 = 711;
+ int32 field_with_fieldoption_bool = 712 [(test.option.v1.fieldoption_bool) = true];
+ int32 field_with_fieldoption_string = 713 [(test.option.v1.fieldoption_string) = "Oh yeah"];
+ int32 field_with_fieldoption_bytes = 714;
+ int32 field_with_fieldoption_message = 715;
+ int32 field_with_fieldoption_enum = 716 [(test.option.v1.fieldoption_enum) = ENUM1];
+
+ int32 field_with_fieldoption_repeated_double = 800;
+ int32 field_with_fieldoption_repeated_float = 801;
+ int32 field_with_fieldoption_repeated_int32 = 802;
+ int32 field_with_fieldoption_repeated_int64 = 803;
+ int32 field_with_fieldoption_repeated_uint32 = 804;
+ int32 field_with_fieldoption_repeated_uint64 = 805;
+ int32 field_with_fieldoption_repeated_sint32 = 806;
+ int32 field_with_fieldoption_repeated_sint64 = 807;
+ int32 field_with_fieldoption_repeated_fixed32 = 808;
+ int32 field_with_fieldoption_repeated_fixed64 = 809;
+ int32 field_with_fieldoption_repeated_sfixed32 = 810;
+ int32 field_with_fieldoption_repeated_sfixed64 = 811;
+ int32 field_with_fieldoption_repeated_bool = 812;
+ int32 field_with_fieldoption_repeated_string = 813 [(test.option.v1.fieldoption_repeated_string) = "Oh yeah",
+ (test.option.v1.fieldoption_repeated_string) = "Oh no"];
+ int32 field_with_fieldoption_repeated_bytes = 814;
+ int32 field_with_fieldoption_repeated_message = 815;
+ int32 field_with_fieldoption_repeated_enum = 816;
+
+}
+
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
index f30f9f3..14774b9 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
@@ -20,6 +20,7 @@
import java.util.List;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamAggregateProjectMergeRule;
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamAggregationRule;
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamBasicAggregationRule;
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcRule;
@@ -41,7 +42,6 @@
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateProjectMergeRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateRemoveRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateUnionAggregateRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
@@ -96,7 +96,7 @@
ProjectSetOpTransposeRule.INSTANCE,
// aggregation and projection rules
- AggregateProjectMergeRule.INSTANCE,
+ BeamAggregateProjectMergeRule.INSTANCE,
// push a projection past a filter or vice versa
ProjectFilterTransposeRule.INSTANCE,
FilterProjectTransposeRule.INSTANCE,
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
new file mode 100644
index 0000000..61abbb6
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
@@ -0,0 +1,82 @@
+/*
+ * 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.extensions.sql.impl.rel;
+
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
+
+/** BeamRelNode to replace {@code Project} and {@code Filter} node. */
+@Internal
+public abstract class AbstractBeamCalcRel extends Calc implements BeamRelNode {
+
+ public AbstractBeamCalcRel(
+ RelOptCluster cluster, RelTraitSet traits, RelNode input, RexProgram program) {
+ super(cluster, traits, input, program);
+ }
+
+ public boolean isInputSortRelAndLimitOnly() {
+ return (input instanceof BeamSortRel) && ((BeamSortRel) input).isLimitOnly();
+ }
+
+ public int getLimitCountOfSortRel() {
+ if (input instanceof BeamSortRel) {
+ return ((BeamSortRel) input).getCount();
+ }
+
+ throw new RuntimeException("Could not get the limit count from a non BeamSortRel input.");
+ }
+
+ @Override
+ public NodeStats estimateNodeStats(RelMetadataQuery mq) {
+ NodeStats inputStat = BeamSqlRelUtils.getNodeStats(input, mq);
+ double selectivity = estimateFilterSelectivity(getInput(), program, mq);
+
+ return inputStat.multiply(selectivity);
+ }
+
+ private static double estimateFilterSelectivity(
+ RelNode child, RexProgram program, RelMetadataQuery mq) {
+ // Similar to calcite, if the calc node is representing filter operation we estimate the filter
+ // selectivity based on the number of equality conditions, number of inequality conditions, ....
+ RexLocalRef programCondition = program.getCondition();
+ RexNode condition;
+ if (programCondition == null) {
+ condition = null;
+ } else {
+ condition = program.expandLocalRef(programCondition);
+ }
+ // Currently this gets the selectivity based on Calcite's Selectivity Handler (RelMdSelectivity)
+ return mq.getSelectivity(child, condition);
+ }
+
+ @Override
+ public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+ NodeStats inputStat = BeamSqlRelUtils.getNodeStats(this.input, mq);
+ return BeamCostModel.FACTORY.makeCost(inputStat.getRowCount(), inputStat.getRate());
+ }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
index e65d582..6c5e3cf 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
@@ -33,9 +33,7 @@
import java.util.Map;
import java.util.Set;
import javax.annotation.Nullable;
-import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamJavaTypeFactory;
-import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.CharType;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils.DateType;
@@ -65,15 +63,12 @@
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.ParameterExpression;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Types;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPredicateList;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexSimplify;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexUtil;
@@ -90,8 +85,8 @@
import org.joda.time.DateTimeZone;
import org.joda.time.ReadableInstant;
-/** BeamRelNode to replace a {@code Project} node. */
-public class BeamCalcRel extends Calc implements BeamRelNode {
+/** BeamRelNode to replace {@code Project} and {@code Filter} node. */
+public class BeamCalcRel extends AbstractBeamCalcRel {
private static final ParameterExpression outputSchemaParam =
Expressions.parameter(Schema.class, "outputSchema");
@@ -147,7 +142,7 @@
final RelMetadataQuery mq = RelMetadataQuery.instance();
final RelOptPredicateList predicates = mq.getPulledUpPredicates(getInput());
final RexSimplify simplify = new RexSimplify(rexBuilder, predicates, RexUtil.EXECUTOR);
- final RexProgram program = BeamCalcRel.this.program.normalize(rexBuilder, simplify);
+ final RexProgram program = getProgram().normalize(rexBuilder, simplify);
Expression condition =
RexToLixTranslator.translateCondition(
@@ -210,47 +205,6 @@
}
}
- public int getLimitCountOfSortRel() {
- if (input instanceof BeamSortRel) {
- return ((BeamSortRel) input).getCount();
- }
-
- throw new RuntimeException("Could not get the limit count from a non BeamSortRel input.");
- }
-
- @Override
- public NodeStats estimateNodeStats(RelMetadataQuery mq) {
- NodeStats inputStat = BeamSqlRelUtils.getNodeStats(this.input, mq);
- double selectivity = estimateFilterSelectivity(getInput(), program, mq);
-
- return inputStat.multiply(selectivity);
- }
-
- private static double estimateFilterSelectivity(
- RelNode child, RexProgram program, RelMetadataQuery mq) {
- // Similar to calcite, if the calc node is representing filter operation we estimate the filter
- // selectivity based on the number of equality conditions, number of inequality conditions, ....
- RexLocalRef programCondition = program.getCondition();
- RexNode condition;
- if (programCondition == null) {
- condition = null;
- } else {
- condition = program.expandLocalRef(programCondition);
- }
- // Currently this gets the selectivity based on Calcite's Selectivity Handler (RelMdSelectivity)
- return mq.getSelectivity(child, condition);
- }
-
- @Override
- public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
- NodeStats inputStat = BeamSqlRelUtils.getNodeStats(this.input, mq);
- return BeamCostModel.FACTORY.makeCost(inputStat.getRowCount(), inputStat.getRate());
- }
-
- public boolean isInputSortRelAndLimitOnly() {
- return (input instanceof BeamSortRel) && ((BeamSortRel) input).isLimitOnly();
- }
-
/** {@code CalcFn} is the executor for a {@link BeamCalcRel} step. */
private static class CalcFn extends DoFn<Row, Row> {
private final String processElementBlock;
@@ -306,7 +260,7 @@
.put(TypeName.DOUBLE, Double.class)
.build();
- private Expression castOutput(Expression value, FieldType toType) {
+ private static Expression castOutput(Expression value, FieldType toType) {
if (value.getType() == Object.class || !(value.getType() instanceof Class)) {
// fast copy path, just pass object through
return value;
@@ -334,7 +288,7 @@
return value;
}
- private Expression castOutputTime(Expression value, FieldType toType) {
+ private static Expression castOutputTime(Expression value, FieldType toType) {
Expression valueDateTime = value;
// First, convert to millis
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
index 7e78945..4ec00cf 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
@@ -383,14 +383,15 @@
private static boolean isLimitQuery(BeamRelNode node) {
return (node instanceof BeamSortRel && ((BeamSortRel) node).isLimitOnly())
- || (node instanceof BeamCalcRel && ((BeamCalcRel) node).isInputSortRelAndLimitOnly());
+ || (node instanceof AbstractBeamCalcRel
+ && ((AbstractBeamCalcRel) node).isInputSortRelAndLimitOnly());
}
private static int getLimitCount(BeamRelNode node) {
if (node instanceof BeamSortRel) {
return ((BeamSortRel) node).getCount();
- } else if (node instanceof BeamCalcRel) {
- return ((BeamCalcRel) node).getLimitCountOfSortRel();
+ } else if (node instanceof AbstractBeamCalcRel) {
+ return ((AbstractBeamCalcRel) node).getLimitCountOfSortRel();
}
throw new RuntimeException(
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
new file mode 100644
index 0000000..b146b04
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
@@ -0,0 +1,97 @@
+/*
+ * 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.extensions.sql.impl.rule;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.SingleRel;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Filter;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Project;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateProjectMergeRule;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
+
+/**
+ * This rule is essentially a wrapper around Calcite's {@code AggregateProjectMergeRule}. In the
+ * case when an underlying IO supports project push-down it is more efficient to not merge {@code
+ * Project} with an {@code Aggregate}, leaving it for the {@code BeamIOPUshDownRule}.
+ */
+public class BeamAggregateProjectMergeRule extends AggregateProjectMergeRule {
+ public static final AggregateProjectMergeRule INSTANCE =
+ new BeamAggregateProjectMergeRule(
+ Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
+
+ public BeamAggregateProjectMergeRule(
+ Class<? extends Aggregate> aggregateClass,
+ Class<? extends Project> projectClass,
+ RelBuilderFactory relBuilderFactory) {
+ super(aggregateClass, projectClass, relBuilderFactory);
+ }
+
+ @Override
+ public void onMatch(RelOptRuleCall call) {
+ final Project project = call.rel(1);
+ BeamIOSourceRel io = getUnderlyingIO(new HashSet<>(), project);
+
+ // Only perform AggregateProjectMergeRule when IO is not present or project push-down is not
+ // supported.
+ if (io == null || !io.getBeamSqlTable().supportsProjects().isSupported()) {
+ super.onMatch(call);
+ }
+ }
+
+ /**
+ * Following scenarios are possible:<br>
+ * 1) Aggregate <- Project <- IO.<br>
+ * 2) Aggregate <- Project <- Chain of Project/Filter <- IO.<br>
+ * 3) Aggregate <- Project <- Something else.<br>
+ * 4) Aggregate <- Project <- Chain of Project/Filter <- Something else.
+ *
+ * @param parent project that matched this rule.
+ * @return {@code BeamIOSourceRel} when it is present or null when some other {@code RelNode} is
+ * present.
+ */
+ private BeamIOSourceRel getUnderlyingIO(Set<RelNode> visitedNodes, SingleRel parent) {
+ // No need to look at the same node more than once.
+ if (visitedNodes.contains(parent)) {
+ return null;
+ }
+ visitedNodes.add(parent);
+ List<RelNode> nodes = ((RelSubset) parent.getInput()).getRelList();
+
+ for (RelNode node : nodes) {
+ if (node instanceof Filter || node instanceof Project) {
+ // Search node inputs for an IO.
+ BeamIOSourceRel child = getUnderlyingIO(visitedNodes, (SingleRel) node);
+ if (child != null) {
+ return child;
+ }
+ } else if (node instanceof BeamIOSourceRel) {
+ return (BeamIOSourceRel) node;
+ }
+ }
+
+ return null;
+ }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
new file mode 100644
index 0000000..432b3b5
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
@@ -0,0 +1,84 @@
+/*
+ * 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.extensions.sql.meta.provider.bigquery;
+
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor.POS;
+
+import java.util.function.IntFunction;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlLiteral;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.BitString;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.commons.lang.StringEscapeUtils;
+
+public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext {
+
+ public BeamSqlUnparseContext(IntFunction<SqlNode> field) {
+ super(BeamBigQuerySqlDialect.DEFAULT, field);
+ }
+
+ @Override
+ public SqlNode toSql(RexProgram program, RexNode rex) {
+ if (rex.getKind().equals(SqlKind.LITERAL)) {
+ final RexLiteral literal = (RexLiteral) rex;
+ SqlTypeFamily family = literal.getTypeName().getFamily();
+ if (SqlTypeFamily.BINARY.equals(family)) {
+ BitString bitString = BitString.createFromBytes(literal.getValueAs(byte[].class));
+ return new SqlByteStringLiteral(bitString, POS);
+ } else if (SqlTypeFamily.CHARACTER.equals(family)) {
+ String escaped = StringEscapeUtils.escapeJava(literal.getValueAs(String.class));
+ return SqlLiteral.createCharString(escaped, POS);
+ }
+ }
+
+ return super.toSql(program, rex);
+ }
+
+ private static class SqlByteStringLiteral extends SqlLiteral {
+
+ SqlByteStringLiteral(BitString bytes, SqlParserPos pos) {
+ super(bytes, SqlTypeName.BINARY, pos);
+ }
+
+ @Override
+ public SqlByteStringLiteral clone(SqlParserPos pos) {
+ return new SqlByteStringLiteral((BitString) this.value, pos);
+ }
+
+ @Override
+ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ assert this.value instanceof BitString;
+
+ StringBuilder builder = new StringBuilder("B'");
+ for (byte b : ((BitString) this.value).getAsByteArray()) {
+ builder.append(String.format("\\x%02X", b));
+ }
+ builder.append("'");
+
+ writer.literal(builder.toString());
+ }
+ }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
index 789d8ec..71f4235 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
@@ -188,8 +188,7 @@
// TODO: BigQuerySqlDialectWithTypeTranslation can be replaced with BigQuerySqlDialect after
// updating vendor Calcite version.
- SqlImplementor.SimpleContext context =
- new SqlImplementor.SimpleContext(BeamBigQuerySqlDialect.DEFAULT, field);
+ SqlImplementor.Context context = new BeamSqlUnparseContext(field);
// Create a single SqlNode from a list of RexNodes
SqlNode andSqlNode = null;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
new file mode 100644
index 0000000..2bce48c
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.extensions.sql.impl.rule;
+
+import static org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PUSH_DOWN_OPTION;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
+
+import com.alibaba.fastjson.JSON;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamAggregationRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamPushDownIOSourceRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider;
+import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PushDownOptions;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class BeamAggregateProjectMergeRuleTest {
+ private static final Schema BASIC_SCHEMA =
+ Schema.builder()
+ .addInt32Field("unused1")
+ .addInt32Field("id")
+ .addStringField("name")
+ .addInt32Field("unused2")
+ .build();
+ private BeamSqlEnv sqlEnv;
+
+ @Rule public TestPipeline pipeline = TestPipeline.create();
+
+ @Before
+ public void buildUp() {
+ TestTableProvider tableProvider = new TestTableProvider();
+ Table projectTable = getTable("TEST_PROJECT", PushDownOptions.PROJECT);
+ Table filterTable = getTable("TEST_FILTER", PushDownOptions.FILTER);
+ Table noneTable = getTable("TEST_NONE", PushDownOptions.NONE);
+ tableProvider.createTable(projectTable);
+ tableProvider.createTable(filterTable);
+ tableProvider.createTable(noneTable);
+ sqlEnv = BeamSqlEnv.inMemory(tableProvider);
+ }
+
+ @Test
+ public void testBeamAggregateProjectMergeRule_withProjectTable() {
+ // When an IO supports project push-down, Projects should be merged with an IO.
+ String sqlQuery = "select SUM(id) as id_sum from TEST_PROJECT group by name";
+ BeamRelNode beamRel = sqlEnv.parseQuery(sqlQuery);
+
+ BeamAggregationRel aggregate = (BeamAggregationRel) beamRel.getInput(0);
+ BeamIOSourceRel ioSourceRel = (BeamIOSourceRel) aggregate.getInput();
+
+ // Make sure project push-down took place.
+ assertThat(ioSourceRel, instanceOf(BeamPushDownIOSourceRel.class));
+ assertThat(ioSourceRel.getRowType().getFieldNames(), containsInAnyOrder("name", "id"));
+ }
+
+ @Test
+ public void testBeamAggregateProjectMergeRule_withProjectTable_withPredicate() {
+ // When an IO supports project push-down, Projects should be merged with an IO.
+ String sqlQuery = "select SUM(id) as id_sum from TEST_PROJECT where unused1=1 group by name";
+ BeamRelNode beamRel = sqlEnv.parseQuery(sqlQuery);
+
+ BeamAggregationRel aggregate = (BeamAggregationRel) beamRel.getInput(0);
+ BeamCalcRel calc = (BeamCalcRel) aggregate.getInput();
+ BeamIOSourceRel ioSourceRel = (BeamIOSourceRel) calc.getInput();
+
+ // Make sure project push-down took place.
+ assertThat(ioSourceRel, instanceOf(BeamPushDownIOSourceRel.class));
+ assertThat(
+ ioSourceRel.getRowType().getFieldNames(), containsInAnyOrder("name", "id", "unused1"));
+ }
+
+ @Test
+ public void testBeamAggregateProjectMergeRule_withFilterTable() {
+ // When an IO does not supports project push-down, Projects should be merged with an aggregate.
+ String sqlQuery = "select SUM(id) as id_sum from TEST_FILTER group by name";
+ BeamRelNode beamRel = sqlEnv.parseQuery(sqlQuery);
+
+ BeamAggregationRel aggregate = (BeamAggregationRel) beamRel.getInput(0);
+ BeamIOSourceRel ioSourceRel = (BeamIOSourceRel) aggregate.getInput();
+
+ // Make sure project merged with an aggregate.
+ assertThat(aggregate.getRowType().getFieldNames(), containsInAnyOrder("id_sum", "name"));
+
+ // IO projects al fields.
+ assertThat(ioSourceRel, instanceOf(BeamIOSourceRel.class));
+ assertThat(
+ ioSourceRel.getRowType().getFieldNames(),
+ containsInAnyOrder("unused1", "name", "id", "unused2"));
+ }
+
+ @Test
+ public void testBeamAggregateProjectMergeRule_withNoneTable() {
+ // When an IO does not supports project push-down, Projects should be merged with an aggregate.
+ String sqlQuery = "select SUM(id) as id_sum from TEST_NONE group by name";
+ BeamRelNode beamRel = sqlEnv.parseQuery(sqlQuery);
+
+ BeamAggregationRel aggregate = (BeamAggregationRel) beamRel.getInput(0);
+ BeamIOSourceRel ioSourceRel = (BeamIOSourceRel) aggregate.getInput();
+
+ // Make sure project merged with an aggregate.
+ assertThat(aggregate.getRowType().getFieldNames(), containsInAnyOrder("id_sum", "name"));
+
+ // IO projects al fields.
+ assertThat(ioSourceRel, instanceOf(BeamIOSourceRel.class));
+ assertThat(
+ ioSourceRel.getRowType().getFieldNames(),
+ containsInAnyOrder("unused1", "name", "id", "unused2"));
+ }
+
+ private static Table getTable(String name, PushDownOptions options) {
+ return Table.builder()
+ .name(name)
+ .comment(name + " table")
+ .schema(BASIC_SCHEMA)
+ .properties(
+ JSON.parseObject("{ " + PUSH_DOWN_OPTION + ": " + "\"" + options.toString() + "\" }"))
+ .type("test")
+ .build();
+ }
+}
diff --git a/sdks/java/extensions/sql/zetasql/build.gradle b/sdks/java/extensions/sql/zetasql/build.gradle
index 560b454..330209b 100644
--- a/sdks/java/extensions/sql/zetasql/build.gradle
+++ b/sdks/java/extensions/sql/zetasql/build.gradle
@@ -25,7 +25,7 @@
description = "Apache Beam :: SDKs :: Java :: Extensions :: SQL :: ZetaSQL"
ext.summary = "ZetaSQL to Calcite translator"
-def zetasql_version = "2019.10.1"
+def zetasql_version = "2019.12.1"
dependencies {
compile project(":sdks:java:core")
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
new file mode 100644
index 0000000..330fb2d
--- /dev/null
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
@@ -0,0 +1,212 @@
+/*
+ * 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.extensions.sql.zetasql;
+
+import com.google.zetasql.AnalyzerOptions;
+import com.google.zetasql.PreparedExpression;
+import com.google.zetasql.Value;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.IntFunction;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.sql.impl.rel.AbstractBeamCalcRel;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamBigQuerySqlDialect;
+import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamSqlUnparseContext;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDialect;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/**
+ * TODO[BEAM-8630]: This class is currently a prototype and not used in runtime.
+ *
+ * <p>BeamRelNode to replace {@code Project} and {@code Filter} node based on the {@code ZetaSQL}
+ * expression evaluator.
+ */
+@Internal
+public class BeamZetaSqlCalcRel extends AbstractBeamCalcRel {
+
+ private static final SqlDialect DIALECT = BeamBigQuerySqlDialect.DEFAULT;
+ private final SqlImplementor.Context context;
+
+ public BeamZetaSqlCalcRel(
+ RelOptCluster cluster, RelTraitSet traits, RelNode input, RexProgram program) {
+ super(cluster, traits, input, program);
+ final IntFunction<SqlNode> fn =
+ i ->
+ new SqlIdentifier(
+ getProgram().getInputRowType().getFieldList().get(i).getName(), SqlParserPos.ZERO);
+ context = new BeamSqlUnparseContext(fn);
+ }
+
+ @Override
+ public Calc copy(RelTraitSet traitSet, RelNode input, RexProgram program) {
+ return new BeamZetaSqlCalcRel(getCluster(), traitSet, input, program);
+ }
+
+ @Override
+ public PTransform<PCollectionList<Row>, PCollection<Row>> buildPTransform() {
+ return new Transform();
+ }
+
+ private class Transform extends PTransform<PCollectionList<Row>, PCollection<Row>> {
+ @Override
+ public PCollection<Row> expand(PCollectionList<Row> pinput) {
+ Preconditions.checkArgument(
+ pinput.size() == 1,
+ "%s expected a single input PCollection, but received %d.",
+ BeamZetaSqlCalcRel.class.getSimpleName(),
+ pinput.size());
+ PCollection<Row> upstream = pinput.get(0);
+
+ final List<String> projects =
+ getProgram().getProjectList().stream()
+ .map(BeamZetaSqlCalcRel.this::unparseRexNode)
+ .collect(Collectors.toList());
+ final RexNode condition = getProgram().getCondition();
+
+ // TODO[BEAM-8630]: validate sql expressions at pipeline construction time
+ Schema outputSchema = CalciteUtils.toSchema(getRowType());
+ CalcFn calcFn =
+ new CalcFn(
+ projects,
+ condition == null ? null : unparseRexNode(condition),
+ upstream.getSchema(),
+ outputSchema);
+ return upstream.apply(ParDo.of(calcFn)).setRowSchema(outputSchema);
+ }
+ }
+
+ private String unparseRexNode(RexNode rex) {
+ return context.toSql(getProgram(), rex).toSqlString(DIALECT).getSql();
+ }
+
+ /**
+ * {@code CalcFn} is the executor for a {@link BeamZetaSqlCalcRel} step. The implementation is
+ * based on the {@code ZetaSQL} expression evaluator.
+ */
+ private static class CalcFn extends DoFn<Row, Row> {
+ private final List<String> projects;
+ @Nullable private final String condition;
+ private final Schema inputSchema;
+ private final Schema outputSchema;
+ private transient List<PreparedExpression> projectExps;
+ @Nullable private transient PreparedExpression conditionExp;
+
+ CalcFn(
+ List<String> projects,
+ @Nullable String condition,
+ Schema inputSchema,
+ Schema outputSchema) {
+ Preconditions.checkArgument(projects.size() == outputSchema.getFieldCount());
+ this.projects = ImmutableList.copyOf(projects);
+ this.condition = condition;
+ this.inputSchema = inputSchema;
+ this.outputSchema = outputSchema;
+ }
+
+ @Setup
+ public void setup() {
+ AnalyzerOptions options = SqlAnalyzer.initAnalyzerOptions();
+ for (Field field : inputSchema.getFields()) {
+ options.addExpressionColumn(
+ sanitize(field.getName()), ZetaSqlUtils.beamFieldTypeToZetaSqlType(field.getType()));
+ }
+
+ // TODO[BEAM-8630]: use a single PreparedExpression for all condition and projects
+ projectExps = new ArrayList<>();
+ for (String project : projects) {
+ PreparedExpression projectExp = new PreparedExpression(sanitize(project));
+ projectExp.prepare(options);
+ projectExps.add(projectExp);
+ }
+ if (condition != null) {
+ conditionExp = new PreparedExpression(sanitize(condition));
+ conditionExp.prepare(options);
+ }
+ }
+
+ @ProcessElement
+ public void processElement(ProcessContext c) {
+ Map<String, Value> columns = new HashMap<>();
+ Row row = c.element();
+ for (Field field : inputSchema.getFields()) {
+ columns.put(
+ sanitize(field.getName()),
+ ZetaSqlUtils.javaObjectToZetaSqlValue(row.getValue(field.getName()), field.getType()));
+ }
+
+ // TODO[BEAM-8630]: support parameters in expression evaluation
+ // The map is empty because parameters in the query string have already been substituted.
+ Map<String, Value> params = Collections.emptyMap();
+
+ if (conditionExp != null && !conditionExp.execute(columns, params).getBoolValue()) {
+ return;
+ }
+
+ Row.Builder output = Row.withSchema(outputSchema);
+ for (int i = 0; i < outputSchema.getFieldCount(); i++) {
+ // TODO[BEAM-8630]: performance optimization by bundling the gRPC calls
+ Value v = projectExps.get(i).execute(columns, params);
+ output.addValue(
+ ZetaSqlUtils.zetaSqlValueToJavaObject(v, outputSchema.getField(i).getType()));
+ }
+ c.output(output.build());
+ }
+
+ @Teardown
+ public void teardown() {
+ for (PreparedExpression projectExp : projectExps) {
+ projectExp.close();
+ }
+ if (conditionExp != null) {
+ conditionExp.close();
+ }
+ }
+
+ // Replaces "$" with "_" because "$" is not allowed in a valid ZetaSQL identifier
+ // (ZetaSQL identifier syntax: [A-Za-z_][A-Za-z_0-9]*)
+ // TODO[BEAM-8630]: check if this is sufficient and correct, or even better fix this in Calcite
+ private static String sanitize(String identifier) {
+ return identifier.replaceAll("\\$", "_");
+ }
+ }
+}
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java
new file mode 100644
index 0000000..2e7ea0f
--- /dev/null
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java
@@ -0,0 +1,54 @@
+/*
+ * 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.extensions.sql.zetasql;
+
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
+
+/** A {@code ConverterRule} to replace {@link Calc} with {@link BeamZetaSqlCalcRel}. */
+public class BeamZetaSqlCalcRule extends ConverterRule {
+ public static final BeamZetaSqlCalcRule INSTANCE = new BeamZetaSqlCalcRule();
+
+ private BeamZetaSqlCalcRule() {
+ super(
+ LogicalCalc.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamZetaSqlCalcRule");
+ }
+
+ @Override
+ public boolean matches(RelOptRuleCall x) {
+ return true;
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ final Calc calc = (Calc) rel;
+ final RelNode input = calc.getInput();
+
+ return new BeamZetaSqlCalcRel(
+ calc.getCluster(),
+ calc.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
+ RelOptRule.convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)),
+ calc.getProgram());
+ }
+}
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
index 6ec56ae..0f5ec16 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
@@ -26,12 +26,15 @@
import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
import org.apache.beam.sdk.extensions.sql.impl.planner.RelMdNodeStats;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionConfig;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.CalciteCatalogReader;
@@ -50,6 +53,7 @@
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelConversionException;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
/** ZetaSQLQueryPlanner. */
@@ -61,9 +65,34 @@
}
public ZetaSQLQueryPlanner(JdbcConnection jdbcConnection, RuleSet[] ruleSets) {
+ // TODO[BEAM-8630]: uncomment the next lines once we have fully migrated to BeamZetaSqlCalcRel
+ // plannerImpl =
+ // new ZetaSQLPlannerImpl(defaultConfig(jdbcConnection, replaceBeamCalcRule(ruleSets)));
plannerImpl = new ZetaSQLPlannerImpl(defaultConfig(jdbcConnection, ruleSets));
}
+ public static RuleSet[] getZetaSqlRuleSets() {
+ // TODO[BEAM-8630]: uncomment the next line once we have fully migrated to BeamZetaSqlCalcRel
+ // return replaceBeamCalcRule(BeamRuleSets.getRuleSets());
+ return BeamRuleSets.getRuleSets();
+ }
+
+ private static RuleSet[] replaceBeamCalcRule(RuleSet[] ruleSets) {
+ RuleSet[] ret = new RuleSet[ruleSets.length];
+ for (int i = 0; i < ruleSets.length; i++) {
+ ImmutableList.Builder<RelOptRule> bd = ImmutableList.builder();
+ for (RelOptRule rule : ruleSets[i]) {
+ if (rule instanceof BeamCalcRule) {
+ bd.add(BeamZetaSqlCalcRule.INSTANCE);
+ } else {
+ bd.add(rule);
+ }
+ }
+ ret[i] = RuleSets.ofList(bd.build());
+ }
+ return ret;
+ }
+
@Override
public BeamRelNode convertToBeamRel(String sqlStatement)
throws ParseException, SqlConversionException {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
new file mode 100644
index 0000000..d771857
--- /dev/null
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtils.java
@@ -0,0 +1,204 @@
+/*
+ * 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.extensions.sql.zetasql;
+
+import com.google.protobuf.ByteString;
+import com.google.zetasql.ArrayType;
+import com.google.zetasql.StructType;
+import com.google.zetasql.StructType.StructField;
+import com.google.zetasql.Type;
+import com.google.zetasql.TypeFactory;
+import com.google.zetasql.Value;
+import com.google.zetasql.ZetaSQLType.TypeKind;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.math.LongMath;
+import org.joda.time.Instant;
+
+/** Utility methods for ZetaSQL related operations. */
+@Internal
+public final class ZetaSqlUtils {
+
+ private static final long MICROS_PER_MILLI = 1000L;
+
+ private ZetaSqlUtils() {}
+
+ // Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM, PROTO, GEOGRAPHY
+ // TODO[BEAM-8630]: support ZetaSQL types: DATE, TIME, DATETIME
+ public static Type beamFieldTypeToZetaSqlType(FieldType fieldType) {
+ switch (fieldType.getTypeName()) {
+ case INT64:
+ return TypeFactory.createSimpleType(TypeKind.TYPE_INT64);
+ case DECIMAL:
+ return TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC);
+ case DOUBLE:
+ return TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE);
+ case STRING:
+ return TypeFactory.createSimpleType(TypeKind.TYPE_STRING);
+ case DATETIME:
+ // TODO[BEAM-8630]: Mapping Timestamp to DATETIME results in some timezone/precision issues.
+ // Can we convert Timestamp to a LogicalType? Will it solve the problem?
+ return TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP);
+ case BOOLEAN:
+ return TypeFactory.createSimpleType(TypeKind.TYPE_BOOL);
+ case BYTES:
+ return TypeFactory.createSimpleType(TypeKind.TYPE_BYTES);
+ case ARRAY:
+ return createZetaSqlArrayTypeFromBeamElementFieldType(fieldType.getCollectionElementType());
+ case ROW:
+ return createZetaSqlStructTypeFromBeamSchema(fieldType.getRowSchema());
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported Beam fieldType: " + fieldType.getTypeName());
+ }
+ }
+
+ private static ArrayType createZetaSqlArrayTypeFromBeamElementFieldType(
+ FieldType elementFieldType) {
+ return TypeFactory.createArrayType(beamFieldTypeToZetaSqlType(elementFieldType));
+ }
+
+ private static StructType createZetaSqlStructTypeFromBeamSchema(Schema schema) {
+ return TypeFactory.createStructType(
+ schema.getFields().stream()
+ .map(ZetaSqlUtils::beamFieldToZetaSqlStructField)
+ .collect(Collectors.toList()));
+ }
+
+ private static StructField beamFieldToZetaSqlStructField(Field field) {
+ return new StructField(field.getName(), beamFieldTypeToZetaSqlType(field.getType()));
+ }
+
+ public static Value javaObjectToZetaSqlValue(Object object, FieldType fieldType) {
+ if (object == null) {
+ return Value.createNullValue(beamFieldTypeToZetaSqlType(fieldType));
+ }
+ switch (fieldType.getTypeName()) {
+ case INT64:
+ return Value.createInt64Value((Long) object);
+ // TODO[BEAM-8630]: Value.createNumericValue() is broken due to a dependency issue
+ // case DECIMAL:
+ // return Value.createNumericValue((BigDecimal) object);
+ case DOUBLE:
+ return Value.createDoubleValue((Double) object);
+ case STRING:
+ return Value.createStringValue((String) object);
+ case DATETIME:
+ return jodaInstantToZetaSqlTimestampValue((Instant) object);
+ case BOOLEAN:
+ return Value.createBoolValue((Boolean) object);
+ case BYTES:
+ return Value.createBytesValue(ByteString.copyFrom((byte[]) object));
+ case ARRAY:
+ return javaListToZetaSqlArrayValue(
+ (List<Object>) object, fieldType.getCollectionElementType());
+ case ROW:
+ return beamRowToZetaSqlStructValue((Row) object, fieldType.getRowSchema());
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported Beam fieldType: " + fieldType.getTypeName());
+ }
+ }
+
+ private static Value jodaInstantToZetaSqlTimestampValue(Instant instant) {
+ return javaLongToZetaSqlTimestampValue(instant.getMillis());
+ }
+
+ private static Value javaLongToZetaSqlTimestampValue(Long millis) {
+ return Value.createTimestampValueFromUnixMicros(
+ LongMath.checkedMultiply(millis, MICROS_PER_MILLI));
+ }
+
+ private static Value javaListToZetaSqlArrayValue(List<Object> elements, FieldType elementType) {
+ List<Value> values =
+ elements.stream()
+ .map(e -> javaObjectToZetaSqlValue(e, elementType))
+ .collect(Collectors.toList());
+ return Value.createArrayValue(
+ createZetaSqlArrayTypeFromBeamElementFieldType(elementType), values);
+ }
+
+ private static Value beamRowToZetaSqlStructValue(Row row, Schema schema) {
+ List<Value> values = new ArrayList<>(row.getFieldCount());
+
+ for (int i = 0; i < row.getFieldCount(); i++) {
+ values.add(javaObjectToZetaSqlValue(row.getValue(i), schema.getField(i).getType()));
+ }
+ return Value.createStructValue(createZetaSqlStructTypeFromBeamSchema(schema), values);
+ }
+
+ public static Object zetaSqlValueToJavaObject(Value value, FieldType fieldType) {
+ if (value.isNull()) {
+ return null;
+ }
+ switch (fieldType.getTypeName()) {
+ case INT64:
+ return value.getInt64Value();
+ case DECIMAL:
+ return value.getNumericValue();
+ case DOUBLE:
+ // Floats with a floating part equal to zero are treated as whole (INT64).
+ // Cast to double when that happens.
+ if (value.getType().getKind().equals(TypeKind.TYPE_INT64)) {
+ return (double) value.getInt64Value();
+ }
+ return value.getDoubleValue();
+ case STRING:
+ return value.getStringValue();
+ case DATETIME:
+ return zetaSqlTimestampValueToJodaInstant(value);
+ case BOOLEAN:
+ return value.getBoolValue();
+ case BYTES:
+ return value.getBytesValue().toByteArray();
+ case ARRAY:
+ return zetaSqlArrayValueToJavaList(value, fieldType.getCollectionElementType());
+ case ROW:
+ return zetaSqlStructValueToBeamRow(value, fieldType.getRowSchema());
+ default:
+ throw new IllegalArgumentException(
+ "Unsupported Beam fieldType: " + fieldType.getTypeName());
+ }
+ }
+
+ private static Instant zetaSqlTimestampValueToJodaInstant(Value timestampValue) {
+ long millis = timestampValue.getTimestampUnixMicros() / MICROS_PER_MILLI;
+ return Instant.ofEpochMilli(millis);
+ }
+
+ private static List<Object> zetaSqlArrayValueToJavaList(Value arrayValue, FieldType elementType) {
+ return arrayValue.getElementList().stream()
+ .map(e -> zetaSqlValueToJavaObject(e, elementType))
+ .collect(Collectors.toList());
+ }
+
+ private static Row zetaSqlStructValueToBeamRow(Value structValue, Schema schema) {
+ List<Object> objects = new ArrayList<>(schema.getFieldCount());
+ List<Value> values = structValue.getFieldList();
+ for (int i = 0; i < values.size(); i++) {
+ objects.add(zetaSqlValueToJavaObject(values.get(i), schema.getField(i).getType()));
+ }
+ return Row.withSchema(schema).addValues(objects).build();
+ }
+}
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java
index 4721b33..a16a443 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java
@@ -18,9 +18,15 @@
package org.apache.beam.sdk.extensions.sql.zetasql.translation;
import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedSingleRowScan;
+import java.math.BigDecimal;
import java.util.List;
+import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
/** Converts a single row value. */
class SingleRowScanConverter extends RelConverter<ResolvedSingleRowScan> {
@@ -36,6 +42,26 @@
@Override
public RelNode convert(ResolvedSingleRowScan zetaNode, List<RelNode> inputs) {
- return LogicalValues.createOneRow(getCluster());
+ return createOneRow(getCluster());
+ }
+
+ // This function creates a single dummy input row for queries that don't read from a table.
+ // For example: SELECT "hello"
+ // The code is copy-pasted from Calcite's LogicalValues.createOneRow() with a single line
+ // change: SqlTypeName.INTEGER replaced by SqlTypeName.BIGINT.
+ // Would like to call LogicalValues.createOneRow() directly, but it uses type SqlTypeName.INTEGER
+ // which corresponds to TypeKind.TYPE_INT32 in ZetaSQL, a type not supported in ZetaSQL
+ // PRODUCT_EXTERNAL mode. See
+ // https://github.com/google/zetasql/blob/c610a21ffdc110293c1c7bd255a2674ebc7ec7a8/java/com/google/zetasql/TypeFactory.java#L61
+ private static LogicalValues createOneRow(RelOptCluster cluster) {
+ final RelDataType rowType =
+ cluster.getTypeFactory().builder().add("ZERO", SqlTypeName.BIGINT).nullable(false).build();
+ final ImmutableList<ImmutableList<RexLiteral>> tuples =
+ ImmutableList.of(
+ ImmutableList.of(
+ cluster
+ .getRexBuilder()
+ .makeExactLiteral(BigDecimal.ZERO, rowType.getFieldList().get(0).getType())));
+ return LogicalValues.create(cluster, rowType, tuples);
}
}
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java
index 5cfd878..e140c02 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLDialectSpecTest.java
@@ -59,7 +59,6 @@
import org.apache.beam.sdk.extensions.sql.impl.JdbcConnection;
import org.apache.beam.sdk.extensions.sql.impl.JdbcDriver;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
-import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
@@ -142,6 +141,56 @@
}
@Test
+ public void testByteString() {
+ String sql = "SELECT @p0 IS NULL AS ColA";
+
+ ByteString byteString = ByteString.copyFrom(new byte[] {0x62});
+
+ ImmutableMap<String, Value> params =
+ ImmutableMap.<String, Value>builder().put("p0", Value.createBytesValue(byteString)).build();
+
+ ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+ BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params);
+ PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+ final Schema schema = Schema.builder().addNullableField("ColA", FieldType.BOOLEAN).build();
+
+ PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(false).build());
+
+ pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+ }
+
+ @Test
+ public void testFloat() {
+ String sql = "SELECT 3.0";
+
+ ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+ BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+ PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+ final Schema schema = Schema.builder().addNullableField("ColA", FieldType.DOUBLE).build();
+
+ PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(3.0).build());
+
+ pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+ }
+
+ @Test
+ public void testStringLiterals() {
+ String sql = "SELECT 'abc\\n'";
+
+ ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+ BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+ PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+ final Schema schema = Schema.builder().addNullableField("ColA", FieldType.STRING).build();
+
+ PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues("abc\n").build());
+
+ pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+ }
+
+ @Test
public void testEQ1() {
String sql = "SELECT @p0 = @p1 AS ColA";
@@ -3755,7 +3804,7 @@
.defaultSchema(defaultSchemaPlus)
.traitDefs(traitDefs)
.context(Contexts.of(contexts))
- .ruleSets(BeamRuleSets.getRuleSets())
+ .ruleSets(ZetaSQLQueryPlanner.getZetaSqlRuleSets())
.costFactory(BeamCostModel.FACTORY)
.typeSystem(jdbcConnection.getTypeFactory().getTypeSystem())
.build();
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
index a75db39..a96b957 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
@@ -27,7 +27,6 @@
import org.apache.beam.sdk.extensions.sql.impl.JdbcConnection;
import org.apache.beam.sdk.extensions.sql.impl.JdbcDriver;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
-import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
import org.apache.beam.sdk.extensions.sql.meta.Table;
@@ -187,7 +186,7 @@
.defaultSchema(defaultSchemaPlus)
.traitDefs(traitDefs)
.context(Contexts.of(contexts))
- .ruleSets(BeamRuleSets.getRuleSets())
+ .ruleSets(ZetaSQLQueryPlanner.getZetaSqlRuleSets())
.costFactory(BeamCostModel.FACTORY)
.typeSystem(jdbcConnection.getTypeFactory().getTypeSystem())
.build();
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java
new file mode 100644
index 0000000..a2da5c1
--- /dev/null
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlUtilsTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.extensions.sql.zetasql;
+
+import static org.junit.Assert.assertEquals;
+
+import com.google.protobuf.ByteString;
+import com.google.zetasql.ArrayType;
+import com.google.zetasql.StructType;
+import com.google.zetasql.StructType.StructField;
+import com.google.zetasql.TypeFactory;
+import com.google.zetasql.Value;
+import com.google.zetasql.ZetaSQLType.TypeKind;
+import java.util.Arrays;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.values.Row;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for utility methods for ZetaSQL related operations. */
+@RunWith(JUnit4.class)
+public class ZetaSqlUtilsTest {
+
+ private static final Schema TEST_INNER_SCHEMA =
+ Schema.builder().addField("i1", FieldType.INT64).addField("i2", FieldType.STRING).build();
+
+ private static final Schema TEST_SCHEMA =
+ Schema.builder()
+ .addField("f1", FieldType.INT64)
+ // .addField("f2", FieldType.DECIMAL)
+ .addField("f3", FieldType.DOUBLE)
+ .addField("f4", FieldType.STRING)
+ .addField("f5", FieldType.DATETIME)
+ .addField("f6", FieldType.BOOLEAN)
+ .addField("f7", FieldType.BYTES)
+ .addArrayField("f8", FieldType.DOUBLE)
+ .addRowField("f9", TEST_INNER_SCHEMA)
+ .addNullableField("f10", FieldType.INT64)
+ .build();
+
+ private static final FieldType TEST_FIELD_TYPE = FieldType.row(TEST_SCHEMA);
+
+ private static final ArrayType TEST_INNER_ARRAY_TYPE =
+ TypeFactory.createArrayType(TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE));
+
+ private static final StructType TEST_INNER_STRUCT_TYPE =
+ TypeFactory.createStructType(
+ Arrays.asList(
+ new StructField("i1", TypeFactory.createSimpleType(TypeKind.TYPE_INT64)),
+ new StructField("i2", TypeFactory.createSimpleType(TypeKind.TYPE_STRING))));
+
+ private static final StructType TEST_TYPE =
+ TypeFactory.createStructType(
+ Arrays.asList(
+ new StructField("f1", TypeFactory.createSimpleType(TypeKind.TYPE_INT64)),
+ // new StructField("f2", TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC)),
+ new StructField("f3", TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE)),
+ new StructField("f4", TypeFactory.createSimpleType(TypeKind.TYPE_STRING)),
+ new StructField("f5", TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP)),
+ new StructField("f6", TypeFactory.createSimpleType(TypeKind.TYPE_BOOL)),
+ new StructField("f7", TypeFactory.createSimpleType(TypeKind.TYPE_BYTES)),
+ new StructField("f8", TEST_INNER_ARRAY_TYPE),
+ new StructField("f9", TEST_INNER_STRUCT_TYPE),
+ new StructField("f10", TypeFactory.createSimpleType(TypeKind.TYPE_INT64))));
+
+ private static final Row TEST_ROW =
+ Row.withSchema(TEST_SCHEMA)
+ .addValue(64L)
+ // .addValue(BigDecimal.valueOf(9999L))
+ .addValue(5.0)
+ .addValue("Hello")
+ .addValue(Instant.ofEpochMilli(12345678L))
+ .addValue(false)
+ .addValue(new byte[] {0x11, 0x22})
+ .addArray(3.0, 6.5)
+ .addValue(Row.withSchema(TEST_INNER_SCHEMA).addValues(0L, "world").build())
+ .addValue(null)
+ .build();
+
+ private static final Value TEST_VALUE =
+ Value.createStructValue(
+ TEST_TYPE,
+ Arrays.asList(
+ Value.createInt64Value(64L),
+ // TODO[BEAM-8630]: Value.createNumericValue() is broken due to a dependency issue
+ // Value.createNumericValue(BigDecimal.valueOf(9999L)),
+ Value.createDoubleValue(5.0),
+ Value.createStringValue("Hello"),
+ Value.createTimestampValueFromUnixMicros(12345678000L),
+ Value.createBoolValue(false),
+ Value.createBytesValue(ByteString.copyFrom(new byte[] {0x11, 0x22})),
+ Value.createArrayValue(
+ TEST_INNER_ARRAY_TYPE,
+ Arrays.asList(Value.createDoubleValue(3.0), Value.createDoubleValue(6.5))),
+ Value.createStructValue(
+ TEST_INNER_STRUCT_TYPE,
+ Arrays.asList(Value.createInt64Value(0L), Value.createStringValue("world"))),
+ Value.createNullValue(TypeFactory.createSimpleType(TypeKind.TYPE_INT64))));
+
+ @Test
+ public void testBeamFieldTypeToZetaSqlType() {
+ assertEquals(ZetaSqlUtils.beamFieldTypeToZetaSqlType(TEST_FIELD_TYPE), TEST_TYPE);
+ }
+
+ @Test
+ public void testJavaObjectToZetaSqlValue() {
+ assertEquals(ZetaSqlUtils.javaObjectToZetaSqlValue(TEST_ROW, TEST_FIELD_TYPE), TEST_VALUE);
+ }
+
+ @Test
+ public void testZetaSqlValueToJavaObject() {
+ assertEquals(ZetaSqlUtils.zetaSqlValueToJavaObject(TEST_VALUE, TEST_FIELD_TYPE), TEST_ROW);
+ }
+}
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java
index 72ab5d6..bbc2916 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java
@@ -33,20 +33,26 @@
* {@link BeamFnApi.Elements} message when the buffer threshold is surpassed.
*
* <p>The default size-based buffer threshold can be overridden by specifying the experiment {@code
- * beam_fn_api_data_buffer_size_limit=<bytes>}
+ * data_buffer_size_limit=<bytes>}
*
* <p>The default time-based buffer threshold can be overridden by specifying the experiment {@code
- * beam_fn_api_data_buffer_time_limit=<milliseconds>}
+ * data_buffer_time_limit_ms=<milliseconds>}
*/
public interface BeamFnDataBufferingOutboundObserver<T> extends CloseableFnDataReceiver<T> {
// TODO: Consider moving this constant out of this interface
- /** @deprecated Use BEAM_FN_API_DATA_BUFFER_SIZE_LIMIT instead. */
+ /** @deprecated Use DATA_BUFFER_SIZE_LIMIT instead. */
@Deprecated String BEAM_FN_API_DATA_BUFFER_LIMIT = "beam_fn_api_data_buffer_limit=";
- String BEAM_FN_API_DATA_BUFFER_SIZE_LIMIT = "beam_fn_api_data_buffer_size_limit=";
+ /** @deprecated Use DATA_BUFFER_SIZE_LIMIT instead. */
+ @Deprecated String BEAM_FN_API_DATA_BUFFER_SIZE_LIMIT = "beam_fn_api_data_buffer_size_limit=";
+
+ String DATA_BUFFER_SIZE_LIMIT = "data_buffer_size_limit=";
@VisibleForTesting int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
- String BEAM_FN_API_DATA_BUFFER_TIME_LIMIT = "beam_fn_api_data_buffer_time_limit=";
+ /** @deprecated Use DATA_BUFFER_TIME_LIMIT_MS instead. */
+ @Deprecated String BEAM_FN_API_DATA_BUFFER_TIME_LIMIT = "beam_fn_api_data_buffer_time_limit=";
+
+ String DATA_BUFFER_TIME_LIMIT_MS = "data_buffer_time_limit_ms=";
long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
static <T> BeamFnDataSizeBasedBufferingOutboundObserver<T> forLocation(
@@ -68,6 +74,9 @@
static int getSizeLimit(PipelineOptions options) {
List<String> experiments = options.as(ExperimentalOptions.class).getExperiments();
for (String experiment : experiments == null ? Collections.<String>emptyList() : experiments) {
+ if (experiment.startsWith(DATA_BUFFER_SIZE_LIMIT)) {
+ return Integer.parseInt(experiment.substring(DATA_BUFFER_SIZE_LIMIT.length()));
+ }
if (experiment.startsWith(BEAM_FN_API_DATA_BUFFER_SIZE_LIMIT)) {
return Integer.parseInt(experiment.substring(BEAM_FN_API_DATA_BUFFER_SIZE_LIMIT.length()));
}
@@ -81,6 +90,9 @@
static long getTimeLimit(PipelineOptions options) {
List<String> experiments = options.as(ExperimentalOptions.class).getExperiments();
for (String experiment : experiments == null ? Collections.<String>emptyList() : experiments) {
+ if (experiment.startsWith(DATA_BUFFER_TIME_LIMIT_MS)) {
+ return Long.parseLong(experiment.substring(DATA_BUFFER_TIME_LIMIT_MS.length()));
+ }
if (experiment.startsWith(BEAM_FN_API_DATA_BUFFER_TIME_LIMIT)) {
return Long.parseLong(experiment.substring(BEAM_FN_API_DATA_BUFFER_TIME_LIMIT.length()));
}
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java
index dde456b..83f94f9 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java
@@ -28,7 +28,8 @@
public abstract class OutboundObserverFactory {
/**
* Create a buffering {@link OutboundObserverFactory} for client-side RPCs with the specified
- * {@link ExecutorService} and the default buffer size.
+ * {@link ExecutorService} and the default buffer size. All {@link StreamObserver}s created by
+ * this factory are thread safe.
*/
public static OutboundObserverFactory clientBuffered(ExecutorService executorService) {
return new Buffered(executorService, Buffered.DEFAULT_BUFFER_SIZE);
@@ -36,7 +37,8 @@
/**
* Create a buffering {@link OutboundObserverFactory} for client-side RPCs with the specified
- * {@link ExecutorService} and buffer size.
+ * {@link ExecutorService} and buffer size. All {@link StreamObserver}s created by this factory
+ * are thread safe.
*/
public static OutboundObserverFactory clientBuffered(
ExecutorService executorService, int bufferSize) {
@@ -45,8 +47,7 @@
/**
* Create the default {@link OutboundObserverFactory} for client-side RPCs, which uses basic
- * unbuffered flow control and adds synchronization to provide thread safety of access to the
- * returned observer.
+ * unbuffered flow control. All {@link StreamObserver}s created by this factory are thread safe.
*/
public static OutboundObserverFactory clientDirect() {
return new DirectClient();
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java
index 0e53b26..ed2f700 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java
@@ -130,7 +130,7 @@
PipelineOptions options = PipelineOptionsFactory.create();
options
.as(ExperimentalOptions.class)
- .setExperiments(Arrays.asList("beam_fn_api_data_buffer_size_limit=100"));
+ .setExperiments(Arrays.asList("data_buffer_size_limit=100"));
CloseableFnDataReceiver<WindowedValue<byte[]>> consumer =
BeamFnDataBufferingOutboundObserver.forLocation(
options,
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
index f4effa8..eaf6290 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
@@ -54,7 +54,7 @@
PipelineOptions options = PipelineOptionsFactory.create();
options
.as(ExperimentalOptions.class)
- .setExperiments(Arrays.asList("beam_fn_api_data_buffer_time_limit=1"));
+ .setExperiments(Arrays.asList("data_buffer_time_limit_ms=1"));
final CountDownLatch waitForFlush = new CountDownLatch(1);
CloseableFnDataReceiver<WindowedValue<byte[]>> consumer =
BeamFnDataBufferingOutboundObserver.forLocation(
@@ -80,7 +80,7 @@
PipelineOptions options = PipelineOptionsFactory.create();
options
.as(ExperimentalOptions.class)
- .setExperiments(Arrays.asList("beam_fn_api_data_buffer_time_limit=1"));
+ .setExperiments(Arrays.asList("data_buffer_time_limit_ms=1"));
BeamFnDataTimeBasedBufferingOutboundObserver<WindowedValue<byte[]>> consumer =
(BeamFnDataTimeBasedBufferingOutboundObserver<WindowedValue<byte[]>>)
BeamFnDataBufferingOutboundObserver.forLocation(
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
index fbdb95d..5240e0c 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
@@ -157,7 +157,7 @@
CombineFn<InputT, AccumT, ?> combineFn =
(CombineFn)
SerializableUtils.deserializeFromByteArray(
- combinePayload.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn");
+ combinePayload.getCombineFn().getPayload().toByteArray(), "CombineFn");
Coder<AccumT> accumCoder =
(Coder<AccumT>) rehydratedComponents.getCoder(combinePayload.getAccumulatorCoderId());
@@ -190,7 +190,7 @@
CombineFn<?, AccumT, ?> combineFn =
(CombineFn)
SerializableUtils.deserializeFromByteArray(
- combinePayload.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn");
+ combinePayload.getCombineFn().getPayload().toByteArray(), "CombineFn");
return (KV<KeyT, Iterable<AccumT>> input) ->
KV.of(input.getKey(), combineFn.mergeAccumulators(input.getValue()));
@@ -203,7 +203,7 @@
CombineFn<?, AccumT, OutputT> combineFn =
(CombineFn)
SerializableUtils.deserializeFromByteArray(
- combinePayload.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn");
+ combinePayload.getCombineFn().getPayload().toByteArray(), "CombineFn");
return (KV<KeyT, AccumT> input) ->
KV.of(input.getKey(), combineFn.extractOutput(input.getValue()));
@@ -217,7 +217,7 @@
CombineFn<InputT, AccumT, OutputT> combineFn =
(CombineFn)
SerializableUtils.deserializeFromByteArray(
- combinePayload.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn");
+ combinePayload.getCombineFn().getPayload().toByteArray(), "CombineFn");
return (KV<KeyT, Iterable<InputT>> input) -> {
return KV.of(input.getKey(), combineFn.apply(input.getValue()));
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
index 1aa5ba5..6ec1673 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
@@ -198,6 +198,9 @@
handlers.put(
BeamFnApi.InstructionRequest.RequestCase.PROCESS_BUNDLE,
processBundleHandler::processBundle);
+ handlers.put(
+ BeamFnApi.InstructionRequest.RequestCase.PROCESS_BUNDLE_SPLIT,
+ processBundleHandler::split);
BeamFnControlClient control =
new BeamFnControlClient(
id, controlApiServiceDescriptor, channelFactory, outboundObserverFactory, handlers);
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableProcessElementsRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableProcessElementsRunner.java
index 7670a9a..b32f28a 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableProcessElementsRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableProcessElementsRunner.java
@@ -50,7 +50,7 @@
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Timestamps;
+import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Durations;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
@@ -246,10 +246,8 @@
ImmutableList.of(
DelayedBundleApplication.newBuilder()
.setApplication(residualApplication)
- .setRequestedExecutionTime(
- Timestamps.fromMillis(
- System.currentTimeMillis()
- + result.getContinuation().resumeDelay().getMillis()))
+ .setRequestedTimeDelay(
+ Durations.fromMillis(result.getContinuation().resumeDelay().getMillis()))
.build()));
}
}
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java
index 7fd71da..bdebcb4 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMappingFnRunner.java
@@ -20,8 +20,8 @@
import com.google.auto.service.AutoService;
import java.io.IOException;
import java.util.Map;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms;
import org.apache.beam.runners.core.construction.BeamUrns;
import org.apache.beam.runners.core.construction.PCollectionViewTranslation;
@@ -65,8 +65,7 @@
static <T, W1 extends BoundedWindow, W2 extends BoundedWindow>
ThrowingFunction<KV<T, W1>, KV<T, W2>> createMapFunctionForPTransform(
String ptransformId, PTransform pTransform) throws IOException {
- SdkFunctionSpec windowMappingFnPayload =
- SdkFunctionSpec.parseFrom(pTransform.getSpec().getPayload());
+ FunctionSpec windowMappingFnPayload = FunctionSpec.parseFrom(pTransform.getSpec().getPayload());
WindowMappingFn<W2> windowMappingFn =
(WindowMappingFn<W2>)
PCollectionViewTranslation.windowMappingFnFromProto(windowMappingFnPayload);
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java
index d97f8de..ec79163 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/WindowMergingFnRunner.java
@@ -75,8 +75,8 @@
ThrowingFunction<KV<T, Iterable<W>>, KV<T, KV<Iterable<W>, Iterable<KV<W, Iterable<W>>>>>>
createMapFunctionForPTransform(String ptransformId, PTransform ptransform)
throws IOException {
- RunnerApi.SdkFunctionSpec payload =
- RunnerApi.SdkFunctionSpec.parseFrom(ptransform.getSpec().getPayload());
+ RunnerApi.FunctionSpec payload =
+ RunnerApi.FunctionSpec.parseFrom(ptransform.getSpec().getPayload());
WindowFn<?, W> windowFn =
(WindowFn<?, W>) WindowingStrategyTranslation.windowFnFromProto(payload);
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
index 11222859..a258e01 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
@@ -21,11 +21,13 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.ServiceLoader;
import java.util.Set;
+import java.util.WeakHashMap;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Phaser;
@@ -77,12 +79,20 @@
import org.slf4j.LoggerFactory;
/**
- * Processes {@link BeamFnApi.ProcessBundleRequest}s by materializing the set of required runners
- * for each {@link RunnerApi.FunctionSpec}, wiring them together based upon the {@code input} and
- * {@code output} map definitions.
+ * Processes {@link BeamFnApi.ProcessBundleRequest}s and {@link
+ * BeamFnApi.ProcessBundleSplitRequest}s.
*
- * <p>Finally executes the DAG based graph by starting all runners in reverse topological order, and
- * finishing all runners in forward topological order.
+ * <p>{@link BeamFnApi.ProcessBundleSplitRequest}s use a {@link BundleProcessorCache cache} to
+ * find/create a {@link BundleProcessor}. The creation of a {@link BundleProcessor} uses the
+ * associated {@link BeamFnApi.ProcessBundleDescriptor} definition; creating runners for each {@link
+ * RunnerApi.FunctionSpec}; wiring them together based upon the {@code input} and {@code output} map
+ * definitions. The {@link BundleProcessor} executes the DAG based graph by starting all runners in
+ * reverse topological order, and finishing all runners in forward topological order.
+ *
+ * <p>{@link BeamFnApi.ProcessBundleSplitRequest}s finds an {@code active} {@link BundleProcessor}
+ * associated with a currently processing {@link BeamFnApi.ProcessBundleRequest} and uses it to
+ * perform a split request. See <a href="https://s.apache.org/beam-breaking-fusion">breaking the
+ * fusion barrier</a> for further details.
*/
public class ProcessBundleHandler {
@@ -231,6 +241,7 @@
BundleProcessor bundleProcessor =
bundleProcessorCache.get(
request.getProcessBundle().getProcessBundleDescriptorId(),
+ request.getInstructionId(),
() -> {
try {
return createBundleProcessor(
@@ -240,7 +251,6 @@
throw new RuntimeException(e);
}
});
- bundleProcessor.setInstructionId(request.getInstructionId());
PTransformFunctionRegistry startFunctionRegistry = bundleProcessor.getStartFunctionRegistry();
PTransformFunctionRegistry finishFunctionRegistry = bundleProcessor.getFinishFunctionRegistry();
Multimap<String, DelayedBundleApplication> allResiduals = bundleProcessor.getAllResiduals();
@@ -294,6 +304,19 @@
return BeamFnApi.InstructionResponse.newBuilder().setProcessBundle(response);
}
+ /** Splits an active bundle. */
+ public BeamFnApi.InstructionResponse.Builder split(BeamFnApi.InstructionRequest request) {
+ BundleProcessor bundleProcessor =
+ bundleProcessorCache.find(request.getProcessBundleSplit().getInstructionId());
+ if (bundleProcessor == null) {
+ throw new IllegalStateException(
+ String.format(
+ "Unable to find active bundle for instruction id %s.",
+ request.getProcessBundleSplit().getInstructionId()));
+ }
+ throw new UnsupportedOperationException("TODO: BEAM-3836, support splitting within SDK.");
+ }
+
/** Shutdown the bundles, running the tearDown() functions. */
public void shutdown() throws Exception {
bundleProcessorCache.shutdown();
@@ -406,9 +429,18 @@
public static class BundleProcessorCache {
private final Map<String, ConcurrentLinkedQueue<BundleProcessor>> cachedBundleProcessors;
+ private final Map<String, BundleProcessor> activeBundleProcessors;
+
+ @Override
+ public int hashCode() {
+ return super.hashCode();
+ }
BundleProcessorCache() {
this.cachedBundleProcessors = Maps.newConcurrentMap();
+ // We specifically use a weak hash map so that references will automatically go out of scope
+ // and not need to be freed explicitly from the cache.
+ this.activeBundleProcessors = Collections.synchronizedMap(new WeakHashMap<>());
}
Map<String, ConcurrentLinkedQueue<BundleProcessor>> getCachedBundleProcessors() {
@@ -417,26 +449,43 @@
/**
* Get a {@link BundleProcessor} from the cache if it's available. Otherwise, create one using
- * the specified bundleProcessorSupplier.
+ * the specified {@code bundleProcessorSupplier}. The {@link BundleProcessor} that is returned
+ * can be {@link #find found} using the specified method.
+ *
+ * <p>The caller is responsible for calling {@link #release} to return the bundle processor back
+ * to this cache if and only if the bundle processor successfully processed a bundle.
*/
BundleProcessor get(
- String bundleDescriptorId, Supplier<BundleProcessor> bundleProcessorSupplier) {
+ String bundleDescriptorId,
+ String instructionId,
+ Supplier<BundleProcessor> bundleProcessorSupplier) {
ConcurrentLinkedQueue<BundleProcessor> bundleProcessors =
cachedBundleProcessors.computeIfAbsent(
bundleDescriptorId, descriptorId -> new ConcurrentLinkedQueue<>());
BundleProcessor bundleProcessor = bundleProcessors.poll();
- if (bundleProcessor != null) {
- return bundleProcessor;
+ if (bundleProcessor == null) {
+ bundleProcessor = bundleProcessorSupplier.get();
}
- return bundleProcessorSupplier.get();
+ bundleProcessor.setInstructionId(instructionId);
+ activeBundleProcessors.put(instructionId, bundleProcessor);
+ return bundleProcessor;
+ }
+
+ /**
+ * Finds an active bundle processor for the specified {@code instructionId} or null if one could
+ * not be found.
+ */
+ BundleProcessor find(String instructionId) {
+ return activeBundleProcessors.get(instructionId);
}
/**
* Add a {@link BundleProcessor} to cache. The {@link BundleProcessor} will be reset before
- * being added to the cache.
+ * being added to the cache and will be marked as inactive.
*/
void release(String bundleDescriptorId, BundleProcessor bundleProcessor) {
+ activeBundleProcessors.remove(bundleProcessor.getInstructionId());
bundleProcessor.reset();
cachedBundleProcessors.get(bundleDescriptorId).add(bundleProcessor);
}
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java
index 15b28f6..7f21991 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java
@@ -29,6 +29,11 @@
* to use in the java SDK harness.
*/
public abstract class HarnessStreamObserverFactories {
+
+ /**
+ * Creates an {@link OutboundObserverFactory} for client-side RPCs. All {@link StreamObserver}s
+ * created by this factory are thread safe.
+ */
public static OutboundObserverFactory fromOptions(PipelineOptions options) {
List<String> experiments = options.as(ExperimentalOptions.class).getExperiments();
if (experiments != null && experiments.contains("beam_fn_api_buffered_stream")) {
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
index 1a460e6..a7a233d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
@@ -21,6 +21,8 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
@@ -222,8 +224,11 @@
@Override
BundleProcessor get(
- String bundleDescriptorId, Supplier<BundleProcessor> bundleProcessorSupplier) {
- return new TestBundleProcessor(super.get(bundleDescriptorId, bundleProcessorSupplier));
+ String bundleDescriptorId,
+ String instructionId,
+ Supplier<BundleProcessor> bundleProcessorSupplier) {
+ return new TestBundleProcessor(
+ super.get(bundleDescriptorId, instructionId, bundleProcessorSupplier));
}
}
@@ -325,9 +330,7 @@
SerializableUtils.serializeToByteArray(doFnWithExecutionInformation)))
.build();
RunnerApi.ParDoPayload parDoPayload =
- RunnerApi.ParDoPayload.newBuilder()
- .setDoFn(RunnerApi.SdkFunctionSpec.newBuilder().setSpec(functionSpec))
- .build();
+ RunnerApi.ParDoPayload.newBuilder().setDoFn(functionSpec).build();
BeamFnApi.ProcessBundleDescriptor processBundleDescriptor =
BeamFnApi.ProcessBundleDescriptor.newBuilder()
.putTransforms(
@@ -356,11 +359,8 @@
WindowingStrategy.newBuilder()
.setWindowCoderId("window-strategy-coder")
.setWindowFn(
- RunnerApi.SdkFunctionSpec.newBuilder()
- .setSpec(
- RunnerApi.FunctionSpec.newBuilder()
- .setUrn("beam:windowfn:global_windows:v0.1"))
- .build())
+ RunnerApi.FunctionSpec.newBuilder()
+ .setUrn("beam:windowfn:global_windows:v0.1"))
.setOutputTime(RunnerApi.OutputTime.Enum.END_OF_WINDOW)
.setAccumulationMode(RunnerApi.AccumulationMode.Enum.ACCUMULATING)
.setTrigger(
@@ -488,6 +488,24 @@
}
@Test
+ public void testBundleProcessorIsFoundWhenActive() {
+ BundleProcessor bundleProcessor = mock(BundleProcessor.class);
+ when(bundleProcessor.getInstructionId()).thenReturn("known");
+ BundleProcessorCache cache = new BundleProcessorCache();
+
+ // Check that an unknown bundle processor is not found
+ assertNull(cache.find("unknown"));
+
+ // Once it is active, ensure the bundle processor is found
+ cache.get("descriptorId", "known", () -> bundleProcessor);
+ assertSame(bundleProcessor, cache.find("known"));
+
+ // After it is released, ensure the bundle processor is no longer found
+ cache.release("descriptorId", bundleProcessor);
+ assertNull(cache.find("known"));
+ }
+
+ @Test
public void testBundleProcessorReset() {
PTransformFunctionRegistry startFunctionRegistry = mock(PTransformFunctionRegistry.class);
PTransformFunctionRegistry finishFunctionRegistry = mock(PTransformFunctionRegistry.class);
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
index 672d41b..deb6218 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
@@ -292,7 +292,7 @@
BeamFnDataGrpcClient clientFactory =
new BeamFnDataGrpcClient(
PipelineOptionsFactory.fromArgs(
- new String[] {"--experiments=beam_fn_api_data_buffer_size_limit=20"})
+ new String[] {"--experiments=data_buffer_size_limit=20"})
.create(),
(Endpoints.ApiServiceDescriptor descriptor) -> channel,
OutboundObserverFactory.trivial());
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index 85419c6..89d0b28 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -19,14 +19,13 @@
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.ES_TYPE;
-import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.UPDATE_INDEX;
-import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.UPDATE_TYPE;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.getEsIndex;
import java.io.IOException;
import java.io.Serializable;
import org.apache.beam.sdk.io.common.NetworkTestHelper;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.elasticsearch.client.Request;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.Node;
@@ -86,15 +85,16 @@
connectionConfiguration =
ConnectionConfiguration.create(
new String[] {"http://" + ES_IP + ":" + esHttpPort}, getEsIndex(), ES_TYPE)
- .withSocketAndRetryTimeout(120000)
+ .withSocketTimeout(120000)
.withConnectTimeout(5000);
restClient = connectionConfiguration.createClient();
elasticsearchIOTestCommon =
new ElasticsearchIOTestCommon(connectionConfiguration, restClient, false);
int waitingTime = 0;
int healthCheckFrequency = 500;
+ Request request = new Request("HEAD", "/");
while ((waitingTime < MAX_STARTUP_WAITING_TIME_MSEC)
- && restClient.performRequest("HEAD", "/").getStatusLine().getStatusCode() != 200) {
+ && restClient.performRequest(request).getStatusLine().getStatusCode() != 200) {
try {
Thread.sleep(healthCheckFrequency);
waitingTime += healthCheckFrequency;
@@ -202,18 +202,6 @@
}
@Test
- public void testWritePartialUpdateWithErrors() throws Exception {
- // cannot share elasticsearchIOTestCommon because tests run in parallel.
- ConnectionConfiguration connectionConfiguration =
- ConnectionConfiguration.create(
- new String[] {"http://" + ES_IP + ":" + esHttpPort}, UPDATE_INDEX, UPDATE_TYPE);
- ElasticsearchIOTestCommon elasticsearchIOTestCommonWithErrors =
- new ElasticsearchIOTestCommon(connectionConfiguration, restClient, false);
- elasticsearchIOTestCommonWithErrors.setPipeline(pipeline);
- elasticsearchIOTestCommonWithErrors.testWritePartialUpdateWithErrors();
- }
-
- @Test
public void testReadWithMetadata() throws Exception {
elasticsearchIOTestCommon.setPipeline(pipeline);
elasticsearchIOTestCommon.testReadWithMetadata();
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index d809cfd..4f2fc28 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -19,8 +19,6 @@
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.ES_TYPE;
-import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.UPDATE_INDEX;
-import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.UPDATE_TYPE;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.getEsIndex;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
@@ -95,7 +93,7 @@
if (connectionConfiguration == null) {
connectionConfiguration =
ConnectionConfiguration.create(fillAddresses(), getEsIndex(), ES_TYPE)
- .withSocketAndRetryTimeout(120000)
+ .withSocketTimeout(120000)
.withConnectTimeout(5000);
elasticsearchIOTestCommon =
new ElasticsearchIOTestCommon(connectionConfiguration, getRestClient(), false);
@@ -202,17 +200,6 @@
}
@Test
- public void testWritePartialUpdateWithErrors() throws Exception {
- // cannot share elasticsearchIOTestCommon because tests run in parallel.
- ConnectionConfiguration connectionConfiguration =
- ConnectionConfiguration.create(fillAddresses(), UPDATE_INDEX, UPDATE_TYPE);
- ElasticsearchIOTestCommon elasticsearchIOTestCommonWithErrors =
- new ElasticsearchIOTestCommon(connectionConfiguration, getRestClient(), false);
- elasticsearchIOTestCommonWithErrors.setPipeline(pipeline);
- elasticsearchIOTestCommonWithErrors.testWritePartialUpdateWithErrors();
- }
-
- @Test
public void testReadWithMetadata() throws Exception {
elasticsearchIOTestCommon.setPipeline(pipeline);
elasticsearchIOTestCommon.testReadWithMetadata();
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index 84696e5..279fa1e 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -19,8 +19,6 @@
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.ES_TYPE;
-import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.UPDATE_INDEX;
-import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.UPDATE_TYPE;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.getEsIndex;
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
@@ -95,7 +93,7 @@
if (connectionConfiguration == null) {
connectionConfiguration =
ConnectionConfiguration.create(fillAddresses(), getEsIndex(), ES_TYPE)
- .withSocketAndRetryTimeout(120000)
+ .withSocketTimeout(120000)
.withConnectTimeout(5000);
elasticsearchIOTestCommon =
new ElasticsearchIOTestCommon(connectionConfiguration, getRestClient(), false);
@@ -196,17 +194,6 @@
}
@Test
- public void testWritePartialUpdateWithErrors() throws Exception {
- // cannot share elasticsearchIOTestCommon because tests run in parallel.
- ConnectionConfiguration connectionConfiguration =
- ConnectionConfiguration.create(fillAddresses(), UPDATE_INDEX, UPDATE_TYPE);
- ElasticsearchIOTestCommon elasticsearchIOTestCommonWithErrors =
- new ElasticsearchIOTestCommon(connectionConfiguration, getRestClient(), false);
- elasticsearchIOTestCommonWithErrors.setPipeline(pipeline);
- elasticsearchIOTestCommonWithErrors.testWritePartialUpdateWithErrors();
- }
-
- @Test
public void testReadWithMetadata() throws Exception {
elasticsearchIOTestCommon.setPipeline(pipeline);
elasticsearchIOTestCommon.testReadWithMetadata();
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle
new file mode 100644
index 0000000..924b3fc
--- /dev/null
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+applyJavaNature(
+ publish: false,
+ archivesBaseName: 'beam-sdks-java-io-elasticsearch-tests-7'
+)
+provideIntegrationTestingDependencies()
+enableJavaPerformanceTesting()
+
+description = "Apache Beam :: SDKs :: Java :: IO :: Elasticsearch-Tests :: 7.x"
+ext.summary = "Tests of ElasticsearchIO on Elasticsearch 7.x"
+
+test {
+ // needed for ESIntegTestCase
+ systemProperty "tests.security.manager", "false"
+}
+
+def jna_version = "4.1.0"
+def log4j_version = "2.11.1"
+def elastic_search_version = "7.5.0"
+
+configurations.all {
+ resolutionStrategy {
+ // Make sure the log4j versions for api and core match instead of taking the default
+ // Gradle rule of using the latest.
+ force "org.apache.logging.log4j:log4j-api:$log4j_version"
+ force "org.apache.logging.log4j:log4j-core:$log4j_version"
+ }
+}
+
+dependencies {
+ testCompile project(path: ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-common", configuration: "testRuntime")
+ testCompile "org.elasticsearch.test:framework:$elastic_search_version"
+ testCompile "org.elasticsearch.plugin:transport-netty4-client:$elastic_search_version"
+ testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:2.7.5"
+ testCompile "org.elasticsearch:elasticsearch:$elastic_search_version"
+
+ testCompile project(path: ":sdks:java:core", configuration: "shadow")
+ testCompile project(":sdks:java:io:elasticsearch")
+ testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
+ testCompile "org.apache.logging.log4j:log4j-core:$log4j_version"
+ testCompile "org.apache.logging.log4j:log4j-api:$log4j_version"
+ testCompile library.java.slf4j_api
+ testCompile "net.java.dev.jna:jna:$jna_version"
+ testCompile library.java.hamcrest_core
+ testCompile library.java.hamcrest_library
+ testCompile library.java.commons_io_1x
+ testCompile library.java.junit
+ testCompile "org.elasticsearch.client:elasticsearch-rest-client:$elastic_search_version"
+ testRuntimeOnly library.java.slf4j_jdk14
+ testRuntimeOnly project(":runners:direct-java")
+}
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/contrib/create_elk_container.sh b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/contrib/create_elk_container.sh
new file mode 100755
index 0000000..7a2862d
--- /dev/null
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/contrib/create_elk_container.sh
@@ -0,0 +1,24 @@
+#!/bin/sh
+################################################################################
+#
+# 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.
+#
+################################################################################
+
+#Create an ELK (Elasticsearch Logstash Kibana) container for ES v6.4.0 and compatible Logstash and Kibana versions,
+#bind then on host ports, allow shell access to container and mount current directory on /home/$USER inside the container
+
+docker create -p 5601:5601 -p 9200:9200 -p 5044:5044 -p 5000:5000 -p 9300:9300 -it -v $(pwd):/home/$USER/ --name elk-7.5.0 sebp/elk:740
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
new file mode 100644
index 0000000..42ae6d5
--- /dev/null
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
@@ -0,0 +1,147 @@
+/*
+ * 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.io.elasticsearch;
+
+import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration;
+
+import org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOITCommon.ElasticsearchPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.elasticsearch.client.RestClient;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * A test of {@link ElasticsearchIO} on an independent Elasticsearch v7.x instance.
+ *
+ * <p>This test requires a running instance of Elasticsearch, and the test dataset must exist in the
+ * database. See {@link ElasticsearchIOITCommon} for instructions to achieve this.
+ *
+ * <p>You can run this test by doing the following from the beam parent module directory with the
+ * correct server IP:
+ *
+ * <pre>
+ * ./gradlew integrationTest -p sdks/java/io/elasticsearch-tests/elasticsearch-tests-7
+ * -DintegrationTestPipelineOptions='[
+ * "--elasticsearchServer=1.2.3.4",
+ * "--elasticsearchHttpPort=9200"]'
+ * --tests org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOIT
+ * -DintegrationTestRunner=direct
+ * </pre>
+ *
+ * <p>It is likely that you will need to configure <code>thread_pool.write.queue_size: 250</code>
+ * (or higher) in the backend Elasticsearch server for this test to run.
+ */
+@RunWith(JUnit4.class)
+public class ElasticsearchIOIT {
+ private static RestClient restClient;
+ private static ElasticsearchPipelineOptions options;
+ private static ConnectionConfiguration readConnectionConfiguration;
+ private static ConnectionConfiguration writeConnectionConfiguration;
+ private static ConnectionConfiguration updateConnectionConfiguration;
+ private static ElasticsearchIOTestCommon elasticsearchIOTestCommon;
+
+ @Rule public TestPipeline pipeline = TestPipeline.create();
+
+ @BeforeClass
+ public static void beforeClass() throws Exception {
+ PipelineOptionsFactory.register(ElasticsearchPipelineOptions.class);
+ options = TestPipeline.testingPipelineOptions().as(ElasticsearchPipelineOptions.class);
+ readConnectionConfiguration =
+ ElasticsearchIOITCommon.getConnectionConfiguration(
+ options, ElasticsearchIOITCommon.IndexMode.READ);
+ writeConnectionConfiguration =
+ ElasticsearchIOITCommon.getConnectionConfiguration(
+ options, ElasticsearchIOITCommon.IndexMode.WRITE);
+ updateConnectionConfiguration =
+ ElasticsearchIOITCommon.getConnectionConfiguration(
+ options, ElasticsearchIOITCommon.IndexMode.WRITE_PARTIAL);
+ restClient = readConnectionConfiguration.createClient();
+ elasticsearchIOTestCommon =
+ new ElasticsearchIOTestCommon(readConnectionConfiguration, restClient, true);
+ }
+
+ @AfterClass
+ public static void afterClass() throws Exception {
+ ElasticsearchIOTestUtils.deleteIndex(writeConnectionConfiguration, restClient);
+ ElasticsearchIOTestUtils.deleteIndex(updateConnectionConfiguration, restClient);
+ restClient.close();
+ }
+
+ @Test
+ public void testSplitsVolume() throws Exception {
+ elasticsearchIOTestCommon.testSplit(10_000);
+ }
+
+ @Test
+ public void testReadVolume() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testRead();
+ }
+
+ @Test
+ public void testWriteVolume() throws Exception {
+ // cannot share elasticsearchIOTestCommon because tests run in parallel.
+ ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite =
+ new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true);
+ elasticsearchIOTestCommonWrite.setPipeline(pipeline);
+ elasticsearchIOTestCommonWrite.testWrite();
+ }
+
+ @Test
+ public void testSizesVolume() throws Exception {
+ elasticsearchIOTestCommon.testSizes();
+ }
+
+ /**
+ * This test verifies volume loading of Elasticsearch using explicit document IDs and routed to an
+ * index named the same as the scientist, and type which is based on the modulo 2 of the scientist
+ * name. The goal of this IT is to help observe and verify that the overhead of adding the
+ * functions to parse the document and extract the ID is acceptable.
+ */
+ @Test
+ public void testWriteWithFullAddressingVolume() throws Exception {
+ // cannot share elasticsearchIOTestCommon because tests run in parallel.
+ ElasticsearchIOTestCommon elasticsearchIOTestCommonWrite =
+ new ElasticsearchIOTestCommon(writeConnectionConfiguration, restClient, true);
+ elasticsearchIOTestCommonWrite.setPipeline(pipeline);
+ elasticsearchIOTestCommonWrite.testWriteWithFullAddressing();
+ }
+
+ /**
+ * This test verifies volume partial updates of Elasticsearch. The test dataset index is cloned
+ * and then a new field is added to each document using a partial update. The test then asserts
+ * the updates were applied.
+ */
+ @Test
+ public void testWritePartialUpdate() throws Exception {
+ ElasticsearchIOTestUtils.copyIndex(
+ restClient,
+ readConnectionConfiguration.getIndex(),
+ updateConnectionConfiguration.getIndex());
+ // cannot share elasticsearchIOTestCommon because tests run in parallel.
+ ElasticsearchIOTestCommon elasticsearchIOTestCommonUpdate =
+ new ElasticsearchIOTestCommon(updateConnectionConfiguration, restClient, true);
+ elasticsearchIOTestCommonUpdate.setPipeline(pipeline);
+ elasticsearchIOTestCommonUpdate.testWritePartialUpdate();
+ }
+}
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
new file mode 100644
index 0000000..2ccbda3
--- /dev/null
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -0,0 +1,223 @@
+/*
+ * 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.io.elasticsearch;
+
+import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.ConnectionConfiguration;
+import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.ES_TYPE;
+import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestCommon.getEsIndex;
+import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.elasticsearch.transport.Netty4Plugin;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/*
+Cannot use @RunWith(JUnit4.class) with ESIntegTestCase
+Cannot have @BeforeClass @AfterClass with ESIntegTestCase
+*/
+
+/** Tests for {@link ElasticsearchIO} version 7. */
+@ThreadLeakScope(ThreadLeakScope.Scope.NONE)
+// use cluster of 1 node that has data + master roles
+@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, supportsDedicatedMasters = false)
+public class ElasticsearchIOTest extends ESIntegTestCase implements Serializable {
+
+ private ElasticsearchIOTestCommon elasticsearchIOTestCommon;
+ private ConnectionConfiguration connectionConfiguration;
+
+ private String[] fillAddresses() {
+ ArrayList<String> result = new ArrayList<>();
+ for (InetSocketAddress address : cluster().httpAddresses()) {
+ result.add(String.format("http://%s:%s", address.getHostString(), address.getPort()));
+ }
+ return result.toArray(new String[result.size()]);
+ }
+
+ @Override
+ protected boolean addMockHttpTransport() {
+ return false;
+ }
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ System.setProperty("es.set.netty.runtime.available.processors", "false");
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ // had problems with some jdk, embedded ES was too slow for bulk insertion,
+ // and queue of 50 was full. No pb with real ES instance (cf testWrite integration test)
+ .put("thread_pool.write.queue_size", 400)
+ .build();
+ }
+
+ @Override
+ public Settings indexSettings() {
+ return Settings.builder()
+ .put(super.indexSettings())
+ // useful to have updated sizes for getEstimatedSize
+ .put("index.store.stats_refresh_interval", 0)
+ .build();
+ }
+
+ @Override
+ protected Collection<Class<? extends Plugin>> nodePlugins() {
+ ArrayList<Class<? extends Plugin>> plugins = new ArrayList<>();
+ plugins.add(Netty4Plugin.class);
+ return plugins;
+ }
+
+ @Before
+ public void setup() throws IOException {
+ if (connectionConfiguration == null) {
+ connectionConfiguration =
+ ConnectionConfiguration.create(fillAddresses(), getEsIndex(), ES_TYPE)
+ .withSocketTimeout(120000)
+ .withConnectTimeout(5000);
+ elasticsearchIOTestCommon =
+ new ElasticsearchIOTestCommon(connectionConfiguration, getRestClient(), false);
+ }
+ }
+
+ @Rule public TestPipeline pipeline = TestPipeline.create();
+
+ @Test
+ public void testSizes() throws Exception {
+ // need to create the index using the helper method (not create it at first insertion)
+ // for the indexSettings() to be run
+ createIndex(getEsIndex());
+ elasticsearchIOTestCommon.testSizes();
+ }
+
+ @Test
+ public void testRead() throws Exception {
+ // need to create the index using the helper method (not create it at first insertion)
+ // for the indexSettings() to be run
+ createIndex(getEsIndex());
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testRead();
+ }
+
+ @Test
+ public void testReadWithQueryString() throws Exception {
+ // need to create the index using the helper method (not create it at first insertion)
+ // for the indexSettings() to be run
+ createIndex(getEsIndex());
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testReadWithQueryString();
+ }
+
+ @Test
+ public void testReadWithQueryValueProvider() throws Exception {
+ // need to create the index using the helper method (not create it at first insertion)
+ // for the indexSettings() to be run
+ createIndex(getEsIndex());
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testReadWithQueryValueProvider();
+ }
+
+ @Test
+ public void testWrite() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWrite();
+ }
+
+ @Rule public ExpectedException expectedException = ExpectedException.none();
+
+ @Test
+ public void testWriteWithErrors() throws Exception {
+ elasticsearchIOTestCommon.setExpectedException(expectedException);
+ elasticsearchIOTestCommon.testWriteWithErrors();
+ }
+
+ @Test
+ public void testWriteWithMaxBatchSize() throws Exception {
+ elasticsearchIOTestCommon.testWriteWithMaxBatchSize();
+ }
+
+ @Test
+ public void testWriteWithMaxBatchSizeBytes() throws Exception {
+ elasticsearchIOTestCommon.testWriteWithMaxBatchSizeBytes();
+ }
+
+ @Test
+ public void testSplit() throws Exception {
+ // need to create the index using the helper method (not create it at first insertion)
+ // for the indexSettings() to be run
+ createIndex(getEsIndex());
+ elasticsearchIOTestCommon.testSplit(2_000);
+ }
+
+ @Test
+ public void testWriteWithIdFn() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWriteWithIdFn();
+ }
+
+ @Test
+ public void testWriteWithIndexFn() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWriteWithIndexFn();
+ }
+
+ @Test
+ public void testWriteFullAddressing() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWriteWithFullAddressing();
+ }
+
+ @Test
+ public void testWritePartialUpdate() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWritePartialUpdate();
+ }
+
+ @Test
+ public void testReadWithMetadata() throws Exception {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testReadWithMetadata();
+ }
+
+ @Test
+ public void testDefaultRetryPredicate() throws IOException {
+ elasticsearchIOTestCommon.testDefaultRetryPredicate(getRestClient());
+ }
+
+ @Test
+ public void testWriteRetry() throws Throwable {
+ elasticsearchIOTestCommon.setExpectedException(expectedException);
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWriteRetry();
+ }
+
+ @Test
+ public void testWriteRetryValidRequest() throws Throwable {
+ elasticsearchIOTestCommon.setPipeline(pipeline);
+ elasticsearchIOTestCommon.testWriteRetryValidRequest();
+ }
+}
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/elasticsearch/bootstrap/JarHell.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/elasticsearch/bootstrap/JarHell.java
new file mode 100644
index 0000000..be74371
--- /dev/null
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/elasticsearch/bootstrap/JarHell.java
@@ -0,0 +1,36 @@
+/*
+ * 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.elasticsearch.bootstrap;
+
+import java.util.function.Consumer;
+
+/**
+ * We need a real Elasticsearch instance to properly test the IO (split, slice API, scroll API,
+ * ...). Starting at ES 5, to have Elasticsearch embedded, we are forced to use Elasticsearch test
+ * framework. But this framework checks for class duplicates in classpath and it cannot be
+ * deactivated. When the class duplication come from a dependency, then it cannot be avoided.
+ * Elasticsearch community does not provide a way of deactivating the jar hell test, so skip it by
+ * making this hack. In this case duplicate class is class:
+ * org.apache.maven.surefire.report.SafeThrowable jar1: surefire-api-2.20.jar jar2:
+ * surefire-junit47-2.20.jar
+ */
+class JarHell {
+
+ @SuppressWarnings("EmptyMethod")
+ public static void checkJarHell(Consumer<String> output) {}
+}
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle
index 53a1ff4..a77a29a 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle
@@ -26,15 +26,20 @@
ext.summary = "Common test classes for ElasticsearchIO"
def jna_version = "4.1.0"
-def log4j_version = "2.6.2"
+def log4j_version = "2.11.1"
+def elastic_search_version = "7.5.0"
+
+configurations.all {
+ resolutionStrategy {
+ // Make sure the log4j versions for api and core match instead of taking the default
+ // Gradle rule of using the latest.
+ force "org.apache.logging.log4j:log4j-api:$log4j_version"
+ force "org.apache.logging.log4j:log4j-core:$log4j_version"
+ }
+}
dependencies {
testCompile library.java.jackson_databind
- testCompile "org.apache.httpcomponents:httpasyncclient:4.1.4"
- testCompile "org.apache.httpcomponents:httpcore-nio:4.4.10"
- testCompile "org.apache.httpcomponents:httpcore:4.4.10"
- testCompile "org.apache.httpcomponents:httpclient:4.5.6"
-
testCompile project(path: ":sdks:java:core", configuration: "shadow")
testCompile project(":sdks:java:io:elasticsearch")
testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
@@ -46,7 +51,7 @@
testCompile library.java.hamcrest_library
testCompile library.java.commons_io_1x
testCompile library.java.junit
- testCompile "org.elasticsearch.client:elasticsearch-rest-client:6.4.0"
+ testCompile "org.elasticsearch.client:elasticsearch-rest-client:$elastic_search_version"
testRuntimeOnly library.java.slf4j_jdk14
testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow")
}
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
index 386a518..de1f8b0 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
@@ -22,6 +22,7 @@
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.Read;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.RetryConfiguration.DEFAULT_RETRY_PREDICATE;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.Write;
+import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIO.getBackendVersion;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.FAMOUS_SCIENTISTS;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.NUM_SCIENTISTS;
import static org.apache.beam.sdk.io.elasticsearch.ElasticsearchIOTestUtils.countByMatch;
@@ -63,6 +64,7 @@
import org.apache.http.HttpEntity;
import org.apache.http.entity.ContentType;
import org.apache.http.nio.entity.NStringEntity;
+import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response;
import org.elasticsearch.client.RestClient;
import org.hamcrest.CustomMatcher;
@@ -440,7 +442,11 @@
for (String scientist : FAMOUS_SCIENTISTS) {
String index = scientist.toLowerCase();
long count =
- refreshIndexAndGetCurrentNumDocs(restClient, index, connectionConfiguration.getType());
+ refreshIndexAndGetCurrentNumDocs(
+ restClient,
+ index,
+ connectionConfiguration.getType(),
+ getBackendVersion(connectionConfiguration));
assertEquals(scientist + " index holds incorrect count", docsPerScientist, count);
}
}
@@ -485,7 +491,11 @@
for (int i = 0; i < 2; i++) {
String type = "TYPE_" + i;
long count =
- refreshIndexAndGetCurrentNumDocs(restClient, connectionConfiguration.getIndex(), type);
+ refreshIndexAndGetCurrentNumDocs(
+ restClient,
+ connectionConfiguration.getIndex(),
+ type,
+ getBackendVersion(connectionConfiguration));
assertEquals(type + " holds incorrect count", adjustedNumDocs / 2, count);
}
}
@@ -514,7 +524,9 @@
String index = scientist.toLowerCase();
for (int i = 0; i < 2; i++) {
String type = "TYPE_" + scientist.hashCode() % 2;
- long count = refreshIndexAndGetCurrentNumDocs(restClient, index, type);
+ long count =
+ refreshIndexAndGetCurrentNumDocs(
+ restClient, index, type, getBackendVersion(connectionConfiguration));
assertEquals("Incorrect count for " + index + "/" + type, numDocs / NUM_SCIENTISTS, count);
}
}
@@ -563,41 +575,6 @@
assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
}
- /** Tests partial updates with errors by adding some invalid info to test set. */
- void testWritePartialUpdateWithErrors() throws Exception {
- // put a mapping to simulate error of insertion
- ElasticsearchIOTestUtils.setIndexMapping(connectionConfiguration, restClient);
-
- if (!useAsITests) {
- ElasticsearchIOTestUtils.insertTestDocuments(connectionConfiguration, numDocs, restClient);
- }
-
- // try to partial update a document with an incompatible date format for the age to generate
- // an update error
- List<String> data = new ArrayList<>();
- data.add("{\"id\" : 1, \"age\" : \"2018-08-10:00:00\"}");
-
- try {
- pipeline
- .apply(Create.of(data))
- .apply(
- ElasticsearchIO.write()
- .withConnectionConfiguration(connectionConfiguration)
- .withIdFn(new ExtractValueFn("id"))
- .withUsePartialUpdate(true));
- pipeline.run();
- } catch (Exception e) {
- boolean matches =
- e.getLocalizedMessage()
- .matches(
- "(?is).*Error writing to Elasticsearch, some elements could not be inserted:"
- + ".*Document id .+: failed to parse .*Caused by: .*"
- + ".*For input string: \"2018-08-10:00:00\".*");
-
- assertTrue(matches);
- }
- }
-
/**
* Function for checking if any string in iterable contains expected substring. Fails if no match
* is found.
@@ -627,13 +604,15 @@
void testDefaultRetryPredicate(RestClient restClient) throws IOException {
HttpEntity entity1 = new NStringEntity(BAD_REQUEST, ContentType.APPLICATION_JSON);
- Response response1 =
- restClient.performRequest("POST", "/_bulk", Collections.emptyMap(), entity1);
+ Request request = new Request("POST", "/_bulk");
+ request.addParameters(Collections.emptyMap());
+ request.setEntity(entity1);
+ Response response1 = restClient.performRequest(request);
assertTrue(CUSTOM_RETRY_PREDICATE.test(response1.getEntity()));
HttpEntity entity2 = new NStringEntity(OK_REQUEST, ContentType.APPLICATION_JSON);
- Response response2 =
- restClient.performRequest("POST", "/_bulk", Collections.emptyMap(), entity2);
+ request.setEntity(entity2);
+ Response response2 = restClient.performRequest(request);
assertFalse(DEFAULT_RETRY_PREDICATE.test(response2.getEntity()));
}
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java
index ee9e47c..cc5a18e 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java
@@ -62,14 +62,16 @@
}
private static void closeIndex(RestClient restClient, String index) throws IOException {
- restClient.performRequest("POST", String.format("/%s/_close", index));
+ Request request = new Request("POST", String.format("/%s/_close", index));
+ restClient.performRequest(request);
}
private static void deleteIndex(RestClient restClient, String index) throws IOException {
try {
closeIndex(restClient, index);
- restClient.performRequest(
- "DELETE", String.format("/%s", index), Collections.singletonMap("refresh", "wait_for"));
+ Request request = new Request("DELETE", String.format("/%s", index));
+ request.addParameters(Collections.singletonMap("refresh", "wait_for"));
+ restClient.performRequest(request);
} catch (IOException e) {
// it is fine to ignore this expression as deleteIndex occurs in @before,
// so when the first tests is run, the index does not exist yet
@@ -91,8 +93,10 @@
"{\"source\" : { \"index\" : \"%s\" }, \"dest\" : { \"index\" : \"%s\" } }",
source, target),
ContentType.APPLICATION_JSON);
- restClient.performRequest(
- "POST", "/_reindex", Collections.singletonMap("refresh", "wait_for"), entity);
+ Request request = new Request("POST", "/_reindex");
+ request.addParameters(Collections.singletonMap("refresh", "wait_for"));
+ request.setEntity(entity);
+ restClient.performRequest(request);
}
/** Inserts the given number of test documents into Elasticsearch. */
@@ -118,9 +122,10 @@
"/%s/%s/_bulk", connectionConfiguration.getIndex(), connectionConfiguration.getType());
HttpEntity requestBody =
new NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON);
- Response response =
- restClient.performRequest(
- "POST", endPoint, Collections.singletonMap("refresh", "wait_for"), requestBody);
+ Request request = new Request("POST", endPoint);
+ request.addParameters(Collections.singletonMap("refresh", "wait_for"));
+ request.setEntity(requestBody);
+ Response response = restClient.performRequest(request);
ElasticsearchIO.checkForErrors(
response.getEntity(), ElasticsearchIO.getBackendVersion(connectionConfiguration), false);
}
@@ -136,7 +141,10 @@
static long refreshIndexAndGetCurrentNumDocs(
ConnectionConfiguration connectionConfiguration, RestClient restClient) throws IOException {
return refreshIndexAndGetCurrentNumDocs(
- restClient, connectionConfiguration.getIndex(), connectionConfiguration.getType());
+ restClient,
+ connectionConfiguration.getIndex(),
+ connectionConfiguration.getType(),
+ getBackendVersion(connectionConfiguration));
}
/**
@@ -148,17 +156,23 @@
* @return The number of docs in the index
* @throws IOException On error communicating with Elasticsearch
*/
- static long refreshIndexAndGetCurrentNumDocs(RestClient restClient, String index, String type)
- throws IOException {
+ static long refreshIndexAndGetCurrentNumDocs(
+ RestClient restClient, String index, String type, int backenVersion) throws IOException {
long result = 0;
try {
String endPoint = String.format("/%s/_refresh", index);
- restClient.performRequest("POST", endPoint);
+ Request request = new Request("POST", endPoint);
+ restClient.performRequest(request);
endPoint = String.format("/%s/%s/_search", index, type);
- Response response = restClient.performRequest("GET", endPoint);
+ request = new Request("GET", endPoint);
+ Response response = restClient.performRequest(request);
JsonNode searchResult = ElasticsearchIO.parseResponse(response.getEntity());
- result = searchResult.path("hits").path("total").asLong();
+ if (backenVersion >= 7) {
+ result = searchResult.path("hits").path("total").path("value").asLong();
+ } else {
+ result = searchResult.path("hits").path("total").asLong();
+ }
} catch (IOException e) {
// it is fine to ignore bellow exceptions because in testWriteWithBatchSize* sometimes,
// we call upgrade before any doc have been written
@@ -199,7 +213,7 @@
* @param connectionConfiguration Specifies the index and type
* @param restClient To use to execute the call
* @param scientistName The scientist to query for
- * @return The cound of documents found
+ * @return The count of documents found
* @throws IOException On error talking to Elasticsearch
*/
static int countByScientistName(
@@ -239,24 +253,16 @@
"/%s/%s/_search",
connectionConfiguration.getIndex(), connectionConfiguration.getType());
HttpEntity httpEntity = new NStringEntity(requestBody, ContentType.APPLICATION_JSON);
- Response response =
- restClient.performRequest("GET", endPoint, Collections.emptyMap(), httpEntity);
- JsonNode searchResult = parseResponse(response.getEntity());
- return searchResult.path("hits").path("total").asInt();
- }
- public static void setIndexMapping(
- ConnectionConfiguration connectionConfiguration, RestClient restClient) throws IOException {
- String endpoint = String.format("/%s", connectionConfiguration.getIndex());
- String requestString =
- String.format(
- "{\"mappings\":{\"%s\":{\"properties\":{\"age\":{\"type\":\"long\"},"
- + " \"scientist\":{\"type\":\"%s\"}, \"id\":{\"type\":\"long\"}}}}}",
- connectionConfiguration.getType(),
- getBackendVersion(connectionConfiguration) == 2 ? "string" : "text");
- HttpEntity requestBody = new NStringEntity(requestString, ContentType.APPLICATION_JSON);
- Request request = new Request("PUT", endpoint);
- request.setEntity(requestBody);
- restClient.performRequest(request);
+ Request request = new Request("GET", endPoint);
+ request.addParameters(Collections.emptyMap());
+ request.setEntity(httpEntity);
+ Response response = restClient.performRequest(request);
+ JsonNode searchResult = parseResponse(response.getEntity());
+ if (getBackendVersion(connectionConfiguration) >= 7) {
+ return searchResult.path("hits").path("total").path("value").asInt();
+ } else {
+ return searchResult.path("hits").path("total").asInt();
+ }
}
}
diff --git a/sdks/java/io/elasticsearch/build.gradle b/sdks/java/io/elasticsearch/build.gradle
index 6eca559..c8a3dc5 100644
--- a/sdks/java/io/elasticsearch/build.gradle
+++ b/sdks/java/io/elasticsearch/build.gradle
@@ -27,10 +27,6 @@
compile project(path: ":sdks:java:core", configuration: "shadow")
compile library.java.jackson_databind
compile library.java.jackson_annotations
- compile "org.elasticsearch.client:elasticsearch-rest-client:6.4.0"
- compile "org.apache.httpcomponents:httpasyncclient:4.1.4"
- compile "org.apache.httpcomponents:httpcore-nio:4.4.10"
- compile "org.apache.httpcomponents:httpcore:4.4.10"
- compile "org.apache.httpcomponents:httpclient:4.5.6"
+ compile "org.elasticsearch.client:elasticsearch-rest-client:7.5.0"
testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
}
diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
index 59f6057..ee56e03 100644
--- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
+++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
@@ -20,11 +20,12 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
import com.google.auto.value.AutoValue;
import java.io.File;
import java.io.FileInputStream;
@@ -146,9 +147,9 @@
* <p>When {withUsePartialUpdate()} is enabled, the input document must contain an id field and
* {@code withIdFn()} must be used to allow its extraction by the ElasticsearchIO.
*
- * <p>Optionally, {@code withSocketAndRetryTimeout()} can be used to override the default retry
- * timeout and socket timeout of 30000ms. {@code withConnectTimeout()} can be used to override the
- * default connect timeout of 1000ms.
+ * <p>Optionally, {@code withSocketTimeout()} can be used to override the default retry timeout and
+ * socket timeout of 30000ms. {@code withConnectTimeout()} can be used to override the default
+ * connect timeout of 1000ms.
*/
@Experimental(Experimental.Kind.SOURCE_SINK)
public class ElasticsearchIO {
@@ -203,7 +204,7 @@
} else {
if (backendVersion == 2) {
errorRootName = "create";
- } else if (backendVersion == 5 || backendVersion == 6) {
+ } else if (backendVersion >= 5) {
errorRootName = "index";
}
}
@@ -249,7 +250,7 @@
public abstract String getType();
@Nullable
- public abstract Integer getSocketAndRetryTimeout();
+ public abstract Integer getSocketTimeout();
@Nullable
public abstract Integer getConnectTimeout();
@@ -274,7 +275,7 @@
abstract Builder setType(String type);
- abstract Builder setSocketAndRetryTimeout(Integer maxRetryTimeout);
+ abstract Builder setSocketTimeout(Integer maxRetryTimeout);
abstract Builder setConnectTimeout(Integer connectTimeout);
@@ -374,13 +375,13 @@
* and the default socket timeout (30000ms) in the {@link RequestConfig} of the Elastic {@link
* RestClient}.
*
- * @param socketAndRetryTimeout the socket and retry timeout in millis.
+ * @param socketTimeout the socket and retry timeout in millis.
* @return a {@link ConnectionConfiguration} describes a connection configuration to
* Elasticsearch.
*/
- public ConnectionConfiguration withSocketAndRetryTimeout(Integer socketAndRetryTimeout) {
- checkArgument(socketAndRetryTimeout != null, "socketAndRetryTimeout can not be null");
- return builder().setSocketAndRetryTimeout(socketAndRetryTimeout).build();
+ public ConnectionConfiguration withSocketTimeout(Integer socketTimeout) {
+ checkArgument(socketTimeout != null, "socketTimeout can not be null");
+ return builder().setSocketTimeout(socketTimeout).build();
}
/**
@@ -402,7 +403,7 @@
builder.add(DisplayData.item("type", getType()));
builder.addIfNotNull(DisplayData.item("username", getUsername()));
builder.addIfNotNull(DisplayData.item("keystore.path", getKeystorePath()));
- builder.addIfNotNull(DisplayData.item("socketAndRetryTimeout", getSocketAndRetryTimeout()));
+ builder.addIfNotNull(DisplayData.item("socketTimeout", getSocketTimeout()));
builder.addIfNotNull(DisplayData.item("connectTimeout", getConnectTimeout()));
builder.addIfNotNull(DisplayData.item("trustSelfSignedCerts", isTrustSelfSignedCerts()));
}
@@ -452,16 +453,12 @@
if (getConnectTimeout() != null) {
requestConfigBuilder.setConnectTimeout(getConnectTimeout());
}
- if (getSocketAndRetryTimeout() != null) {
- requestConfigBuilder.setSocketTimeout(getSocketAndRetryTimeout());
+ if (getSocketTimeout() != null) {
+ requestConfigBuilder.setSocketTimeout(getSocketTimeout());
}
return requestConfigBuilder;
}
});
- if (getSocketAndRetryTimeout() != null) {
- restClientBuilder.setMaxRetryTimeoutMillis(getSocketAndRetryTimeout());
- }
-
return restClientBuilder.build();
}
}
@@ -670,7 +667,7 @@
new BoundedElasticsearchSource(spec, shardId, null, null, null, backendVersion));
}
checkArgument(!sources.isEmpty(), "No shard found");
- } else if (backendVersion == 5 || backendVersion == 6) {
+ } else if (backendVersion >= 5) {
long indexSize = getEstimatedSizeBytes(options);
float nbBundlesFloat = (float) indexSize / desiredBundleSizeBytes;
int nbBundles = (int) Math.ceil(nbBundlesFloat);
@@ -794,7 +791,9 @@
}
String endpoint = String.format("/%s/_stats", connectionConfiguration.getIndex());
try (RestClient restClient = connectionConfiguration.createClient()) {
- return parseResponse(restClient.performRequest("GET", endpoint, params).getEntity());
+ Request request = new Request("GET", endpoint);
+ request.addParameters(params);
+ return parseResponse(restClient.performRequest(request).getEntity());
}
}
}
@@ -820,9 +819,7 @@
if (query == null) {
query = "{\"query\": { \"match_all\": {} }}";
}
- if ((source.backendVersion == 5 || source.backendVersion == 6)
- && source.numSlices != null
- && source.numSlices > 1) {
+ if ((source.backendVersion >= 5) && source.numSlices != null && source.numSlices > 1) {
// if there is more than one slice, add the slice to the user query
String sliceQuery =
String.format("\"slice\": {\"id\": %s,\"max\": %s}", source.sliceId, source.numSlices);
@@ -842,7 +839,10 @@
}
}
HttpEntity queryEntity = new NStringEntity(query, ContentType.APPLICATION_JSON);
- Response response = restClient.performRequest("GET", endPoint, params, queryEntity);
+ Request request = new Request("GET", endPoint);
+ request.addParameters(params);
+ request.setEntity(queryEntity);
+ Response response = restClient.performRequest(request);
JsonNode searchResult = parseResponse(response.getEntity());
updateScrollId(searchResult);
return readNextBatchAndReturnFirstDocument(searchResult);
@@ -863,9 +863,10 @@
"{\"scroll\" : \"%s\",\"scroll_id\" : \"%s\"}",
source.spec.getScrollKeepalive(), scrollId);
HttpEntity scrollEntity = new NStringEntity(requestBody, ContentType.APPLICATION_JSON);
- Response response =
- restClient.performRequest(
- "GET", "/_search/scroll", Collections.emptyMap(), scrollEntity);
+ Request request = new Request("GET", "/_search/scroll");
+ request.addParameters(Collections.emptyMap());
+ request.setEntity(scrollEntity);
+ Response response = restClient.performRequest(request);
JsonNode searchResult = parseResponse(response.getEntity());
updateScrollId(searchResult);
return readNextBatchAndReturnFirstDocument(searchResult);
@@ -910,7 +911,10 @@
String requestBody = String.format("{\"scroll_id\" : [\"%s\"]}", scrollId);
HttpEntity entity = new NStringEntity(requestBody, ContentType.APPLICATION_JSON);
try {
- restClient.performRequest("DELETE", "/_search/scroll", Collections.emptyMap(), entity);
+ Request request = new Request("DELETE", "/_search/scroll");
+ request.addParameters(Collections.emptyMap());
+ request.setEntity(entity);
+ restClient.performRequest(request);
} finally {
if (restClient != null) {
restClient.close();
@@ -1241,19 +1245,10 @@
private long currentBatchSizeBytes;
// Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
- @JsonPropertyOrder({"_index", "_type", "_id"})
- @JsonInclude(JsonInclude.Include.NON_NULL)
private static class DocumentMetadata implements Serializable {
- @JsonProperty("_index")
final String index;
-
- @JsonProperty("_type")
final String type;
-
- @JsonProperty("_id")
final String id;
-
- @JsonProperty("_retry_on_conflict")
final Integer retryOnConflict;
DocumentMetadata(String index, String type, String id, Integer retryOnConflict) {
@@ -1293,6 +1288,35 @@
currentBatchSizeBytes = 0;
}
+ private class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+
+ private DocumentMetadataSerializer() {
+ super(DocumentMetadata.class);
+ }
+
+ @Override
+ public void serialize(
+ DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+ throws IOException {
+ gen.writeStartObject();
+ if (value.index != null) {
+ gen.writeStringField("_index", value.index);
+ }
+ if (value.type != null) {
+ gen.writeStringField("_type", value.type);
+ }
+ if (value.id != null) {
+ gen.writeStringField("_id", value.id);
+ }
+ if (value.retryOnConflict != null && (backendVersion <= 6)) {
+ gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+ }
+ if (value.retryOnConflict != null && backendVersion >= 7) {
+ gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+ }
+ gen.writeEndObject();
+ }
+ }
/**
* Extracts the components that comprise the document address from the document using the
* {@link FieldValueExtractFn} configured. This allows any or all of the index, type and
@@ -1317,8 +1341,10 @@
spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
spec.getUsePartialUpdate() ? DEFAULT_RETRY_ON_CONFLICT : null);
+ SimpleModule module = new SimpleModule();
+ module.addSerializer(DocumentMetadata.class, new DocumentMetadataSerializer());
+ OBJECT_MAPPER.registerModule(module);
return OBJECT_MAPPER.writeValueAsString(metadata);
-
} else {
return "{}"; // use configuration and auto-generated document IDs
}
@@ -1378,7 +1404,10 @@
spec.getConnectionConfiguration().getType());
HttpEntity requestBody =
new NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON);
- response = restClient.performRequest("POST", endPoint, Collections.emptyMap(), requestBody);
+ Request request = new Request("POST", endPoint);
+ request.addParameters(Collections.emptyMap());
+ request.setEntity(requestBody);
+ response = restClient.performRequest(request);
responseEntity = new BufferedHttpEntity(response.getEntity());
if (spec.getRetryConfiguration() != null
&& spec.getRetryConfiguration().getRetryPredicate().test(responseEntity)) {
@@ -1399,7 +1428,10 @@
// while retry policy exists
while (BackOffUtils.next(sleeper, backoff)) {
LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
- response = restClient.performRequest(method, endpoint, params, requestBody);
+ Request request = new Request(method, endpoint);
+ request.addParameters(params);
+ request.setEntity(requestBody);
+ response = restClient.performRequest(request);
responseEntity = new BufferedHttpEntity(response.getEntity());
// if response has no 429 errors
if (!spec.getRetryConfiguration().getRetryPredicate().test(responseEntity)) {
@@ -1420,15 +1452,19 @@
static int getBackendVersion(ConnectionConfiguration connectionConfiguration) {
try (RestClient restClient = connectionConfiguration.createClient()) {
- Response response = restClient.performRequest("GET", "");
+ Request request = new Request("GET", "");
+ Response response = restClient.performRequest(request);
JsonNode jsonNode = parseResponse(response.getEntity());
int backendVersion =
Integer.parseInt(jsonNode.path("version").path("number").asText().substring(0, 1));
checkArgument(
- (backendVersion == 2 || backendVersion == 5 || backendVersion == 6),
+ (backendVersion == 2
+ || backendVersion == 5
+ || backendVersion == 6
+ || backendVersion == 7),
"The Elasticsearch version to connect to is %s.x. "
+ "This version of the ElasticsearchIO is only compatible with "
- + "Elasticsearch v6.x, v5.x and v2.x",
+ + "Elasticsearch v7.x, v6.x, v5.x and v2.x",
backendVersion);
return backendVersion;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index 147a862..218de47 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -741,7 +741,7 @@
int strideIndex = 0;
// Upload in batches.
List<TableDataInsertAllRequest.Rows> rows = new ArrayList<>();
- int dataSize = 0;
+ long dataSize = 0L;
List<Future<List<TableDataInsertAllResponse.InsertErrors>>> futures = new ArrayList<>();
List<Integer> strideIndices = new ArrayList<>();
@@ -755,7 +755,12 @@
out.setJson(row.getUnknownKeys());
rows.add(out);
- dataSize += row.toString().length();
+ try {
+ dataSize += TableRowJsonCoder.of().getEncodedElementByteSize(row);
+ } catch (Exception ex) {
+ throw new RuntimeException("Failed to convert the row to JSON", ex);
+ }
+
if (dataSize >= maxRowBatchSize
|| rows.size() >= maxRowsPerBatch
|| i == rowsToPublish.size() - 1) {
@@ -796,7 +801,7 @@
retTotalDataSize += dataSize;
- dataSize = 0;
+ dataSize = 0L;
strideIndex = i + 1;
rows = new ArrayList<>();
}
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java
index a39a331..7e82eec 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java
@@ -58,7 +58,7 @@
}
@Override
- protected long getEncodedElementByteSize(TableRow value) throws Exception {
+ public long getEncodedElementByteSize(TableRow value) throws Exception {
String strValue = MAPPER.writeValueAsString(value);
return StringUtf8Coder.of().getEncodedElementByteSize(strValue);
}
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java
index cc7ad4f..402da86 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/TableContainer.java
@@ -21,6 +21,7 @@
import com.google.api.services.bigquery.model.TableRow;
import java.util.ArrayList;
import java.util.List;
+import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder;
/** Encapsulates a BigQuery Table, and it's contents. */
class TableContainer {
@@ -40,14 +41,14 @@
long addRow(TableRow row, String id) {
rows.add(row);
ids.add(id);
- long rowSize = row.toString().length();
- Long tableSize = table.getNumBytes();
- if (tableSize == null) {
- table.setNumBytes(rowSize);
- } else {
+ long tableSize = table.getNumBytes() == null ? 0L : table.getNumBytes();
+ try {
+ long rowSize = TableRowJsonCoder.of().getEncodedElementByteSize(row);
table.setNumBytes(tableSize + rowSize);
+ return rowSize;
+ } catch (Exception ex) {
+ throw new RuntimeException("Failed to convert the row to JSON", ex);
}
- return rowSize;
}
Table getTable() {
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java
index 3edd6e3..a2e5006 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java
@@ -627,7 +627,11 @@
.toSource(stepUuid, TableRowJsonCoder.of(), BigQueryIO.TableRowParser.INSTANCE);
PipelineOptions options = PipelineOptionsFactory.create();
- assertEquals(108, bqSource.getEstimatedSizeBytes(options));
+
+ // Each row should have 24 bytes (See StringUtf8Coder in detail):
+ // first 1 byte indicating length and following 23 bytes: {"name":"a","number":1}
+ long expectedSize = 24L * data.size();
+ assertEquals(expectedSize, bqSource.getEstimatedSizeBytes(options));
}
@Test
@@ -661,7 +665,12 @@
.toSource(stepUuid, TableRowJsonCoder.of(), BigQueryIO.TableRowParser.INSTANCE);
PipelineOptions options = PipelineOptionsFactory.create();
- assertEquals(118, bqSource.getEstimatedSizeBytes(options));
+
+ // Each row should have 24 bytes (See StringUtf8Coder in detail):
+ // first 1 byte indicating length and following 23 bytes: {"name":"a","number":1}
+ // 10 bytes comes from the estimated bytes of the Streamingbuffer
+ long expectedSize = 24L * data.size() + 10;
+ assertEquals(expectedSize, bqSource.getEstimatedSizeBytes(options));
}
@Test
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
index 12adc2e..4dddf0d 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
@@ -18,8 +18,8 @@
package org.apache.beam.sdk.io.gcp.bigquery;
import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -208,8 +208,8 @@
ref, rows, ids, InsertRetryPolicy.alwaysRetry(), null, null, false, false);
} finally {
verifyInsertAll(5);
- // Each of the 25 rows is 23 bytes: "{f=[{v=foo}, {v=1234}]}"
- assertEquals("Incorrect byte count", 25L * 23L, totalBytes);
+ // Each of the 25 rows has 1 byte for length and 30 bytes: '{"f":[{"v":"foo"},{"v":1234}]}'
+ assertEquals("Incorrect byte count", 25L * 31L, totalBytes);
}
}
}
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java
index fb443f5..da8f944 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java
@@ -17,8 +17,8 @@
*/
package org.apache.beam.sdk.io.gcp.testing;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.atLeast;
import static org.mockito.Mockito.spy;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java
index 89ebdb5..6bf6f92 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java
@@ -18,7 +18,7 @@
package org.apache.beam.sdk.io.gcp.testing;
import static org.hamcrest.MatcherAssert.assertThat;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java
index 9933aee..1a693ba 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java
@@ -220,28 +220,34 @@
private Set<Function<MetricsReader, NamedTestResult>> getWriteSuppliers(
String uuid, String timestamp) {
Set<Function<MetricsReader, NamedTestResult>> suppliers = new HashSet<>();
+ suppliers.add(getTimeMetric(uuid, timestamp, "write_time"));
suppliers.add(
- reader -> {
- long writeStart = reader.getStartTimeMetric("write_time");
- long writeEnd = reader.getEndTimeMetric("write_time");
- return NamedTestResult.create(
- uuid, timestamp, "write_time", (writeEnd - writeStart) / 1e3);
- });
+ reader ->
+ NamedTestResult.create(
+ uuid,
+ timestamp,
+ "data_size",
+ DatabaseTestHelper.getPostgresTableSize(dataSource, tableName)
+ .orElseThrow(() -> new IllegalStateException("Unable to fetch table size"))));
return suppliers;
}
private Set<Function<MetricsReader, NamedTestResult>> getReadSuppliers(
String uuid, String timestamp) {
Set<Function<MetricsReader, NamedTestResult>> suppliers = new HashSet<>();
- suppliers.add(
- reader -> {
- long readStart = reader.getStartTimeMetric("read_time");
- long readEnd = reader.getEndTimeMetric("read_time");
- return NamedTestResult.create(uuid, timestamp, "read_time", (readEnd - readStart) / 1e3);
- });
+ suppliers.add(getTimeMetric(uuid, timestamp, "read_time"));
return suppliers;
}
+ private Function<MetricsReader, NamedTestResult> getTimeMetric(
+ final String uuid, final String timestamp, final String metricName) {
+ return reader -> {
+ long startTime = reader.getStartTimeMetric(metricName);
+ long endTime = reader.getEndTimeMetric(metricName);
+ return NamedTestResult.create(uuid, timestamp, metricName, (endTime - startTime) / 1e3);
+ };
+ }
+
/**
* Uses the input {@link TestRow} values as seeds to produce new {@link KV}s for {@link
* HadoopFormatIO}.
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
index 046c061..72985cb 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
@@ -21,8 +21,8 @@
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
index 10de1df..5abe605 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
@@ -23,7 +23,7 @@
import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.mock;
diff --git a/sdks/python/apache_beam/coders/coder_impl.pxd b/sdks/python/apache_beam/coders/coder_impl.pxd
index e4b2832..6659078 100644
--- a/sdks/python/apache_beam/coders/coder_impl.pxd
+++ b/sdks/python/apache_beam/coders/coder_impl.pxd
@@ -211,5 +211,12 @@
cpdef encode_to_stream(self, value, OutputStream stream, bint nested)
+cdef class ParamWindowedValueCoderImpl(WindowedValueCoderImpl):
+ """A coder for windowed values with constant timestamp, windows and pane info."""
+ cdef readonly libc.stdint.int64_t _timestamp
+ cdef readonly object _windows
+ cdef readonly windowed_value.PaneInfo _pane_info
+
+
cdef class LengthPrefixCoderImpl(StreamCoderImpl):
cdef CoderImpl _value_coder
diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py
index a12f3c8..379156a 100644
--- a/sdks/python/apache_beam/coders/coder_impl.py
+++ b/sdks/python/apache_beam/coders/coder_impl.py
@@ -1179,6 +1179,55 @@
return estimated_size, observables
+class ParamWindowedValueCoderImpl(WindowedValueCoderImpl):
+ """For internal use only; no backwards-compatibility guarantees.
+
+ A coder for windowed values with constant timestamp, windows and
+ pane info. The coder drops timestamp, windows and pane info during
+ encoding, and uses the supplied parameterized timestamp, windows
+ and pane info values during decoding when reconstructing the windowed
+ value."""
+
+ def __init__(self, value_coder, window_coder, payload):
+ super(ParamWindowedValueCoderImpl, self).__init__(
+ value_coder, TimestampCoderImpl(), window_coder)
+ self._timestamp, self._windows, self._pane_info = self._from_proto(
+ payload, window_coder)
+
+ def _from_proto(self, payload, window_coder):
+ windowed_value_coder = WindowedValueCoderImpl(
+ BytesCoderImpl(), TimestampCoderImpl(), window_coder)
+ wv = windowed_value_coder.decode(payload)
+ return wv.timestamp_micros, wv.windows, wv.pane_info
+
+ def encode_to_stream(self, value, out, nested):
+ wv = value # type cast
+ self._value_coder.encode_to_stream(wv.value, out, nested)
+
+ def decode_from_stream(self, in_stream, nested):
+ value = self._value_coder.decode_from_stream(in_stream, nested)
+ return windowed_value.create(
+ value,
+ self._timestamp,
+ self._windows,
+ self._pane_info)
+
+ def get_estimated_size_and_observables(self, value, nested=False):
+ """Returns estimated size of value along with any nested observables."""
+ if isinstance(value, observable.ObservableMixin):
+ # Should never be here.
+ # TODO(robertwb): Remove when coders are set correctly.
+ return 0, [(value, self._value_coder)]
+ estimated_size = 0
+ observables = []
+ value_estimated_size, value_observables = (
+ self._value_coder.get_estimated_size_and_observables(
+ value.value, nested=nested))
+ estimated_size += value_estimated_size
+ observables += value_observables
+ return estimated_size, observables
+
+
class LengthPrefixCoderImpl(StreamCoderImpl):
"""For internal use only; no backwards-compatibility guarantees.
diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py
index 6ae142b..c6f9f1d 100644
--- a/sdks/python/apache_beam/coders/coders.py
+++ b/sdks/python/apache_beam/coders/coders.py
@@ -80,7 +80,7 @@
'FastPrimitivesCoder', 'FloatCoder', 'IterableCoder', 'PickleCoder',
'ProtoCoder', 'SingletonCoder', 'StrUtf8Coder', 'TimestampCoder',
'TupleCoder', 'TupleSequenceCoder', 'VarIntCoder',
- 'WindowedValueCoder'
+ 'WindowedValueCoder', 'ParamWindowedValueCoder'
]
T = TypeVar('T')
@@ -1234,6 +1234,50 @@
common_urns.coders.WINDOWED_VALUE.urn, WindowedValueCoder)
+class ParamWindowedValueCoder(WindowedValueCoder):
+ """A coder used for parameterized windowed values."""
+
+ def __init__(self, payload, components):
+ super(ParamWindowedValueCoder, self).__init__(components[0], components[1])
+ self.payload = payload
+
+ def _create_impl(self):
+ return coder_impl.ParamWindowedValueCoderImpl(
+ self.wrapped_value_coder.get_impl(),
+ self.window_coder.get_impl(),
+ self.payload)
+
+ def is_deterministic(self):
+ return self.wrapped_value_coder.is_deterministic()
+
+ def as_cloud_object(self, coders_context=None):
+ raise NotImplementedError(
+ "as_cloud_object not supported for ParamWindowedValueCoder")
+
+ def __repr__(self):
+ return 'ParamWindowedValueCoder[%s]' % self.wrapped_value_coder
+
+ def __eq__(self, other):
+ return (type(self) == type(other)
+ and self.wrapped_value_coder == other.wrapped_value_coder
+ and self.window_coder == other.window_coder
+ and self.payload == other.payload)
+
+ def __hash__(self):
+ return hash((self.wrapped_value_coder,
+ self.window_coder,
+ self.payload))
+
+ @Coder.register_urn(common_urns.coders.PARAM_WINDOWED_VALUE.urn, bytes)
+ def from_runner_api_parameter(payload, components, unused_context):
+ return ParamWindowedValueCoder(payload, components)
+
+ def to_runner_api_parameter(self, context):
+ return (common_urns.coders.PARAM_WINDOWED_VALUE.urn,
+ self.payload,
+ (self.wrapped_value_coder, self.window_coder))
+
+
class LengthPrefixCoder(FastCoder):
"""For internal use only; no backwards-compatibility guarantees.
diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py
index 122cbfd..cdab818 100644
--- a/sdks/python/apache_beam/coders/coders_test_common.py
+++ b/sdks/python/apache_beam/coders/coders_test_common.py
@@ -382,6 +382,62 @@
(windowed_value.WindowedValue(1.5, 0, ()),
windowed_value.WindowedValue("abc", 10, ('window',))))
+ def test_param_windowed_value_coder(self):
+ from apache_beam.transforms.window import IntervalWindow
+ from apache_beam.utils.windowed_value import PaneInfo
+ wv = windowed_value.create(
+ b'',
+ # Milliseconds to microseconds
+ 1000 * 1000,
+ (IntervalWindow(11, 21),),
+ PaneInfo(True, False, 1, 2, 3))
+ windowed_value_coder = coders.WindowedValueCoder(
+ coders.BytesCoder(), coders.IntervalWindowCoder())
+ payload = windowed_value_coder.encode(wv)
+ coder = coders.ParamWindowedValueCoder(
+ payload, [coders.VarIntCoder(), coders.IntervalWindowCoder()])
+
+ # Test binary representation
+ self.assertEqual(b'\x01',
+ coder.encode(window.GlobalWindows.windowed_value(1)))
+
+ # Test unnested
+ self.check_coder(
+ coders.ParamWindowedValueCoder(
+ payload, [coders.VarIntCoder(), coders.IntervalWindowCoder()]),
+ windowed_value.WindowedValue(
+ 3,
+ 1,
+ (window.IntervalWindow(11, 21),),
+ PaneInfo(True, False, 1, 2, 3)),
+ windowed_value.WindowedValue(
+ 1,
+ 1,
+ (window.IntervalWindow(11, 21),),
+ PaneInfo(True, False, 1, 2, 3)))
+
+ # Test nested
+ self.check_coder(
+ coders.TupleCoder((
+ coders.ParamWindowedValueCoder(
+ payload, [
+ coders.FloatCoder(),
+ coders.IntervalWindowCoder()]),
+ coders.ParamWindowedValueCoder(
+ payload, [
+ coders.StrUtf8Coder(),
+ coders.IntervalWindowCoder()]))),
+ (windowed_value.WindowedValue(
+ 1.5,
+ 1,
+ (window.IntervalWindow(11, 21),),
+ PaneInfo(True, False, 1, 2, 3)),
+ windowed_value.WindowedValue(
+ "abc",
+ 1,
+ (window.IntervalWindow(11, 21),),
+ PaneInfo(True, False, 1, 2, 3))))
+
def test_proto_coder(self):
# For instructions on how these test proto message were generated,
# see coders_test.py
diff --git a/sdks/python/apache_beam/coders/standard_coders_test.py b/sdks/python/apache_beam/coders/standard_coders_test.py
index e3ccf51..810481b 100644
--- a/sdks/python/apache_beam/coders/standard_coders_test.py
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -41,6 +41,8 @@
from apache_beam.typehints import schemas
from apache_beam.utils import windowed_value
from apache_beam.utils.timestamp import Timestamp
+from apache_beam.utils.windowed_value import PaneInfo
+from apache_beam.utils.windowed_value import PaneInfoTiming
STANDARD_CODERS_YAML = os.path.normpath(os.path.join(
os.path.dirname(__file__), '../portability/api/standard_coders.yaml'))
@@ -125,6 +127,16 @@
lambda x, value_parser, window_parser: windowed_value.create(
value_parser(x['value']), x['timestamp'] * 1000,
tuple([window_parser(w) for w in x['windows']])),
+ 'beam:coder:param_windowed_value:v1':
+ lambda x, value_parser, window_parser: windowed_value.create(
+ value_parser(x['value']), x['timestamp'] * 1000,
+ tuple([window_parser(w) for w in x['windows']]),
+ PaneInfo(
+ x['pane']['is_first'],
+ x['pane']['is_last'],
+ PaneInfoTiming.from_string(x['pane']['timing']),
+ x['pane']['index'],
+ x['pane']['on_time_index'])),
'beam:coder:timer:v1':
lambda x, payload_parser: dict(
payload=payload_parser(x['payload']),
diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py
index f0f53e2..38bcb88 100644
--- a/sdks/python/apache_beam/examples/snippets/snippets_test.py
+++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py
@@ -958,6 +958,7 @@
| 'pair_with_one' >> beam.Map(lambda x: (x, 1))
| WindowInto(FixedWindows(15),
trigger=trigger,
+ allowed_lateness=20,
accumulation_mode=AccumulationMode.DISCARDING)
| 'group' >> beam.GroupByKey()
| 'count' >> beam.Map(
@@ -1014,6 +1015,7 @@
FixedWindows(1 * 60),
trigger=AfterWatermark(
late=AfterProcessingTime(10 * 60)),
+ allowed_lateness=10,
accumulation_mode=AccumulationMode.DISCARDING)
# [END model_composite_triggers]
| 'group' >> beam.GroupByKey()
diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/max.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/max.py
new file mode 100644
index 0000000..7de4050
--- /dev/null
+++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/max.py
@@ -0,0 +1,60 @@
+# coding=utf-8
+#
+# 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.
+#
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+
+def max_globally(test=None):
+ # [START max_globally]
+ import apache_beam as beam
+
+ with beam.Pipeline() as pipeline:
+ max_element = (
+ pipeline
+ | 'Create numbers' >> beam.Create([3, 4, 1, 2])
+ | 'Get max value' >> beam.CombineGlobally(
+ lambda elements: max(elements or [None]))
+ | beam.Map(print)
+ )
+ # [END max_globally]
+ if test:
+ test(max_element)
+
+
+def max_per_key(test=None):
+ # [START max_per_key]
+ import apache_beam as beam
+
+ with beam.Pipeline() as pipeline:
+ elements_with_max_value_per_key = (
+ pipeline
+ | 'Create produce' >> beam.Create([
+ ('🥕', 3),
+ ('🥕', 2),
+ ('🍆', 1),
+ ('🍅', 4),
+ ('🍅', 5),
+ ('🍅', 3),
+ ])
+ | 'Get max value per key' >> beam.CombinePerKey(max)
+ | beam.Map(print)
+ )
+ # [END max_per_key]
+ if test:
+ test(elements_with_max_value_per_key)
diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/max_test.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/max_test.py
new file mode 100644
index 0000000..c229adec
--- /dev/null
+++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/max_test.py
@@ -0,0 +1,60 @@
+# coding=utf-8
+#
+# 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.
+#
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+import unittest
+
+import mock
+
+from apache_beam.examples.snippets.util import assert_matches_stdout
+from apache_beam.testing.test_pipeline import TestPipeline
+
+from . import max as beam_max
+
+
+def check_max_element(actual):
+ expected = '''[START max_element]
+4
+[END max_element]'''.splitlines()[1:-1]
+ assert_matches_stdout(actual, expected)
+
+
+def check_elements_with_max_value_per_key(actual):
+ expected = '''[START elements_with_max_value_per_key]
+('🥕', 3)
+('🍆', 1)
+('🍅', 5)
+[END elements_with_max_value_per_key]'''.splitlines()[1:-1]
+ assert_matches_stdout(actual, expected)
+
+
+@mock.patch('apache_beam.Pipeline', TestPipeline)
+@mock.patch(
+ 'apache_beam.examples.snippets.transforms.aggregation.max.print', str)
+class MaxTest(unittest.TestCase):
+ def test_max_globally(self):
+ beam_max.max_globally(check_max_element)
+
+ def test_max_per_key(self):
+ beam_max.max_per_key(check_elements_with_max_value_per_key)
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py
index 3cca9d3..0296840 100644
--- a/sdks/python/apache_beam/internal/pickler.py
+++ b/sdks/python/apache_beam/internal/pickler.py
@@ -75,6 +75,7 @@
def _is_nested_class(cls):
"""Returns true if argument is a class object that appears to be nested."""
return (isinstance(cls, type)
+ and cls.__module__ is not None
and cls.__module__ != 'builtins' # Python 3
and cls.__module__ != '__builtin__' # Python 2
and cls.__name__ not in sys.modules[cls.__module__].__dict__)
diff --git a/sdks/python/apache_beam/io/aws/__init__.py b/sdks/python/apache_beam/io/aws/__init__.py
new file mode 100644
index 0000000..6569e3f
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/__init__.py
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
diff --git a/sdks/python/apache_beam/io/aws/clients/__init__.py b/sdks/python/apache_beam/io/aws/clients/__init__.py
new file mode 100644
index 0000000..f4f43cb
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/clients/__init__.py
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+from __future__ import absolute_import
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/__init__.py b/sdks/python/apache_beam/io/aws/clients/s3/__init__.py
new file mode 100644
index 0000000..f4f43cb
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/clients/s3/__init__.py
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+from __future__ import absolute_import
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py b/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py
new file mode 100644
index 0000000..4aa8416
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py
@@ -0,0 +1,240 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
+
+from apache_beam.io.aws.clients.s3 import messages
+
+try:
+ # pylint: disable=wrong-import-order, wrong-import-position
+ # pylint: disable=ungrouped-imports
+ import boto3
+
+except ImportError:
+ boto3 = None
+
+
+class Client(object):
+ """
+ Wrapper for boto3 library
+ """
+
+ def __init__(self):
+ assert boto3 is not None, 'Missing boto3 requirement'
+ self.client = boto3.client('s3')
+
+ def get_object_metadata(self, request):
+ r"""Retrieves an object's metadata.
+
+ Args:
+ request: (GetRequest) input message
+
+ Returns:
+ (Object) The response message.
+ """
+ kwargs = {'Bucket': request.bucket, 'Key': request.object}
+
+ try:
+ boto_response = self.client.head_object(**kwargs)
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+
+ item = messages.Item(boto_response['ETag'],
+ request.object,
+ boto_response['LastModified'],
+ boto_response['ContentLength'],
+ boto_response['ContentType'])
+
+ return item
+
+ def get_range(self, request, start, end):
+ r"""Retrieves an object's contents.
+
+ Args:
+ request: (GetRequest) request
+ Returns:
+ (bytes) The response message.
+ """
+ try:
+ boto_response = self.client.get_object(Bucket=request.bucket,
+ Key=request.object,
+ Range='bytes={}-{}'.format(
+ start,
+ end - 1))
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+
+ return boto_response['Body'].read() # A bytes object
+
+ def list(self, request):
+ r"""Retrieves a list of objects matching the criteria.
+
+ Args:
+ request: (ListRequest) input message
+ Returns:
+ (ListResponse) The response message.
+ """
+ kwargs = {'Bucket': request.bucket,
+ 'Prefix': request.prefix}
+
+ if request.continuation_token is not None:
+ kwargs['ContinuationToken'] = request.continuation_token
+
+ try:
+ boto_response = self.client.list_objects_v2(**kwargs)
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+
+ if boto_response['KeyCount'] == 0:
+ message = 'Tried to list nonexistent S3 path: s3://%s/%s' % (
+ request.bucket, request.prefix)
+ raise messages.S3ClientError(message, 404)
+
+ items = [messages.Item(etag=content['ETag'],
+ key=content['Key'],
+ last_modified=content['LastModified'],
+ size=content['Size'])
+ for content in boto_response['Contents']]
+
+ try:
+ next_token = boto_response['NextContinuationToken']
+ except KeyError:
+ next_token = None
+
+ response = messages.ListResponse(items, next_token)
+ return response
+
+ def create_multipart_upload(self, request):
+ r"""Initates a multipart upload to S3 for a given object
+
+ Args:
+ request: (UploadRequest) input message
+ Returns:
+ (UploadResponse) The response message.
+ """
+ try:
+ boto_response = self.client.create_multipart_upload(
+ Bucket=request.bucket,
+ Key=request.object,
+ ContentType=request.mime_type
+ )
+ response = messages.UploadResponse(boto_response['UploadId'])
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+ return response
+
+ def upload_part(self, request):
+ r"""Uploads part of a file to S3 during a multipart upload
+
+ Args:
+ request: (UploadPartRequest) input message
+ Returns:
+ (UploadPartResponse) The response message.
+ """
+ try:
+ boto_response = self.client.upload_part(Body=request.bytes,
+ Bucket=request.bucket,
+ Key=request.object,
+ PartNumber=request.part_number,
+ UploadId=request.upload_id)
+ response = messages.UploadPartResponse(boto_response['ETag'],
+ request.part_number)
+ return response
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+
+ def complete_multipart_upload(self, request):
+ r"""Completes a multipart upload to S3
+
+ Args:
+ request: (UploadPartRequest) input message
+ Returns:
+ (Void) The response message.
+ """
+ parts = {'Parts': request.parts}
+ try:
+ self.client.complete_multipart_upload(Bucket=request.bucket,
+ Key=request.object,
+ UploadId=request.upload_id,
+ MultipartUpload=parts)
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+
+ def delete(self, request):
+ r"""Deletes given object from bucket
+ Args:
+ request: (DeleteRequest) input message
+ Returns:
+ (void) Void, otherwise will raise if an error occurs
+ """
+ try:
+ self.client.delete_object(Bucket=request.bucket,
+ Key=request.object)
+
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
+
+ def delete_batch(self, request):
+
+ aws_request = {
+ 'Bucket': request.bucket,
+ 'Delete': {
+ 'Objects': [{'Key': object} for object in request.objects]
+ }
+ }
+
+ try:
+ aws_response = self.client.delete_objects(**aws_request)
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = int(e.response['ResponseMetadata']['HTTPStatusCode'])
+ raise messages.S3ClientError(message, code)
+
+ deleted = [obj['Key'] for obj in aws_response.get('Deleted', [])]
+
+ failed = [obj['Key'] for obj in aws_response.get('Errors', [])]
+
+ errors = [messages.S3ClientError(obj['Message'], obj['Code'])
+ for obj in aws_response.get('Errors', [])]
+
+ return messages.DeleteBatchResponse(deleted, failed, errors)
+
+ def copy(self, request):
+ try:
+ copy_src = {
+ 'Bucket': request.src_bucket,
+ 'Key': request.src_key
+ }
+ self.client.copy(copy_src, request.dest_bucket, request.dest_key)
+ except Exception as e:
+ message = e.response['Error']['Message']
+ code = e.response['ResponseMetadata']['HTTPStatusCode']
+ raise messages.S3ClientError(message, code)
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/client_test.py b/sdks/python/apache_beam/io/aws/clients/s3/client_test.py
new file mode 100644
index 0000000..e78c484
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/clients/s3/client_test.py
@@ -0,0 +1,254 @@
+#
+# 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.
+#
+import logging
+import os
+import unittest
+
+from apache_beam.io.aws import s3io
+from apache_beam.io.aws.clients.s3 import fake_client
+from apache_beam.io.aws.clients.s3 import messages
+
+
+class ClientErrorTest(unittest.TestCase):
+
+ def setUp(self):
+
+ # These tests can be run locally against a mock S3 client, or as integration
+ # tests against the real S3 client.
+ self.USE_MOCK = True
+
+ # If you're running integration tests with S3, set this variable to be an
+ # s3 path that you have access to where test data can be written. If you're
+ # just running tests against the mock, this can be any s3 path. It should
+ # end with a '/'.
+ self.TEST_DATA_PATH = 's3://random-data-sets/beam_tests/'
+
+ self.test_bucket, self.test_path = s3io.parse_s3_path(self.TEST_DATA_PATH)
+
+ if self.USE_MOCK:
+ self.client = fake_client.FakeS3Client()
+ test_data_bucket, _ = s3io.parse_s3_path(self.TEST_DATA_PATH)
+ self.client.known_buckets.add(test_data_bucket)
+ self.aws = s3io.S3IO(self.client)
+ else:
+ self.aws = s3io.S3IO()
+
+
+ def test_get_object_metadata(self):
+
+ # Test nonexistent object
+ object = self.test_path + 'nonexistent_file_doesnt_exist'
+ request = messages.GetRequest(self.test_bucket, object)
+ self.assertRaises(messages.S3ClientError,
+ self.client.get_object_metadata,
+ request)
+
+ try:
+ self.client.get_object_metadata(request)
+ except Exception as e:
+ self.assertIsInstance(e, messages.S3ClientError)
+ self.assertEqual(e.code, 404)
+
+ def test_get_range_nonexistent(self):
+
+ # Test nonexistent object
+ object = self.test_path + 'nonexistent_file_doesnt_exist'
+ request = messages.GetRequest(self.test_bucket, object)
+ self.assertRaises(messages.S3ClientError,
+ self.client.get_range,
+ request, 0, 10)
+
+ try:
+ self.client.get_range(request, 0, 10)
+ except Exception as e:
+ self.assertIsInstance(e, messages.S3ClientError)
+ self.assertEqual(e.code, 404)
+
+ def test_get_range_bad_start_end(self):
+
+ file_name = self.TEST_DATA_PATH + 'get_range'
+ contents = os.urandom(1024)
+
+ with self.aws.open(file_name, 'w') as f:
+ f.write(contents)
+ bucket, object = s3io.parse_s3_path(file_name)
+
+ response = self.client.get_range(messages.GetRequest(bucket, object),
+ -10, 20)
+ self.assertEqual(response, contents)
+
+ response = self.client.get_range(messages.GetRequest(bucket, object),
+ 20, 10)
+ self.assertEqual(response, contents)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+
+ def test_upload_part_nonexistent_upload_id(self):
+
+ object = self.test_path + 'upload_part'
+ upload_id = 'not-an-id-12345'
+ part_number = 1
+ contents = os.urandom(1024)
+
+ request = messages.UploadPartRequest(self.test_bucket,
+ object,
+ upload_id,
+ part_number,
+ contents)
+
+ self.assertRaises(messages.S3ClientError,
+ self.client.upload_part,
+ request)
+
+ try:
+ self.client.upload_part(request)
+ except Exception as e:
+ self.assertIsInstance(e, messages.S3ClientError)
+ self.assertEqual(e.code, 404)
+
+
+ def test_copy_nonexistent(self):
+
+ src_key = self.test_path + 'not_a_real_file_does_not_exist'
+ dest_key = self.test_path + 'destination_file_location'
+
+ request = messages.CopyRequest(self.test_bucket,
+ src_key,
+ self.test_bucket,
+ dest_key)
+
+ with self.assertRaises(messages.S3ClientError) as e:
+ self.client.copy(request)
+
+ self.assertEqual(e.exception.code, 404)
+
+
+
+ def test_upload_part_bad_number(self):
+
+ object = self.test_path + 'upload_part'
+ contents = os.urandom(1024)
+
+ request = messages.UploadRequest(self.test_bucket, object, None)
+ response = self.client.create_multipart_upload(request)
+ upload_id = response.upload_id
+
+ part_number = 0.5
+ request = messages.UploadPartRequest(self.test_bucket,
+ object,
+ upload_id,
+ part_number,
+ contents)
+
+ self.assertRaises(messages.S3ClientError,
+ self.client.upload_part,
+ request)
+
+ try:
+ response = self.client.upload_part(request)
+ except Exception as e:
+ self.assertIsInstance(e, messages.S3ClientError)
+ self.assertEqual(e.code, 400)
+
+ def test_complete_multipart_upload_too_small(self):
+
+ object = self.test_path + 'upload_part'
+ request = messages.UploadRequest(self.test_bucket, object, None)
+ response = self.client.create_multipart_upload(request)
+ upload_id = response.upload_id
+
+ part_number = 1
+ contents_1 = os.urandom(1024)
+ request_1 = messages.UploadPartRequest(self.test_bucket,
+ object,
+ upload_id,
+ part_number,
+ contents_1)
+ response_1 = self.client.upload_part(request_1)
+
+
+ part_number = 2
+ contents_2 = os.urandom(1024)
+ request_2 = messages.UploadPartRequest(self.test_bucket,
+ object,
+ upload_id,
+ part_number,
+ contents_2)
+ response_2 = self.client.upload_part(request_2)
+
+ parts = [
+ {'PartNumber': 1, 'ETag': response_1.etag},
+ {'PartNumber': 2, 'ETag': response_2.etag}
+ ]
+ complete_request = messages.CompleteMultipartUploadRequest(self.test_bucket,
+ object,
+ upload_id,
+ parts)
+
+ try:
+ self.client.complete_multipart_upload(complete_request)
+ except Exception as e:
+ self.assertIsInstance(e, messages.S3ClientError)
+ self.assertEqual(e.code, 400)
+
+ def test_complete_multipart_upload_too_many(self):
+
+ object = self.test_path + 'upload_part'
+ request = messages.UploadRequest(self.test_bucket, object, None)
+ response = self.client.create_multipart_upload(request)
+ upload_id = response.upload_id
+
+ part_number = 1
+ contents_1 = os.urandom(5 * 1024)
+ request_1 = messages.UploadPartRequest(self.test_bucket,
+ object,
+ upload_id,
+ part_number,
+ contents_1)
+ response_1 = self.client.upload_part(request_1)
+
+
+ part_number = 2
+ contents_2 = os.urandom(1024)
+ request_2 = messages.UploadPartRequest(self.test_bucket,
+ object,
+ upload_id,
+ part_number,
+ contents_2)
+ response_2 = self.client.upload_part(request_2)
+
+ parts = [
+ {'PartNumber': 1, 'ETag': response_1.etag},
+ {'PartNumber': 2, 'ETag': response_2.etag},
+ {'PartNumber': 3, 'ETag': 'fake-etag'},
+ ]
+ complete_request = messages.CompleteMultipartUploadRequest(self.test_bucket,
+ object,
+ upload_id,
+ parts)
+
+ try:
+ self.client.complete_multipart_upload(complete_request)
+ except Exception as e:
+ self.assertIsInstance(e, messages.S3ClientError)
+ self.assertEqual(e.code, 400)
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ unittest.main()
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/fake_client.py b/sdks/python/apache_beam/io/aws/clients/s3/fake_client.py
new file mode 100644
index 0000000..0c7e6c3
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/clients/s3/fake_client.py
@@ -0,0 +1,236 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
+
+import datetime
+import time
+
+from apache_beam.io.aws.clients.s3 import messages
+
+
+class FakeFile(object):
+
+ def __init__(self, bucket, key, contents, etag=None):
+ self.bucket = bucket
+ self.key = key
+ self.contents = contents
+
+ self.last_modified = time.time()
+
+ if not etag:
+ self.etag = '"%s-1"' % ('x' * 32)
+ else:
+ self.etag = etag
+
+ def get_metadata(self):
+ last_modified_datetime = None
+ if self.last_modified:
+ last_modified_datetime = datetime.datetime.utcfromtimestamp(
+ self.last_modified)
+
+ return messages.Item(self.etag,
+ self.key,
+ last_modified_datetime,
+ len(self.contents),
+ mime_type=None)
+
+
+class FakeS3Client(object):
+ def __init__(self):
+ self.files = {}
+ self.list_continuation_tokens = {}
+ self.multipart_uploads = {}
+
+ # boto3 has different behavior when running some operations against a bucket
+ # that exists vs. against one that doesn't. To emulate that behavior, the
+ # mock client keeps a set of bucket names that it knows "exist".
+ self.known_buckets = set()
+
+ def add_file(self, f):
+ self.files[(f.bucket, f.key)] = f
+ if f.bucket not in self.known_buckets:
+ self.known_buckets.add(f.bucket)
+
+ def get_file(self, bucket, obj):
+ try:
+ return self.files[bucket, obj]
+ except:
+ raise messages.S3ClientError('Not Found', 404)
+
+ def delete_file(self, bucket, obj):
+ del self.files[(bucket, obj)]
+
+ def get_object_metadata(self, request):
+ r"""Retrieves an object's metadata.
+
+ Args:
+ request: (GetRequest) input message
+
+ Returns:
+ (Item) The response message.
+ """
+ # TODO: Do we want to mock out a lack of credentials?
+ file_ = self.get_file(request.bucket, request.object)
+ return file_.get_metadata()
+
+ def list(self, request):
+ bucket = request.bucket
+ prefix = request.prefix or ''
+ matching_files = []
+
+ for file_bucket, file_name in sorted(iter(self.files)):
+ if bucket == file_bucket and file_name.startswith(prefix):
+ file_object = self.get_file(file_bucket, file_name).get_metadata()
+ matching_files.append(file_object)
+
+ if not matching_files:
+ message = 'Tried to list nonexistent S3 path: s3://%s/%s' % (
+ bucket, prefix)
+ raise messages.S3ClientError(message, 404)
+
+ # Handle pagination.
+ items_per_page = 5
+ if not request.continuation_token:
+ range_start = 0
+ else:
+ if request.continuation_token not in self.list_continuation_tokens:
+ raise ValueError('Invalid page token.')
+ range_start = self.list_continuation_tokens[request.continuation_token]
+ del self.list_continuation_tokens[request.continuation_token]
+
+ result = messages.ListResponse(
+ items=matching_files[range_start:range_start + items_per_page])
+
+ if range_start + items_per_page < len(matching_files):
+ next_range_start = range_start + items_per_page
+ next_continuation_token = '_page_token_%s_%s_%d' % (bucket, prefix,
+ next_range_start)
+ self.list_continuation_tokens[next_continuation_token] = next_range_start
+ result.next_token = next_continuation_token
+
+ return result
+
+ def get_range(self, request, start, end):
+ r"""Retrieves an object.
+
+ Args:
+ request: (GetRequest) request
+ Returns:
+ (bytes) The response message.
+ """
+
+ file_ = self.get_file(request.bucket, request.object)
+
+ # Replicates S3's behavior, per the spec here:
+ # https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.35
+ if start < 0 or end <= start:
+ return file_.contents
+
+ return file_.contents[start:end]
+
+ def delete(self, request):
+ if request.bucket not in self.known_buckets:
+ raise messages.S3ClientError('The specified bucket does not exist', 404)
+
+ if (request.bucket, request.object) in self.files:
+ self.delete_file(request.bucket, request.object)
+ else:
+ # S3 doesn't raise an error if you try to delete a nonexistent file from
+ # an extant bucket
+ return
+
+ def delete_batch(self, request):
+
+ deleted, failed, errors = [], [], []
+ for object in request.objects:
+ try:
+ delete_request = messages.DeleteRequest(request.bucket, object)
+ self.delete(delete_request)
+ deleted.append(object)
+ except messages.S3ClientError as e:
+ failed.append(object)
+ errors.append(e)
+
+ return messages.DeleteBatchResponse(deleted, failed, errors)
+
+ def copy(self, request):
+
+ src_file = self.get_file(request.src_bucket, request.src_key)
+ dest_file = FakeFile(request.dest_bucket,
+ request.dest_key,
+ src_file.contents)
+ self.add_file(dest_file)
+
+ def create_multipart_upload(self, request):
+ # Create hash of bucket and key
+ # Store upload_id internally
+ upload_id = request.bucket + request.object
+ self.multipart_uploads[upload_id] = {}
+ return messages.UploadResponse(upload_id)
+
+ def upload_part(self, request):
+ # Save off bytes passed to internal data store
+ upload_id, part_number = request.upload_id, request.part_number
+
+ if part_number < 0 or not isinstance(part_number, int):
+ raise messages.S3ClientError('Param validation failed on part number',
+ 400)
+
+ if upload_id not in self.multipart_uploads:
+ raise messages.S3ClientError('The specified upload does not exist', 404)
+
+ self.multipart_uploads[upload_id][part_number] = request.bytes
+
+ etag = '"%s"' % ('x' * 32)
+ return messages.UploadPartResponse(etag, part_number)
+
+ def complete_multipart_upload(self, request):
+ MIN_PART_SIZE = 5 * 2**10 # 5 KiB
+
+ parts_received = self.multipart_uploads[request.upload_id]
+
+ # Check that we got all the parts that they intended to send
+ part_numbers_to_confirm = set(part['PartNumber'] for part in request.parts)
+
+ # Make sure all the expected parts are present
+ if part_numbers_to_confirm != set(parts_received.keys()):
+ raise messages.S3ClientError(
+ 'One or more of the specified parts could not be found', 400)
+
+ # Sort by part number
+ sorted_parts = sorted(parts_received.items(), key=lambda pair: pair[0])
+ sorted_bytes = [bytes_ for (_, bytes_) in sorted_parts]
+
+ # Make sure that the parts aren't too small (except the last part)
+ part_sizes = [len(bytes_) for bytes_ in sorted_bytes]
+ if any(size < MIN_PART_SIZE for size in part_sizes[:-1]):
+ e_message = """
+ All parts but the last must be larger than %d bytes
+ """ % MIN_PART_SIZE
+ raise messages.S3ClientError(e_message, 400)
+
+ # String together all bytes for the given upload
+ final_contents = b''.join(sorted_bytes)
+
+ # Create FakeFile object
+ num_parts = len(parts_received)
+ etag = '"%s-%d"' % ('x' * 32, num_parts)
+ file_ = FakeFile(request.bucket, request.object, final_contents, etag=etag)
+
+ # Store FakeFile in self.files
+ self.add_file(file_)
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/messages.py b/sdks/python/apache_beam/io/aws/clients/s3/messages.py
new file mode 100644
index 0000000..530f454
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/clients/s3/messages.py
@@ -0,0 +1,167 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
+
+
+class GetRequest():
+ """
+ S3 request object for `Get` command
+ """
+
+ def __init__(self, bucket, object):
+ self.bucket = bucket
+ self.object = object
+
+
+class UploadResponse():
+ """
+ S3 response object for `StartUpload` command
+ """
+
+ def __init__(self, upload_id):
+ self.upload_id = upload_id
+
+
+class UploadRequest():
+ """
+ S3 request object for `StartUpload` command
+ """
+
+ def __init__(self, bucket, object, mime_type):
+ self.bucket = bucket
+ self.object = object
+ self.mime_type = mime_type
+
+
+class UploadPartRequest():
+ """
+ S3 request object for `UploadPart` command
+ """
+
+ def __init__(self, bucket, object, upload_id, part_number, bytes):
+ self.bucket = bucket
+ self.object = object
+ self.upload_id = upload_id
+ self.part_number = part_number
+ self.bytes = bytes
+ # self.mime_type = mime_type
+
+
+class UploadPartResponse():
+ """
+ S3 response object for `UploadPart` command
+ """
+
+ def __init__(self, etag, part_number):
+ self.etag = etag
+ self.part_number = part_number
+
+
+class CompleteMultipartUploadRequest():
+ """
+ S3 request object for `UploadPart` command
+ """
+
+ def __init__(self, bucket, object, upload_id, parts):
+ # parts is a list of objects of the form
+ # {'ETag': response.etag, 'PartNumber': response.part_number}
+ self.bucket = bucket
+ self.object = object
+ self.upload_id = upload_id
+ self.parts = parts
+ # self.mime_type = mime_type
+
+
+class ListRequest():
+ """
+ S3 request object for `List` command
+ """
+
+ def __init__(self, bucket, prefix, continuation_token=None):
+ self.bucket = bucket
+ self.prefix = prefix
+ self.continuation_token = continuation_token
+
+
+class ListResponse():
+ """
+ S3 response object for `List` command
+ """
+
+ def __init__(self, items, next_token=None):
+ self.items = items
+ self.next_token = next_token
+
+
+class Item():
+ """
+ An item in S3
+ """
+
+ def __init__(self, etag, key, last_modified, size, mime_type=None):
+ self.etag = etag
+ self.key = key
+ self.last_modified = last_modified
+ self.size = size
+ self.mime_type = mime_type
+
+
+class DeleteRequest():
+ """
+ S3 request object for `Delete` command
+ """
+
+ def __init__(self, bucket, object):
+ self.bucket = bucket
+ self.object = object
+
+
+class DeleteBatchRequest():
+
+ def __init__(self, bucket, objects):
+ # `objects` is a list of strings corresponding to the keys to be deleted
+ # in the bucket
+ self.bucket = bucket
+ self.objects = objects
+
+
+class DeleteBatchResponse():
+
+ def __init__(self, deleted, failed, errors):
+ # `deleted` is a list of strings corresponding to the keys that were deleted
+ # `failed` is a list of strings corresponding to the keys that caused errors
+ # `errors` is a list of S3ClientErrors, aligned with the order of `failed`
+ self.deleted = deleted
+ self.failed = failed
+ self.errors = errors
+
+
+class CopyRequest():
+
+ def __init__(self, src_bucket, src_key, dest_bucket, dest_key):
+ self.src_bucket = src_bucket
+ self.src_key = src_key
+ self.dest_bucket = dest_bucket
+ self.dest_key = dest_key
+
+
+class S3ClientError(Exception):
+
+ def __init__(self, message = None, code = None):
+ self.message = message
+ self.code = code
diff --git a/sdks/python/apache_beam/io/aws/s3filesystem.py b/sdks/python/apache_beam/io/aws/s3filesystem.py
new file mode 100644
index 0000000..4fc7fb1
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/s3filesystem.py
@@ -0,0 +1,275 @@
+#
+# 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.
+#
+"""S3 file system implementation for accessing files on AWS S3."""
+
+from __future__ import absolute_import
+
+from future.utils import iteritems
+
+from apache_beam.io.aws import s3io
+from apache_beam.io.filesystem import BeamIOError
+from apache_beam.io.filesystem import CompressedFile
+from apache_beam.io.filesystem import CompressionTypes
+from apache_beam.io.filesystem import FileMetadata
+from apache_beam.io.filesystem import FileSystem
+
+__all__ = ['S3FileSystem']
+
+
+class S3FileSystem(FileSystem):
+ """An S3 `FileSystem` implementation for accessing files on AWS S3
+ """
+
+ CHUNK_SIZE = s3io.MAX_BATCH_OPERATION_SIZE
+ S3_PREFIX = 's3://'
+
+ @classmethod
+ def scheme(cls):
+ """URI scheme for the FileSystem
+ """
+ return 's3'
+
+ def join(self, basepath, *paths):
+ """Join two or more pathname components for the filesystem
+
+ Args:
+ basepath: string path of the first component of the path
+ paths: path components to be added
+
+ Returns: full path after combining all of the return nulled components
+ """
+ if not basepath.startswith(S3FileSystem.S3_PREFIX):
+ raise ValueError('Basepath %r must be S3 path.' % basepath)
+
+ path = basepath
+ for p in paths:
+ path = path.rstrip('/') + '/' + p.lstrip('/')
+ return path
+
+ def split(self, path):
+ """Splits the given path into two parts.
+
+ Splits the path into a pair (head, tail) such that tail contains the last
+ component of the path and head contains everything up to that.
+
+ Head will include the S3 prefix ('s3://').
+
+ Args:
+ path: path as a string
+ Returns:
+ a pair of path components as strings.
+ """
+ path = path.strip()
+ if not path.startswith(S3FileSystem.S3_PREFIX):
+ raise ValueError('Path %r must be S3 path.' % path)
+
+ prefix_len = len(S3FileSystem.S3_PREFIX)
+ last_sep = path[prefix_len:].rfind('/')
+ if last_sep >= 0:
+ last_sep += prefix_len
+
+ if last_sep > 0:
+ return (path[:last_sep], path[last_sep + 1:])
+ elif last_sep < 0:
+ return (path, '')
+ else:
+ raise ValueError('Invalid path: %s' % path)
+
+ def mkdirs(self, path):
+ """Recursively create directories for the provided path.
+
+ Args:
+ path: string path of the directory structure that should be created
+
+ Raises:
+ IOError if leaf directory already exists.
+ """
+ pass
+
+ def has_dirs(self):
+ """Whether this FileSystem supports directories."""
+ return False
+
+ def _list(self, dir_or_prefix):
+ """List files in a location.
+
+ Listing is non-recursive, for filesystems that support directories.
+
+ Args:
+ dir_or_prefix: (string) A directory or location prefix (for filesystems
+ that don't have directories).
+
+ Returns:
+ Generator of ``FileMetadata`` objects.
+
+ Raises:
+ ``BeamIOError`` if listing fails, but not if no files were found.
+ """
+ try:
+ for path, size in iteritems(s3io.S3IO().list_prefix(dir_or_prefix)):
+ yield FileMetadata(path, size)
+ except Exception as e: # pylint: disable=broad-except
+ raise BeamIOError("List operation failed", {dir_or_prefix: e})
+
+ def _path_open(self, path, mode, mime_type='application/octet-stream',
+ compression_type=CompressionTypes.AUTO):
+ """Helper functions to open a file in the provided mode.
+ """
+ compression_type = FileSystem._get_compression_type(path, compression_type)
+ mime_type = CompressionTypes.mime_type(compression_type, mime_type)
+ raw_file = s3io.S3IO().open(path, mode, mime_type=mime_type)
+ if compression_type == CompressionTypes.UNCOMPRESSED:
+ return raw_file
+ return CompressedFile(raw_file, compression_type=compression_type)
+
+ def create(self, path, mime_type='application/octet-stream',
+ compression_type=CompressionTypes.AUTO):
+ """Returns a write channel for the given file path.
+
+ Args:
+ path: string path of the file object to be written to the system
+ mime_type: MIME type to specify the type of content in the file object
+ compression_type: Type of compression to be used for this object
+
+ Returns: file handle with a close function for the user to use
+ """
+ return self._path_open(path, 'wb', mime_type, compression_type)
+
+ def open(self, path, mime_type='application/octet-stream',
+ compression_type=CompressionTypes.AUTO):
+ """Returns a read channel for the given file path.
+
+ Args:
+ path: string path of the file object to be written to the system
+ mime_type: MIME type to specify the type of content in the file object
+ compression_type: Type of compression to be used for this object
+
+ Returns: file handle with a close function for the user to use
+ """
+ return self._path_open(path, 'rb', mime_type, compression_type)
+
+ def copy(self, source_file_names, destination_file_names):
+ """Recursively copy the file tree from the source to the destination
+
+ Args:
+ source_file_names: list of source file objects that needs to be copied
+ destination_file_names: list of destination of the new object
+
+ Raises:
+ ``BeamIOError`` if any of the copy operations fail
+ """
+ if not len(source_file_names) == len(destination_file_names):
+ message = 'Unable to copy unequal number of sources and destinations'
+ raise BeamIOError(message)
+ src_dest_pairs = list(zip(source_file_names, destination_file_names))
+ return s3io.S3IO().copy_paths(src_dest_pairs)
+
+ def rename(self, source_file_names, destination_file_names):
+ """Rename the files at the source list to the destination list.
+ Source and destination lists should be of the same size.
+
+ Args:
+ source_file_names: List of file paths that need to be moved
+ destination_file_names: List of destination_file_names for the files
+
+ Raises:
+ ``BeamIOError`` if any of the rename operations fail
+ """
+ if not len(source_file_names) == len(destination_file_names):
+ message = 'Unable to rename unequal number of sources and destinations'
+ raise BeamIOError(message)
+ src_dest_pairs = list(zip(source_file_names, destination_file_names))
+ results = s3io.S3IO().rename_files(src_dest_pairs)
+ exceptions = {(src, dest): error for (src, dest, error) in results
+ if error is not None}
+ if exceptions:
+ raise BeamIOError("Rename operation failed", exceptions)
+
+ def exists(self, path):
+ """Check if the provided path exists on the FileSystem.
+
+ Args:
+ path: string path that needs to be checked.
+
+ Returns: boolean flag indicating if path exists
+ """
+ try:
+ return s3io.S3IO().exists(path)
+ except Exception as e: # pylint: disable=broad-except
+ raise BeamIOError("exists() operation failed", {path: e})
+
+ def size(self, path):
+ """Get size of path on the FileSystem.
+
+ Args:
+ path: string path in question.
+
+ Returns: int size of path according to the FileSystem.
+
+ Raises:
+ ``BeamIOError`` if path doesn't exist.
+ """
+ try:
+ return s3io.S3IO().size(path)
+ except Exception as e: # pylint: disable=broad-except
+ raise BeamIOError("size() operation failed", {path: e})
+
+ def last_updated(self, path):
+ """Get UNIX Epoch time in seconds on the FileSystem.
+
+ Args:
+ path: string path of file.
+
+ Returns: float UNIX Epoch time
+
+ Raises:
+ ``BeamIOError`` if path doesn't exist.
+ """
+ try:
+ return s3io.S3IO().last_updated(path)
+ except Exception as e: # pylint: disable=broad-except
+ raise BeamIOError("last_updated operation failed", {path: e})
+
+ def checksum(self, path):
+ """Fetch checksum metadata of a file on the
+ :class:`~apache_beam.io.filesystem.FileSystem`.
+
+ Args:
+ path: string path of a file.
+
+ Returns: string containing checksum
+
+ Raises:
+ ``BeamIOError`` if path isn't a file or doesn't exist.
+ """
+ try:
+ return s3io.S3IO().checksum(path)
+ except Exception as e: # pylint: disable=broad-except
+ raise BeamIOError("Checksum operation failed", {path: e})
+
+ def delete(self, paths):
+ """Deletes files or directories at the provided paths.
+ Directories will be deleted recursively.
+
+ Args:
+ paths: list of paths that give the file objects to be deleted
+ """
+ results = s3io.S3IO().delete_paths(paths)
+ exceptions = {path: error for (path, error) in results
+ if error is not None}
+ if exceptions:
+ raise BeamIOError("Delete operation failed", exceptions)
diff --git a/sdks/python/apache_beam/io/aws/s3filesystem_test.py b/sdks/python/apache_beam/io/aws/s3filesystem_test.py
new file mode 100644
index 0000000..4bddbf2
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/s3filesystem_test.py
@@ -0,0 +1,267 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+#
+
+"""Unit tests for the S3 File System"""
+
+from __future__ import absolute_import
+
+import logging
+import unittest
+
+import mock
+
+from apache_beam.io.aws.clients.s3 import messages
+from apache_beam.io.filesystem import BeamIOError
+from apache_beam.io.filesystem import FileMetadata
+from apache_beam.options.pipeline_options import PipelineOptions
+
+# Protect against environments where boto3 library is not available.
+# pylint: disable=wrong-import-order, wrong-import-position
+try:
+ from apache_beam.io.aws import s3filesystem
+except ImportError:
+ s3filesystem = None
+# pylint: enable=wrong-import-order, wrong-import-position
+
+
+@unittest.skipIf(s3filesystem is None, 'AWS dependencies are not installed')
+class S3FileSystemTest(unittest.TestCase):
+
+ def setUp(self):
+ pipeline_options = PipelineOptions()
+ self.fs = s3filesystem.S3FileSystem(pipeline_options=pipeline_options)
+
+ def test_scheme(self):
+ self.assertEqual(self.fs.scheme(), 's3')
+ self.assertEqual(s3filesystem.S3FileSystem.scheme(), 's3')
+
+ def test_join(self):
+ self.assertEqual('s3://bucket/path/to/file',
+ self.fs.join('s3://bucket/path', 'to', 'file'))
+ self.assertEqual('s3://bucket/path/to/file',
+ self.fs.join('s3://bucket/path', 'to/file'))
+ self.assertEqual('s3://bucket/path/to/file',
+ self.fs.join('s3://bucket/path', '/to/file'))
+ self.assertEqual('s3://bucket/path/to/file',
+ self.fs.join('s3://bucket/path/', 'to', 'file'))
+ self.assertEqual('s3://bucket/path/to/file',
+ self.fs.join('s3://bucket/path/', 'to/file'))
+ self.assertEqual('s3://bucket/path/to/file',
+ self.fs.join('s3://bucket/path/', '/to/file'))
+ with self.assertRaises(ValueError):
+ self.fs.join('/bucket/path/', '/to/file')
+
+ def test_split(self):
+ self.assertEqual(('s3://foo/bar', 'baz'),
+ self.fs.split('s3://foo/bar/baz'))
+ self.assertEqual(('s3://foo', ''),
+ self.fs.split('s3://foo/'))
+ self.assertEqual(('s3://foo', ''),
+ self.fs.split('s3://foo'))
+
+ with self.assertRaises(ValueError):
+ self.fs.split('/no/s3/prefix')
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_match_multiples(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ s3io_mock.list_prefix.return_value = {
+ 's3://bucket/file1': 1,
+ 's3://bucket/file2': 2
+ }
+ expected_results = set([
+ FileMetadata('s3://bucket/file1', 1),
+ FileMetadata('s3://bucket/file2', 2)
+ ])
+ match_result = self.fs.match(['s3://bucket/'])[0]
+
+ self.assertEqual(
+ set(match_result.metadata_list),
+ expected_results)
+ s3io_mock.list_prefix.assert_called_once_with('s3://bucket/')
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_match_multiples_limit(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ limit = 1
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ s3io_mock.list_prefix.return_value = {
+ 's3://bucket/file1': 1
+ }
+ expected_results = set([
+ FileMetadata('s3://bucket/file1', 1)
+ ])
+ match_result = self.fs.match(['s3://bucket/'], [limit])[0]
+ self.assertEqual(
+ set(match_result.metadata_list),
+ expected_results)
+ self.assertEqual(
+ len(match_result.metadata_list),
+ limit)
+ s3io_mock.list_prefix.assert_called_once_with('s3://bucket/')
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_match_multiples_error(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ exception = IOError('Failed')
+ s3io_mock.list_prefix.side_effect = exception
+
+ with self.assertRaises(BeamIOError) as error:
+ self.fs.match(['s3://bucket/'])
+
+ self.assertTrue('Match operation failed' in str(error.exception))
+ s3io_mock.list_prefix.assert_called_once_with('s3://bucket/')
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_match_multiple_patterns(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ s3io_mock.list_prefix.side_effect = [
+ {'s3://bucket/file1': 1},
+ {'s3://bucket/file2': 2},
+ ]
+ expected_results = [
+ [FileMetadata('s3://bucket/file1', 1)],
+ [FileMetadata('s3://bucket/file2', 2)]
+ ]
+ result = self.fs.match(['s3://bucket/file1*', 's3://bucket/file2*'])
+ self.assertEqual(
+ [mr.metadata_list for mr in result],
+ expected_results)
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_create(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ # Issue file copy
+ _ = self.fs.create('s3://bucket/from1', 'application/octet-stream')
+
+ s3io_mock.open.assert_called_once_with(
+ 's3://bucket/from1', 'wb', mime_type='application/octet-stream')
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_open(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ # Issue file copy
+ _ = self.fs.open('s3://bucket/from1', 'application/octet-stream')
+
+ s3io_mock.open.assert_called_once_with(
+ 's3://bucket/from1', 'rb', mime_type='application/octet-stream')
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_copy_file(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+
+ sources = ['s3://bucket/from1', 's3://bucket/from2']
+ destinations = ['s3://bucket/to1', 's3://bucket/to2']
+
+ # Issue file copy
+ self.fs.copy(sources, destinations)
+
+ src_dest_pairs = list(zip(sources, destinations))
+ s3io_mock.copy_paths.assert_called_once_with(src_dest_pairs)
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_copy_file_error(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+
+ sources = ['s3://bucket/from1', 's3://bucket/from2', 's3://bucket/from3']
+ destinations = ['s3://bucket/to1', 's3://bucket/to2']
+
+ # Issue file copy
+ with self.assertRaises(BeamIOError):
+ self.fs.copy(sources, destinations)
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_delete(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+ s3io_mock.size.return_value = 0
+ files = [
+ 's3://bucket/from1',
+ 's3://bucket/from2',
+ 's3://bucket/from3',
+ ]
+
+ # Issue batch delete.
+ self.fs.delete(files)
+ s3io_mock.delete_paths.assert_called_once_with(files)
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_delete_error(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+
+ problematic_directory = 's3://nonexistent-bucket/tree/'
+ exception = messages.S3ClientError('Not found', 404)
+
+ s3io_mock.delete_paths.return_value = [
+ (problematic_directory, exception),
+ ('s3://bucket/object1', None),
+ ('s3://bucket/object2', None)
+ ]
+ s3io_mock.size.return_value = 0
+ files = [
+ problematic_directory,
+ 's3://bucket/object1',
+ 's3://bucket/object2',
+ ]
+ expected_results = {problematic_directory: exception}
+
+ # Issue batch delete.
+ with self.assertRaises(BeamIOError) as error:
+ self.fs.delete(files)
+ self.assertTrue('Delete operation failed' in str(error.exception))
+ self.assertEqual(error.exception.exception_details, expected_results)
+ s3io_mock.delete_paths.assert_called()
+
+ @mock.patch('apache_beam.io.aws.s3filesystem.s3io')
+ def test_rename(self, unused_mock_arg):
+ # Prepare mocks.
+ s3io_mock = mock.MagicMock()
+ s3filesystem.s3io.S3IO = lambda: s3io_mock
+
+ sources = ['s3://bucket/from1', 's3://bucket/from2']
+ destinations = ['s3://bucket/to1', 's3://bucket/to2']
+
+ # Issue file copy
+ self.fs.rename(sources, destinations)
+
+ src_dest_pairs = list(zip(sources, destinations))
+ s3io_mock.rename_files.assert_called_once_with(src_dest_pairs)
+
+
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ unittest.main()
diff --git a/sdks/python/apache_beam/io/aws/s3io.py b/sdks/python/apache_beam/io/aws/s3io.py
new file mode 100644
index 0000000..f5f8e0a
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/s3io.py
@@ -0,0 +1,606 @@
+#
+# 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.
+#
+"""AWS S3 client
+"""
+
+from __future__ import absolute_import
+
+import errno
+import io
+import logging
+import re
+import time
+import traceback
+from builtins import object
+
+from apache_beam.io.aws.clients.s3 import messages
+from apache_beam.io.filesystemio import Downloader
+from apache_beam.io.filesystemio import DownloaderStream
+from apache_beam.io.filesystemio import Uploader
+from apache_beam.io.filesystemio import UploaderStream
+from apache_beam.utils import retry
+
+try:
+ # pylint: disable=wrong-import-order, wrong-import-position
+ # pylint: disable=ungrouped-imports
+ from apache_beam.io.aws.clients.s3 import boto3_client
+ BOTO3_INSTALLED = True
+except ImportError:
+ BOTO3_INSTALLED = False
+
+MAX_BATCH_OPERATION_SIZE = 100
+
+
+def parse_s3_path(s3_path, object_optional=False):
+ """Return the bucket and object names of the given s3:// path."""
+ match = re.match('^s3://([^/]+)/(.*)$', s3_path)
+ if match is None or (match.group(2) == '' and not object_optional):
+ raise ValueError('S3 path must be in the form s3://<bucket>/<object>.')
+ return match.group(1), match.group(2)
+
+
+class S3IO(object):
+ """S3 I/O client."""
+
+ def __init__(self, client=None):
+ if client is not None:
+ self.client = client
+ elif BOTO3_INSTALLED:
+ self.client = boto3_client.Client()
+ else:
+ message = 'AWS dependencies are not installed, and no alternative ' \
+ 'client was provided to S3IO.'
+ raise RuntimeError(message)
+
+ def open(self,
+ filename,
+ mode='r',
+ read_buffer_size=16*1024*1024,
+ mime_type='application/octet-stream'):
+ """Open an S3 file path for reading or writing.
+
+ Args:
+ filename (str): S3 file path in the form ``s3://<bucket>/<object>``.
+ mode (str): ``'r'`` for reading or ``'w'`` for writing.
+ read_buffer_size (int): Buffer size to use during read operations.
+ mime_type (str): Mime type to set for write operations.
+
+ Returns:
+ S3 file object.
+
+ Raises:
+ ~exceptions.ValueError: Invalid open file mode.
+ """
+ if mode == 'r' or mode == 'rb':
+ downloader = S3Downloader(self.client, filename,
+ buffer_size=read_buffer_size)
+ return io.BufferedReader(DownloaderStream(downloader, mode=mode),
+ buffer_size=read_buffer_size)
+ elif mode == 'w' or mode == 'wb':
+ uploader = S3Uploader(self.client, filename, mime_type)
+ return io.BufferedWriter(UploaderStream(uploader, mode=mode),
+ buffer_size=128 * 1024)
+ else:
+ raise ValueError('Invalid file open mode: %s.' % mode)
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def list_prefix(self, path):
+ """Lists files matching the prefix.
+
+ Args:
+ path: S3 file path pattern in the form s3://<bucket>/[name].
+
+ Returns:
+ Dictionary of file name -> size.
+ """
+ bucket, prefix = parse_s3_path(path, object_optional=True)
+ request = messages.ListRequest(bucket=bucket, prefix=prefix)
+
+ file_sizes = {}
+ counter = 0
+ start_time = time.time()
+
+ logging.info("Starting the size estimation of the input")
+
+ while True:
+ response = self.client.list(request)
+ for item in response.items:
+ file_name = 's3://%s/%s' % (bucket, item.key)
+ file_sizes[file_name] = item.size
+ counter += 1
+ if counter % 10000 == 0:
+ logging.info("Finished computing size of: %s files", len(file_sizes))
+ if response.next_token:
+ request.continuation_token = response.next_token
+ else:
+ break
+
+ logging.info("Finished listing %s files in %s seconds.",
+ counter, time.time() - start_time)
+
+ return file_sizes
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def checksum(self, path):
+ """Looks up the checksum of an S3 object.
+
+ Args:
+ path: S3 file path pattern in the form s3://<bucket>/<name>.
+ """
+ bucket, object_path = parse_s3_path(path)
+ request = messages.GetRequest(bucket, object_path)
+ item = self.client.get_object_metadata(request)
+ return item.etag
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def copy(self, src, dest):
+ """Copies a single S3 file object from src to dest.
+
+ Args:
+ src: S3 file path pattern in the form s3://<bucket>/<name>.
+ dest: S3 file path pattern in the form s3://<bucket>/<name>.
+
+ Raises:
+ TimeoutError on timeout.
+ """
+ src_bucket, src_key = parse_s3_path(src)
+ dest_bucket, dest_key = parse_s3_path(dest)
+ request = messages.CopyRequest(src_bucket, src_key, dest_bucket, dest_key)
+ self.client.copy(request)
+
+ # We intentionally do not decorate this method with a retry, since the
+ # underlying copy and delete operations are already idempotent operations
+ # protected by retry decorators.
+ def copy_paths(self, src_dest_pairs):
+ """Copies the given S3 objects from src to dest. This can handle directory
+ or file paths.
+
+ Args:
+ src_dest_pairs: list of (src, dest) tuples of s3://<bucket>/<name> file
+ paths to copy from src to dest
+ Returns: List of tuples of (src, dest, exception) in the same order as the
+ src_dest_pairs argument, where exception is None if the operation
+ succeeded or the relevant exception if the operation failed.
+ """
+ if not src_dest_pairs: return []
+
+ results = []
+
+ for src_path, dest_path in src_dest_pairs:
+
+ # Copy a directory with self.copy_tree
+ if src_path.endswith('/') and dest_path.endswith('/'):
+ try:
+ results += self.copy_tree(src_path, dest_path)
+ except messages.S3ClientError as err:
+ results.append((src_path, dest_path, err))
+
+ # Copy individual files with self.copy
+ elif not src_path.endswith('/') and not dest_path.endswith('/'):
+ src_bucket, src_key = parse_s3_path(src_path)
+ dest_bucket, dest_key = parse_s3_path(dest_path)
+ request = messages.CopyRequest(src_bucket,
+ src_key,
+ dest_bucket,
+ dest_key)
+
+ try:
+ self.client.copy(request)
+ results.append((src_path, dest_path, None))
+ except messages.S3ClientError as e:
+ results.append((src_path, dest_path, e))
+
+ # Mismatched paths (one directory, one non-directory) get an error result
+ else:
+ err = messages.S3ClientError(
+ "Can't copy mismatched paths (one directory, one non-directory):" +
+ ' %s, %s' % (src_path, dest_path),
+ 400)
+ results.append((src_path, dest_path, err))
+
+ return results
+
+ # We intentionally do not decorate this method with a retry, since the
+ # underlying copy and delete operations are already idempotent operations
+ # protected by retry decorators.
+ def copy_tree(self, src, dest):
+ """Renames the given S3 directory and it's contents recursively
+ from src to dest.
+
+ Args:
+ src: S3 file path pattern in the form s3://<bucket>/<name>/.
+ dest: S3 file path pattern in the form s3://<bucket>/<name>/.
+
+ Returns:
+ List of tuples of (src, dest, exception) where exception is None if the
+ operation succeeded or the relevant exception if the operation failed.
+ """
+ assert src.endswith('/')
+ assert dest.endswith('/')
+
+ results = []
+ for entry in self.list_prefix(src):
+ rel_path = entry[len(src):]
+ try:
+ self.copy(entry, dest + rel_path)
+ results.append((entry, dest + rel_path, None))
+ except messages.S3ClientError as e:
+ results.append((entry, dest + rel_path, e))
+
+ return results
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def delete(self, path):
+ """Deletes a single S3 file object from src to dest.
+
+ Args:
+ src: S3 file path pattern in the form s3://<bucket>/<name>/.
+ dest: S3 file path pattern in the form s3://<bucket>/<name>/.
+
+ Returns:
+ List of tuples of (src, dest, exception) in the same order as the
+ src_dest_pairs argument, where exception is None if the operation
+ succeeded or the relevant exception if the operation failed.
+ """
+ bucket, object_path = parse_s3_path(path)
+ request = messages.DeleteRequest(bucket, object_path)
+
+ try:
+ self.client.delete(request)
+ except messages.S3ClientError as e:
+ if e.code == 404:
+ return # Same behavior as GCS - don't surface a 404 error
+ else:
+ logging.error('HTTP error while deleting file %s: %s', path,
+ 3)
+ raise e
+
+ # We intentionally do not decorate this method with a retry, since the
+ # underlying copy and delete operations are already idempotent operations
+ # protected by retry decorators.
+ def delete_paths(self, paths):
+ """Deletes the given S3 objects from src to dest. This can handle directory
+ or file paths.
+
+ Args:
+ src: S3 file path pattern in the form s3://<bucket>/<name>/.
+ dest: S3 file path pattern in the form s3://<bucket>/<name>/.
+
+ Returns:
+ List of tuples of (src, dest, exception) in the same order as the
+ src_dest_pairs argument, where exception is None if the operation
+ succeeded or the relevant exception if the operation failed.
+ """
+ directories, not_directories = [], []
+ for path in paths:
+ if path.endswith('/'): directories.append(path)
+ else: not_directories.append(path)
+
+ results = {}
+
+ for directory in directories:
+ dir_result = dict(self.delete_tree(directory))
+ results.update(dir_result)
+
+ not_directory_results = dict(self.delete_files(not_directories))
+ results.update(not_directory_results)
+
+ return results
+
+ # We intentionally do not decorate this method with a retry, since the
+ # underlying copy and delete operations are already idempotent operations
+ # protected by retry decorators.
+ def delete_files(self, paths, max_batch_size=1000):
+ """Deletes the given S3 file object from src to dest.
+
+ Args:
+ paths: List of S3 file paths in the form s3://<bucket>/<name>
+ max_batch_size: Largest number of keys to send to the client to be deleted
+ simultaneously
+
+ Returns: List of tuples of (path, exception) in the same order as the paths
+ argument, where exception is None if the operation succeeded or
+ the relevant exception if the operation failed.
+ """
+ if not paths: return []
+
+ # Sort paths into bucket: [keys]
+ buckets, keys = zip(*[parse_s3_path(path) for path in paths])
+ grouped_keys = {bucket: [] for bucket in buckets}
+ for bucket, key in zip(buckets, keys): grouped_keys[bucket].append(key)
+
+ # For each bucket, delete minibatches of keys
+ results = {}
+ for bucket, keys in grouped_keys.items():
+ for i in range(0, len(keys), max_batch_size):
+ minibatch_keys = keys[i : i + max_batch_size]
+ results.update(self._delete_minibatch(bucket, minibatch_keys))
+
+ # Organize final results
+ final_results = [(path, results[parse_s3_path(path)]) for path in paths]
+
+ return final_results
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def _delete_minibatch(self, bucket, keys):
+ """A helper method. Boto3 allows batch deletions
+ for files within the same bucket.
+
+ Args:
+ bucket: String bucket name
+ keys: List of keys to be deleted in the bucket
+
+ Returns: dict of the form {(bucket, key): error}, where error is None if the
+ operation succeeded
+ """
+ request = messages.DeleteBatchRequest(bucket, keys)
+ results = {}
+ try:
+ response = self.client.delete_batch(request)
+
+ for key in response.deleted:
+ results[(bucket, key)] = None
+
+ for key, error in zip(response.failed, response.errors):
+ results[(bucket, key)] = error
+
+ except messages.S3ClientError as e:
+ for key in keys:
+ results[(bucket, key)] = e
+
+ return results
+
+ # We intentionally do not decorate this method with a retry, since the
+ # underlying copy and delete operations are already idempotent operations
+ # protected by retry decorators.
+ def delete_tree(self, root):
+ """Deletes all objects under the given S3 directory.
+
+ Args:
+ path: S3 root path in the form s3://<bucket>/<name>/ (ending with a "/")
+
+ Returns: List of tuples of (path, exception), where each path is an object
+ under the given root. exception is None if the operation succeeded
+ or the relevant exception if the operation failed.
+ """
+ assert root.endswith('/')
+
+ paths = self.list_prefix(root)
+ return self.delete_files(paths)
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def size(self, path):
+ """Returns the size of a single S3 object.
+
+ This method does not perform glob expansion. Hence the given path must be
+ for a single S3 object.
+
+ Returns: size of the S3 object in bytes.
+ """
+ bucket, object_path = parse_s3_path(path)
+ request = messages.GetRequest(bucket, object_path)
+ item = self.client.get_object_metadata(request)
+ return item.size
+
+ # We intentionally do not decorate this method with a retry, since the
+ # underlying copy and delete operations are already idempotent operations
+ # protected by retry decorators.
+ def rename(self, src, dest):
+ """Renames the given S3 object from src to dest.
+
+ Args:
+ src: S3 file path pattern in the form s3://<bucket>/<name>.
+ dest: S3 file path pattern in the form s3://<bucket>/<name>.
+ """
+ self.copy(src, dest)
+ self.delete(src)
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def last_updated(self, path):
+ """Returns the last updated epoch time of a single S3 object.
+
+ This method does not perform glob expansion. Hence the given path must be
+ for a single S3 object.
+
+ Returns: last updated time of the S3 object in second.
+ """
+ bucket, object = parse_s3_path(path)
+ request = messages.GetRequest(bucket, object)
+ datetime = self.client.get_object_metadata(request).last_modified
+ return (time.mktime(datetime.timetuple()) - time.timezone
+ + datetime.microsecond / 1000000.0)
+
+ def exists(self, path):
+ """Returns whether the given S3 object exists.
+
+ Args:
+ path: S3 file path pattern in the form s3://<bucket>/<name>.
+ """
+ bucket, object = parse_s3_path(path)
+ request = messages.GetRequest(bucket, object)
+ try:
+ self.client.get_object_metadata(request)
+ return True
+ except messages.S3ClientError as e:
+ if e.code == 404:
+ # HTTP 404 indicates that the file did not exist
+ return False
+ else:
+ # We re-raise all other exceptions
+ raise
+
+ def rename_files(self, src_dest_pairs):
+ """Renames the given S3 objects from src to dest.
+
+ Args:
+ src_dest_pairs: list of (src, dest) tuples of s3://<bucket>/<name> file
+ paths to rename from src to dest
+ Returns: List of tuples of (src, dest, exception) in the same order as the
+ src_dest_pairs argument, where exception is None if the operation
+ succeeded or the relevant exception if the operation failed.
+ """
+ if not src_dest_pairs: return []
+
+ # TODO: Throw value error if path has directory
+ for src, dest in src_dest_pairs:
+ if src.endswith('/') or dest.endswith('/'):
+ raise ValueError('Cannot rename a directory')
+
+ copy_results = self.copy_paths(src_dest_pairs)
+ paths_to_delete = [src for (src, _, err) in copy_results if err is None]
+ delete_results = self.delete_files(paths_to_delete)
+
+ delete_results_dict = {src: err for (src, err) in delete_results}
+ rename_results = []
+ for src, dest, err in copy_results:
+ if err is not None: rename_results.append((src, dest, err))
+ elif delete_results_dict[src] is not None:
+ rename_results.append(src, dest, delete_results_dict[src])
+ else: rename_results.append((src, dest, None))
+
+ return rename_results
+
+
+class S3Downloader(Downloader):
+ def __init__(self, client, path, buffer_size):
+ self._client = client
+ self._path = path
+ self._bucket, self._name = parse_s3_path(path)
+ self._buffer_size = buffer_size
+
+ # Get object state.
+ self._get_request = (messages.GetRequest(
+ bucket=self._bucket,
+ object=self._name))
+
+ try:
+ metadata = self._get_object_metadata(self._get_request)
+
+ except messages.S3ClientError as e:
+ if e.code == 404:
+ raise IOError(errno.ENOENT, 'Not found: %s' % self._path)
+ else:
+ logging.error('HTTP error while requesting file %s: %s', self._path,
+ 3)
+ raise
+
+ self._size = metadata.size
+
+ @retry.with_exponential_backoff(
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def _get_object_metadata(self, get_request):
+ return self._client.get_object_metadata(get_request)
+
+ @property
+ def size(self):
+ return self._size
+
+ def get_range(self, start, end):
+ return self._client.get_range(self._get_request, start, end)
+
+
+class S3Uploader(Uploader):
+ def __init__(self, client, path, mime_type='application/octet-stream'):
+ self._client = client
+ self._path = path
+ self._bucket, self._name = parse_s3_path(path)
+ self._mime_type = mime_type
+
+ self.part_number = 1
+ self.buffer = b''
+
+ self.last_error = None
+
+ self.upload_id = None
+
+ self.parts = []
+
+ self._start_upload()
+
+ # There is retry logic in the underlying transfer library but we should make
+ # it more explicit so we can control the retry parameters.
+ @retry.no_retries # Using no_retries marks this as an integration point.
+ def _start_upload(self):
+ # The uploader by default transfers data in chunks of 1024 * 1024 bytes at
+ # a time, buffering writes until that size is reached.
+ try:
+ request = messages.UploadRequest(self._bucket,
+ self._name,
+ self._mime_type)
+ response = self._client.create_multipart_upload(request)
+ self.upload_id = response.upload_id
+ except Exception as e: # pylint: disable=broad-except
+ logging.error('Error in _start_upload while inserting file %s: %s',
+ self._path, traceback.format_exc())
+ self.last_error = e
+ raise e
+
+ def put(self, data):
+
+ MIN_WRITE_SIZE = 5 * 1024 * 1024
+ MAX_WRITE_SIZE = 5 * 1024 * 1024 * 1024
+
+ # TODO: Byte strings might not be the most performant way to handle this
+ self.buffer += data.tobytes()
+
+ while len(self.buffer) >= MIN_WRITE_SIZE:
+ # Take the first chunk off the buffer and write it to S3
+ chunk = self.buffer[:MAX_WRITE_SIZE]
+ self._write_to_s3(chunk)
+ # Remove the written chunk from the buffer
+ self.buffer = self.buffer[MAX_WRITE_SIZE:]
+
+ def _write_to_s3(self, data):
+
+ try:
+ request = messages.UploadPartRequest(self._bucket,
+ self._name,
+ self.upload_id,
+ self.part_number,
+ data)
+ response = self._client.upload_part(request)
+ self.parts.append({'ETag': response.etag,
+ 'PartNumber': response.part_number})
+ self.part_number = self.part_number + 1
+ except messages.S3ClientError as e:
+ self.last_error = e
+ if e.code == 404:
+ raise IOError(errno.ENOENT, 'Not found: %s' % self._path)
+ else:
+ logging.error('HTTP error while requesting file %s: %s', self._path,
+ 3)
+ raise
+
+ def finish(self):
+
+ self._write_to_s3(self.buffer)
+
+ if self.last_error is not None:
+ raise self.last_error # pylint: disable=raising-bad-type
+
+ request = messages.CompleteMultipartUploadRequest(self._bucket,
+ self._name,
+ self.upload_id,
+ self.parts)
+ self._client.complete_multipart_upload(request)
diff --git a/sdks/python/apache_beam/io/aws/s3io_test.py b/sdks/python/apache_beam/io/aws/s3io_test.py
new file mode 100644
index 0000000..1f55db4
--- /dev/null
+++ b/sdks/python/apache_beam/io/aws/s3io_test.py
@@ -0,0 +1,783 @@
+#
+# 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.
+#
+"""Tests for S3 client."""
+from __future__ import absolute_import
+
+import logging
+import os
+import random
+import time
+import unittest
+
+from apache_beam.io.aws import s3io
+from apache_beam.io.aws.clients.s3 import fake_client
+from apache_beam.io.aws.clients.s3 import messages
+
+
+class TestS3PathParser(unittest.TestCase):
+
+ BAD_S3_PATHS = [
+ 's3://',
+ 's3://bucket',
+ 's3:///name',
+ 's3:///',
+ 's3:/blah/bucket/name',
+ ]
+
+ def test_s3_path(self):
+ self.assertEqual(
+ s3io.parse_s3_path('s3://bucket/name'), ('bucket', 'name'))
+ self.assertEqual(
+ s3io.parse_s3_path('s3://bucket/name/sub'), ('bucket', 'name/sub'))
+
+ def test_bad_s3_path(self):
+ for path in self.BAD_S3_PATHS:
+ self.assertRaises(ValueError, s3io.parse_s3_path, path)
+ self.assertRaises(ValueError, s3io.parse_s3_path, 's3://bucket/')
+
+ def test_s3_path_object_optional(self):
+ self.assertEqual(
+ s3io.parse_s3_path('s3://bucket/name', object_optional=True),
+ ('bucket', 'name'))
+ self.assertEqual(
+ s3io.parse_s3_path('s3://bucket/', object_optional=True),
+ ('bucket', ''))
+
+ def test_bad_s3_path_object_optional(self):
+ for path in self.BAD_S3_PATHS:
+ self.assertRaises(ValueError, s3io.parse_s3_path, path, True)
+
+
+class TestS3IO(unittest.TestCase):
+
+ def _insert_random_file(self, client, path, size):
+ bucket, name = s3io.parse_s3_path(path)
+ contents = os.urandom(size)
+ fakeFile = fake_client.FakeFile(bucket, name, contents)
+
+ if self.USE_MOCK:
+ self.client.add_file(fakeFile)
+
+ else:
+ f = self.aws.open(path, 'w')
+ f.write(contents)
+ f.close()
+
+ return fakeFile
+
+ def setUp(self):
+
+ # These tests can be run locally against a mock S3 client, or as integration
+ # tests against the real S3 client.
+ self.USE_MOCK = True
+
+ # If you're running integration tests with S3, set this variable to be an
+ # s3 path that you have access to where test data can be written. If you're
+ # just running tests against the mock, this can be any s3 path. It should
+ # end with a '/'.
+ self.TEST_DATA_PATH = 's3://random-data-sets/beam_tests/'
+
+ if self.USE_MOCK:
+ self.client = fake_client.FakeS3Client()
+ test_data_bucket, _ = s3io.parse_s3_path(self.TEST_DATA_PATH)
+ self.client.known_buckets.add(test_data_bucket)
+ self.aws = s3io.S3IO(self.client)
+
+ else:
+ self.aws = s3io.S3IO()
+ self.client = self.aws.client
+
+ def test_size(self):
+ file_name = self.TEST_DATA_PATH + 'dummy_file'
+ file_size = 1234
+
+ self._insert_random_file(self.client, file_name, file_size)
+ self.assertTrue(self.aws.exists(file_name))
+ self.assertEqual(1234, self.aws.size(file_name))
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_last_updated(self):
+ file_name = self.TEST_DATA_PATH + 'dummy_file'
+ file_size = 1234
+
+ self._insert_random_file(self.client, file_name, file_size)
+ self.assertTrue(self.aws.exists(file_name))
+
+ tolerance = 5 * 60 # 5 mins
+ low_bound, high_bound = time.time() - tolerance, time.time() + tolerance
+ result = self.aws.last_updated(file_name)
+ self.assertTrue(low_bound <= result <= high_bound)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_checksum(self):
+
+ file_name = self.TEST_DATA_PATH + 'checksum'
+ file_size = 1024
+ file_ = self._insert_random_file(self.client, file_name, file_size)
+
+ original_etag = self.aws.checksum(file_name)
+
+ self.aws.delete(file_name)
+
+ with self.aws.open(file_name, 'w') as f:
+ f.write(file_.contents)
+
+ rewritten_etag = self.aws.checksum(file_name)
+
+ self.assertEqual(original_etag, rewritten_etag)
+ self.assertEqual(len(original_etag), 36)
+ self.assertTrue(original_etag.endswith('-1"'))
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_copy(self):
+ src_file_name = self.TEST_DATA_PATH + 'source'
+ dest_file_name = self.TEST_DATA_PATH + 'dest'
+ file_size = 1024
+ self._insert_random_file(self.client, src_file_name, file_size)
+
+ self.assertTrue(src_file_name in
+ self.aws.list_prefix(self.TEST_DATA_PATH))
+ self.assertFalse(dest_file_name in
+ self.aws.list_prefix(self.TEST_DATA_PATH))
+
+ self.aws.copy(src_file_name, dest_file_name)
+
+ self.assertTrue(src_file_name in
+ self.aws.list_prefix(self.TEST_DATA_PATH))
+ self.assertTrue(dest_file_name in
+ self.aws.list_prefix(self.TEST_DATA_PATH))
+
+ # Clean up
+ self.aws.delete_files([src_file_name, dest_file_name])
+
+ # Test copy of non-existent files.
+ with self.assertRaises(messages.S3ClientError) as err:
+ self.aws.copy(self.TEST_DATA_PATH + 'non-existent',
+ self.TEST_DATA_PATH + 'non-existent-destination')
+
+ self.assertTrue('Not Found' in err.exception.message)
+
+ def test_copy_paths(self):
+ from_name_pattern = self.TEST_DATA_PATH + 'copy_me_%d'
+ to_name_pattern = self.TEST_DATA_PATH + 'destination_%d'
+ file_size = 1024
+ num_files = 10
+
+ src_dest_pairs = [(from_name_pattern % i, to_name_pattern % i)
+ for i in range(num_files)]
+
+ result = self.aws.copy_paths(src_dest_pairs)
+
+ self.assertTrue(result)
+ for i, (src, dest, exception) in enumerate(result):
+ self.assertEqual(src, from_name_pattern % i)
+ self.assertEqual(dest, to_name_pattern % i)
+ self.assertTrue(isinstance(exception, messages.S3ClientError))
+ self.assertEqual(exception.code, 404)
+ self.assertFalse(self.aws.exists(from_name_pattern % i))
+ self.assertFalse(self.aws.exists(to_name_pattern % i))
+
+ # Insert some files.
+ for i in range(num_files):
+ self._insert_random_file(self.client, from_name_pattern % i, file_size)
+
+ # Check files inserted properly.
+ for i in range(num_files):
+ self.assertTrue(self.aws.exists(from_name_pattern % i))
+
+ # Execute batch copy.
+ result = self.aws.copy_paths(src_dest_pairs)
+
+ # Check files copied properly.
+ for i in range(num_files):
+ self.assertTrue(self.aws.exists(from_name_pattern % i))
+ self.assertTrue(self.aws.exists(to_name_pattern % i))
+
+ # Check results
+ for i, (src, dest, exception) in enumerate(result):
+ self.assertEqual(src_dest_pairs[i], (src, dest))
+ self.assertEqual(exception, None)
+
+ # Clean up
+ all_files = set().union(*[set(pair) for pair in src_dest_pairs])
+ self.aws.delete_files(all_files)
+
+ def test_copy_paths_error(self):
+ n_real_files = 3
+
+ # Create some files
+ from_path = self.TEST_DATA_PATH + 'copy_paths/'
+ files = [from_path + '%d' % i for i in range(n_real_files)]
+ to_path = self.TEST_DATA_PATH + 'destination/'
+ destinations = [to_path + '%d' % i for i in range(n_real_files)]
+ for file_ in files: self._insert_random_file(self.client, file_, 1024)
+
+ # Add nonexistent files to the sources and destinations
+ sources = files + [
+ from_path + 'X',
+ from_path + 'fake_directory_1/',
+ from_path + 'fake_directory_2/'
+ ]
+ destinations += [
+ to_path + 'X',
+ to_path + 'fake_directory_1/',
+ to_path + 'fake_directory_2'
+ ]
+
+ result = self.aws.copy_paths(list(zip(sources, destinations)))
+ self.assertEqual(len(result), len(sources))
+
+ for _, _, err in result[:n_real_files]:
+ self.assertTrue(err is None)
+
+ for _, _, err in result[n_real_files:]:
+ self.assertIsInstance(err, messages.S3ClientError)
+
+ self.assertEqual(result[-3][2].code, 404)
+ self.assertEqual(result[-2][2].code, 404)
+ self.assertEqual(result[-1][2].code, 400)
+
+ # Clean up
+ self.aws.delete_files(files)
+ self.aws.delete_files(destinations)
+
+
+ def test_copy_tree(self):
+ src_dir_name = self.TEST_DATA_PATH + 'source/'
+ dest_dir_name = self.TEST_DATA_PATH + 'dest/'
+ file_size = 1024
+ paths = ['a', 'b/c', 'b/d']
+ for path in paths:
+ src_file_name = src_dir_name + path
+ dest_file_name = dest_dir_name + path
+ self._insert_random_file(self.client, src_file_name, file_size)
+ self.assertTrue(
+ src_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+ self.assertFalse(
+ dest_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+
+ results = self.aws.copy_tree(src_dir_name, dest_dir_name)
+
+ for src_file_name, dest_file_name, err in results:
+
+ self.assertTrue(src_dir_name in src_file_name)
+ self.assertTrue(dest_dir_name in dest_file_name)
+ self.assertIsNone(err)
+
+ self.assertTrue(
+ src_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+ self.assertTrue(
+ dest_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+
+ # Clean up
+ for path in paths:
+ src_file_name = src_dir_name + path
+ dest_file_name = dest_dir_name + path
+ self.aws.delete_files([src_file_name, dest_file_name])
+
+ def test_rename(self):
+ src_file_name = self.TEST_DATA_PATH + 'source'
+ dest_file_name = self.TEST_DATA_PATH + 'dest'
+ file_size = 1024
+
+ self._insert_random_file(self.client, src_file_name, file_size)
+
+ self.assertTrue(
+ src_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+ self.assertFalse(
+ dest_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+
+ self.aws.rename(src_file_name, dest_file_name)
+
+ self.assertFalse(
+ src_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+ self.assertTrue(
+ dest_file_name in self.aws.list_prefix(self.TEST_DATA_PATH))
+
+ # Clean up
+ self.aws.delete_files([src_file_name, dest_file_name])
+
+ def test_rename_files(self):
+ from_name_pattern = self.TEST_DATA_PATH + 'to_rename_%d'
+ to_name_pattern = self.TEST_DATA_PATH + 'been_renamed_%d'
+ file_size = 1024
+ num_files = 10
+
+ src_dest_pairs = [(from_name_pattern % i, to_name_pattern % i)
+ for i in range(num_files)]
+
+ result = self.aws.rename_files(src_dest_pairs)
+
+ self.assertTrue(result)
+ for i, (src, dest, exception) in enumerate(result):
+ self.assertEqual(src, from_name_pattern % i)
+ self.assertEqual(dest, to_name_pattern % i)
+ self.assertTrue(isinstance(exception, messages.S3ClientError))
+ self.assertEqual(exception.code, 404)
+ self.assertFalse(self.aws.exists(from_name_pattern % i))
+ self.assertFalse(self.aws.exists(to_name_pattern % i))
+
+ # Insert some files.
+ for i in range(num_files):
+ self._insert_random_file(self.client, from_name_pattern % i, file_size)
+
+ # Check files inserted properly.
+ for i in range(num_files):
+ self.assertTrue(self.aws.exists(from_name_pattern % i))
+ self.assertFalse(self.aws.exists(to_name_pattern % i))
+
+ # Execute batch rename.
+ self.aws.rename_files(src_dest_pairs)
+
+ # Check files were renamed properly.
+ for i in range(num_files):
+ self.assertFalse(self.aws.exists(from_name_pattern % i))
+ self.assertTrue(self.aws.exists(to_name_pattern % i))
+
+ # Clean up
+ all_files = set().union(*[set(pair) for pair in src_dest_pairs])
+ self.aws.delete_files(all_files)
+
+ def test_rename_files_with_errors(self):
+ real_prefix = self.TEST_DATA_PATH + 'rename_batch_%s'
+ fake_prefix = 's3://fake-bucket-68ae4b0ef7b9/rename_batch_%s'
+ src_dest_pairs = [(prefix % 'src', prefix % 'dest')
+ for prefix in (real_prefix, fake_prefix)]
+
+ # Create the file in the real bucket
+ self._insert_random_file(self.client, real_prefix % 'src', 1024)
+
+ # Execute batch rename
+ result = self.aws.rename_files(src_dest_pairs)
+
+ # First is the file in the real bucket, which shouldn't throw an error
+ self.assertEqual(result[0][0], src_dest_pairs[0][0])
+ self.assertEqual(result[0][1], src_dest_pairs[0][1])
+ self.assertIsNone(result[0][2])
+
+ # Second is the file in the fake bucket, which should throw a 404
+ self.assertEqual(result[1][0], src_dest_pairs[1][0])
+ self.assertEqual(result[1][1], src_dest_pairs[1][1])
+ self.assertEqual(result[1][2].code, 404)
+
+ # Clean up
+ self.aws.delete(real_prefix % 'dest')
+
+ def test_rename_files_with_errors_directory(self):
+
+ # Make file
+ dir_name = self.TEST_DATA_PATH + 'rename_dir/'
+ file_name = dir_name + 'file'
+ self._insert_random_file(self.client, file_name, 1024)
+
+ self.assertTrue(self.aws.exists(file_name))
+
+ with self.assertRaises(ValueError):
+ self.aws.rename_files([(file_name, self.TEST_DATA_PATH + 'dir_dest/')])
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_delete_paths(self):
+ # Make files
+ prefix = self.TEST_DATA_PATH + 'delete_paths/'
+ file_names = [prefix + 'a', prefix + 'b/c']
+ for file_name in file_names:
+ self._insert_random_file(self.client, file_name, 1024)
+
+ self.assertTrue(self.aws.exists(file_names[0]))
+ self.assertTrue(self.aws.exists(file_names[1]))
+
+ # Delete paths
+ paths = [prefix + 'a', prefix + 'b/']
+ self.aws.delete_paths(paths)
+
+ self.assertFalse(self.aws.exists(file_names[0]))
+ self.assertFalse(self.aws.exists(file_names[1]))
+
+ def test_delete(self):
+ file_name = self.TEST_DATA_PATH + 'delete_file'
+ file_size = 1024
+
+ # Test deletion of non-existent file (shouldn't raise any error)
+ self.aws.delete(file_name)
+
+ # Create the file and check that it was created
+ self._insert_random_file(self.aws.client, file_name, file_size)
+ files = self.aws.list_prefix(self.TEST_DATA_PATH)
+ self.assertTrue(file_name in files)
+
+ # Delete the file and check that it was deleted
+ self.aws.delete(file_name)
+ self.assertFalse(self.aws.exists(file_name))
+
+ def test_delete_files(self, *unused_args):
+ file_name_pattern = self.TEST_DATA_PATH + 'delete_batch/%d'
+ file_size = 1024
+ num_files = 5
+
+ # Test deletion of non-existent files.
+ result = self.aws.delete_files(
+ [file_name_pattern % i for i in range(num_files)])
+ self.assertTrue(result)
+ for i, (file_name, exception) in enumerate(result):
+ self.assertEqual(file_name, file_name_pattern % i)
+ self.assertEqual(exception, None)
+ self.assertFalse(self.aws.exists(file_name_pattern % i))
+
+ # Insert some files.
+ for i in range(num_files):
+ self._insert_random_file(self.client, file_name_pattern % i, file_size)
+
+ # Check files inserted properly.
+ for i in range(num_files):
+ self.assertTrue(self.aws.exists(file_name_pattern % i))
+
+ # Execute batch delete.
+ self.aws.delete_files([file_name_pattern % i for i in range(num_files)])
+
+ # Check files deleted properly.
+ for i in range(num_files):
+ self.assertFalse(self.aws.exists(file_name_pattern % i))
+
+ def test_delete_files_with_errors(self, *unused_args):
+ real_file = self.TEST_DATA_PATH + 'delete_batch/file'
+ fake_file = 's3://fake-bucket-68ae4b0ef7b9/delete_batch/file'
+ filenames = [real_file, fake_file]
+
+ result = self.aws.delete_files(filenames)
+
+ # First is the file in the real bucket, which shouldn't throw an error
+ self.assertEqual(result[0][0], filenames[0])
+ self.assertIsNone(result[0][1])
+
+ # Second is the file in the fake bucket, which should throw a 404
+ self.assertEqual(result[1][0], filenames[1])
+ self.assertEqual(result[1][1].code, 404)
+
+ def test_delete_tree(self):
+
+ root_path = self.TEST_DATA_PATH + 'delete_tree/'
+ leaf_paths = ['a', 'b/c', 'b/d', 'b/d/e']
+ paths = [root_path + leaf for leaf in leaf_paths]
+
+ # Create file tree
+ file_size = 1024
+ for path in paths:
+ self._insert_random_file(self.client, path, file_size)
+
+ # Check that the files exist
+ for path in paths:
+ self.assertTrue(self.aws.exists(path))
+
+ # Delete the tree
+ self.aws.delete_tree(root_path)
+
+ # Check that the files have been deleted
+ for path in paths:
+ self.assertFalse(self.aws.exists(path))
+
+ def test_exists(self):
+ file_name = self.TEST_DATA_PATH + 'exists'
+ file_size = 1024
+
+ self.assertFalse(self.aws.exists(file_name))
+
+ self._insert_random_file(self.aws.client, file_name, file_size)
+
+ self.assertTrue(self.aws.exists(file_name))
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ self.assertFalse(self.aws.exists(file_name))
+
+ def test_file_mode(self):
+ file_name = self.TEST_DATA_PATH + 'jerry/pigpen/bobby'
+ with self.aws.open(file_name, 'w') as f:
+ assert f.mode == 'w'
+ with self.aws.open(file_name, 'r') as f:
+ assert f.mode == 'r'
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_full_file_read(self):
+ file_name = self.TEST_DATA_PATH + 'jerry/pigpen/phil'
+ file_size = 1024
+
+ f = self._insert_random_file(self.aws.client, file_name, file_size)
+ contents = f.contents
+
+ f = self.aws.open(file_name)
+ self.assertEqual(f.mode, 'r')
+ f.seek(0, os.SEEK_END)
+ self.assertEqual(f.tell(), file_size)
+ self.assertEqual(f.read(), b'')
+ f.seek(0)
+ self.assertEqual(f.read(), contents)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_file_write(self):
+ file_name = self.TEST_DATA_PATH + 'write_file'
+ file_size = 8 * 1024 * 1024 + 2000
+ contents = os.urandom(file_size)
+ f = self.aws.open(file_name, 'w')
+ self.assertEqual(f.mode, 'w')
+ f.write(contents[0:1000])
+ f.write(contents[1000:1024 * 1024])
+ f.write(contents[1024 * 1024:])
+ f.close()
+ new_f = self.aws.open(file_name, 'r')
+ new_f_contents = new_f.read()
+ self.assertEqual(
+ new_f_contents, contents)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_file_mime_type(self):
+ if self.USE_MOCK:
+ self.skipTest("The boto3_client mock doesn't support mime_types")
+
+ mime_type = 'example/example'
+ file_name = self.TEST_DATA_PATH + 'write_file'
+ f = self.aws.open(file_name, 'w', mime_type=mime_type)
+ f.write(b'a string of binary text')
+ f.close()
+
+ bucket, key = s3io.parse_s3_path(file_name)
+ metadata = self.client.get_object_metadata(messages.GetRequest(bucket, key))
+
+ self.assertEqual(mime_type, metadata.mime_type)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+
+ def test_file_random_seek(self):
+ file_name = self.TEST_DATA_PATH + 'write_seek_file'
+ file_size = 5 * 1024 * 1024 - 100
+ contents = os.urandom(file_size)
+ with self.aws.open(file_name, 'w') as wf:
+ wf.write(contents)
+
+ f = self.aws.open(file_name)
+ random.seed(0)
+
+ for _ in range(0, 10):
+ a = random.randint(0, file_size - 1)
+ b = random.randint(0, file_size - 1)
+ start, end = min(a, b), max(a, b)
+ f.seek(start)
+
+ self.assertEqual(f.tell(), start)
+
+ self.assertEqual(
+ f.read(end - start + 1), contents[start:end + 1]
+ )
+ self.assertEqual(f.tell(), end + 1)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_file_flush(self):
+ file_name = self.TEST_DATA_PATH + 'flush_file'
+ file_size = 5 * 1024 * 1024 + 2000
+ contents = os.urandom(file_size)
+ f = self.aws.open(file_name, 'w')
+ self.assertEqual(f.mode, 'w')
+ f.write(contents[0:1000])
+ f.flush()
+ f.write(contents[1000:1024 * 1024])
+ f.flush()
+ f.flush() # Should be a NOOP.
+ f.write(contents[1024 * 1024:])
+ f.close() # This should al`read`y call the equivalent of flush() in its body
+ new_f = self.aws.open(file_name, 'r')
+ new_f_contents = new_f.read()
+ self.assertEqual(
+ new_f_contents, contents)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_file_iterator(self):
+ file_name = self.TEST_DATA_PATH + 'iterate_file'
+ lines = []
+ line_count = 10
+ for _ in range(line_count):
+ line_length = random.randint(100, 500)
+ line = os.urandom(line_length).replace(b'\n', b' ') + b'\n'
+ lines.append(line)
+
+ contents = b''.join(lines)
+
+ with self.aws.open(file_name, 'w') as wf:
+ wf.write(contents)
+
+ f = self.aws.open(file_name)
+
+ read_lines = 0
+ for line in f:
+ read_lines += 1
+
+ self.assertEqual(read_lines, line_count)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_file_read_line(self):
+ file_name = self.TEST_DATA_PATH + 'read_line_file'
+ lines = []
+
+ # Set a small buffer size to exercise refilling the buffer.
+ # First line is carefully crafted so the newline falls as the last character
+ # of the buffer to exercise this code path.
+ read_buffer_size = 1099
+ lines.append(b'x' * 1023 + b'\n')
+
+ for _ in range(1, 1000):
+ line_length = random.randint(100, 500)
+ line = os.urandom(line_length).replace(b'\n', b' ') + b'\n'
+ lines.append(line)
+ contents = b''.join(lines)
+
+ file_size = len(contents)
+
+ with self.aws.open(file_name, 'wb') as wf:
+ wf.write(contents)
+
+ f = self.aws.open(file_name, 'rb', read_buffer_size=read_buffer_size)
+
+ # Test read of first two lines.
+ f.seek(0)
+ self.assertEqual(f.readline(), lines[0])
+ self.assertEqual(f.tell(), len(lines[0]))
+ self.assertEqual(f.readline(), lines[1])
+
+ # Test read at line boundary.
+ f.seek(file_size - len(lines[-1]) - 1)
+ self.assertEqual(f.readline(), b'\n')
+
+ # Test read at end of file.
+ f.seek(file_size)
+ self.assertEqual(f.readline(), b'')
+
+ # Test reads at random positions.
+ random.seed(0)
+ for _ in range(0, 10):
+ start = random.randint(0, file_size - 1)
+ line_index = 0
+ # Find line corresponding to start index.
+ chars_left = start
+ while True:
+ next_line_length = len(lines[line_index])
+ if chars_left - next_line_length < 0:
+ break
+ chars_left -= next_line_length
+ line_index += 1
+ f.seek(start)
+ self.assertEqual(f.readline(), lines[line_index][chars_left:])
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_file_close(self):
+ file_name = self.TEST_DATA_PATH + 'close_file'
+ file_size = 5 * 1024 * 1024 + 2000
+ contents = os.urandom(file_size)
+ f = self.aws.open(file_name, 'w')
+ self.assertEqual(f.mode, 'w')
+ f.write(contents)
+ f.close()
+ f.close() # This should not crash.
+
+ with self.aws.open(file_name, 'r') as f:
+ read_contents = f.read()
+
+ self.assertEqual(
+ read_contents, contents)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_context_manager(self):
+ # Test writing with a context manager.
+ file_name = self.TEST_DATA_PATH + 'context_manager_file'
+ file_size = 1024
+ contents = os.urandom(file_size)
+ with self.aws.open(file_name, 'w') as f:
+ f.write(contents)
+
+ with self.aws.open(file_name, 'r') as f:
+ self.assertEqual(f.read(), contents)
+
+ # Clean up
+ self.aws.delete(file_name)
+
+ def test_list_prefix(self):
+
+ objects = [
+ ('jerry/pigpen/phil', 5),
+ ('jerry/pigpen/bobby', 3),
+ ('jerry/billy/bobby', 4),
+ ]
+
+ for (object_name, size) in objects:
+ file_name = self.TEST_DATA_PATH + object_name
+ self._insert_random_file(self.aws.client, file_name, size)
+
+ test_cases = [
+ (self.TEST_DATA_PATH + 'j', [
+ ('jerry/pigpen/phil', 5),
+ ('jerry/pigpen/bobby', 3),
+ ('jerry/billy/bobby', 4),
+ ]),
+ (self.TEST_DATA_PATH + 'jerry/', [
+ ('jerry/pigpen/phil', 5),
+ ('jerry/pigpen/bobby', 3),
+ ('jerry/billy/bobby', 4),
+ ]),
+ (self.TEST_DATA_PATH + 'jerry/pigpen/phil', [
+ ('jerry/pigpen/phil', 5),
+ ]),
+ ]
+
+ for file_pattern, expected_object_names in test_cases:
+ expected_file_names = [(self.TEST_DATA_PATH + object_name, size)
+ for (object_name, size) in expected_object_names]
+ self.assertEqual(
+ set(self.aws.list_prefix(file_pattern).items()),
+ set(expected_file_names))
+
+ # Clean up
+ for (object_name, size) in objects:
+ self.aws.delete(self.TEST_DATA_PATH + object_name)
+
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ unittest.main()
diff --git a/sdks/python/apache_beam/io/fileio_test.py b/sdks/python/apache_beam/io/fileio_test.py
index b724910..c83c8d7 100644
--- a/sdks/python/apache_beam/io/fileio_test.py
+++ b/sdks/python/apache_beam/io/fileio_test.py
@@ -513,6 +513,7 @@
# TODO(BEAM-3759): Add many firings per window after getting PaneInfo.
ts.advance_processing_time(5)
ts.advance_watermark_to(timestamp)
+ ts.advance_watermark_to_infinity()
def no_colon_file_naming(*args):
file_name = fileio.destination_prefix_naming()(*args)
@@ -572,7 +573,8 @@
.add_elements([next(input), next(input)])
.advance_watermark_to(30)
.add_elements([next(input), next(input)])
- .advance_watermark_to(40))
+ .advance_watermark_to(40)
+ .advance_watermark_to_infinity())
def no_colon_file_naming(*args):
file_name = fileio.destination_prefix_naming()(*args)
diff --git a/sdks/python/apache_beam/io/filesystems.py b/sdks/python/apache_beam/io/filesystems.py
index 86fe09c..39e6100 100644
--- a/sdks/python/apache_beam/io/filesystems.py
+++ b/sdks/python/apache_beam/io/filesystems.py
@@ -51,6 +51,11 @@
except ImportError:
pass
+try:
+ from apache_beam.io.aws.s3filesystem import S3FileSystem
+except ImportError:
+ pass
+
# pylint: enable=wrong-import-position, unused-import
@@ -91,7 +96,9 @@
systems = [fs for fs in FileSystem.get_all_subclasses()
if fs.scheme() == path_scheme]
if len(systems) == 0:
- raise ValueError('Unable to get the Filesystem for path %s' % path)
+ raise ValueError('Unable to get filesystem from specified path, please use the correct path '
+ 'or ensure the required dependency is installed, e.g., pip install apache_beam[gcp]. '
+ 'Path specified: %s' % path)
elif len(systems) == 1:
# Pipeline options could come either from the Pipeline itself (using
# direct runner), or via RuntimeValueProvider (other runners).
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index 61c93ff..e685c8b 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -232,6 +232,7 @@
from __future__ import absolute_import
import collections
+import decimal
import itertools
import json
import logging
@@ -248,12 +249,20 @@
from apache_beam import pvalue
from apache_beam.internal.gcp.json_value import from_json_value
from apache_beam.internal.gcp.json_value import to_json_value
+from apache_beam.io.filesystems import CompressionTypes
+from apache_beam.io.filesystems import FileSystems
from apache_beam.io.gcp import bigquery_tools
from apache_beam.io.gcp.internal.clients import bigquery
+from apache_beam.io.iobase import BoundedSource
+from apache_beam.io.iobase import RangeTracker
+from apache_beam.io.iobase import SourceBundle
+from apache_beam.io.textio import _TextSource as TextSource
from apache_beam.options import value_provider as vp
from apache_beam.options.pipeline_options import DebugOptions
from apache_beam.options.pipeline_options import GoogleCloudOptions
from apache_beam.options.pipeline_options import StandardOptions
+from apache_beam.options.value_provider import StaticValueProvider
+from apache_beam.options.value_provider import ValueProvider
from apache_beam.runners.dataflow.native_io import iobase as dataflow_io
from apache_beam.transforms import DoFn
from apache_beam.transforms import ParDo
@@ -262,6 +271,7 @@
from apache_beam.transforms.window import GlobalWindows
from apache_beam.utils import retry
from apache_beam.utils.annotations import deprecated
+from apache_beam.utils.annotations import experimental
__all__ = [
'TableRowJsonCoder',
@@ -269,6 +279,7 @@
'BigQuerySource',
'BigQuerySink',
'WriteToBigQuery',
+ '_ReadFromBigQuery',
'SCHEMA_AUTODETECT',
]
@@ -499,6 +510,194 @@
kms_key=self.kms_key)
+FieldSchema = collections.namedtuple('FieldSchema', 'fields mode name type')
+
+
+def _to_bool(value):
+ return value == 'true'
+
+
+def _to_decimal(value):
+ return decimal.Decimal(value)
+
+
+def _to_bytes(value):
+ """Converts value from str to bytes on Python 3.x. Does nothing on
+ Python 2.7."""
+ return value.encode('utf-8')
+
+
+class _JsonToDictCoder(coders.Coder):
+ """A coder for a JSON string to a Python dict."""
+
+ def __init__(self, table_schema):
+ self.fields = self._convert_to_tuple(table_schema.fields)
+ self._converters = {
+ 'INTEGER': int,
+ 'INT64': int,
+ 'FLOAT': float,
+ 'BOOLEAN': _to_bool,
+ 'NUMERIC': _to_decimal,
+ 'BYTES': _to_bytes,
+ }
+
+ @classmethod
+ def _convert_to_tuple(cls, table_field_schemas):
+ """Recursively converts the list of TableFieldSchema instances to the
+ list of tuples to prevent errors when pickling and unpickling
+ TableFieldSchema instances.
+ """
+ if not table_field_schemas:
+ return []
+
+ return [FieldSchema(cls._convert_to_tuple(x.fields), x.mode, x.name,
+ x.type)
+ for x in table_field_schemas]
+
+ def decode(self, value):
+ value = json.loads(value.decode('utf-8'))
+ return self._decode_with_schema(value, self.fields)
+
+ def _decode_with_schema(self, value, schema_fields):
+ for field in schema_fields:
+ if field.name not in value:
+ # The field exists in the schema, but it doesn't exist in this row.
+ # It probably means its value was null, as the extract to JSON job
+ # doesn't preserve null fields
+ value[field.name] = None
+ continue
+
+ if field.type == 'RECORD':
+ value[field.name] = self._decode_with_schema(value[field.name],
+ field.fields)
+ else:
+ try:
+ converter = self._converters[field.type]
+ value[field.name] = converter(value[field.name])
+ except KeyError:
+ # No need to do any conversion
+ pass
+ return value
+
+ def is_deterministic(self):
+ return True
+
+ def to_type_hint(self):
+ return dict
+
+
+class _CustomBigQuerySource(BoundedSource):
+ def __init__(self, gcs_location=None, table=None, dataset=None,
+ project=None, query=None, validate=False, coder=None,
+ use_standard_sql=False, flatten_results=True, kms_key=None):
+ if table is not None and query is not None:
+ raise ValueError('Both a BigQuery table and a query were specified.'
+ ' Please specify only one of these.')
+ elif table is None and query is None:
+ raise ValueError('A BigQuery table or a query must be specified')
+ elif table is not None:
+ self.table_reference = bigquery_tools.parse_table_reference(
+ table, dataset, project)
+ self.query = None
+ self.use_legacy_sql = True
+ else:
+ self.query = query
+ # TODO(BEAM-1082): Change the internal flag to be standard_sql
+ self.use_legacy_sql = not use_standard_sql
+ self.table_reference = None
+
+ self.gcs_location = gcs_location
+ self.project = project
+ self.validate = validate
+ self.flatten_results = flatten_results
+ self.coder = coder or _JsonToDictCoder
+ self.kms_key = kms_key
+ self.split_result = None
+
+ def estimate_size(self):
+ bq = bigquery_tools.BigQueryWrapper()
+ if self.table_reference is not None:
+ table = bq.get_table(self.table_reference.projectId,
+ self.table_reference.datasetId,
+ self.table_reference.tableId)
+ return int(table.numBytes)
+ else:
+ job = bq._start_query_job(self.project, self.query,
+ self.use_legacy_sql, self.flatten_results,
+ job_id=uuid.uuid4().hex, dry_run=True,
+ kms_key=self.kms_key)
+ size = int(job.statistics.totalBytesProcessed)
+ return size
+
+ def split(self, desired_bundle_size, start_position=None, stop_position=None):
+ if self.split_result is None:
+ bq = bigquery_tools.BigQueryWrapper()
+
+ if self.query is not None:
+ self._setup_temporary_dataset(bq)
+ self.table_reference = self._execute_query(bq)
+
+ schema, metadata_list = self._export_files(bq)
+ self.split_result = [TextSource(metadata.path, 0,
+ CompressionTypes.UNCOMPRESSED, True,
+ self.coder(schema))
+ for metadata in metadata_list]
+
+ if self.query is not None:
+ bq.clean_up_temporary_dataset(self.project)
+
+ for source in self.split_result:
+ yield SourceBundle(0, source, None, None)
+
+ def get_range_tracker(self, start_position, stop_position):
+ class CustomBigQuerySourceRangeTracker(RangeTracker):
+ """A RangeTracker that always returns positions as None."""
+
+ def start_position(self):
+ return None
+
+ def stop_position(self):
+ return None
+
+ return CustomBigQuerySourceRangeTracker()
+
+ def read(self, range_tracker):
+ raise NotImplementedError('BigQuery source must be split before being read')
+
+ def _setup_temporary_dataset(self, bq):
+ location = bq.get_query_location(self.project, self.query,
+ self.use_legacy_sql)
+ bq.create_temporary_dataset(self.project, location)
+
+ def _execute_query(self, bq):
+ job = bq._start_query_job(self.project, self.query,
+ self.use_legacy_sql, self.flatten_results,
+ job_id=uuid.uuid4().hex, kms_key=self.kms_key)
+ job_ref = job.jobReference
+ bq.wait_for_bq_job(job_ref)
+ return bq._get_temp_table(self.project)
+
+ def _export_files(self, bq):
+ """Runs a BigQuery export job.
+
+ Returns:
+ bigquery.TableSchema instance, a list of FileMetadata instances
+ """
+ job_id = uuid.uuid4().hex
+ job_ref = bq.perform_extract_job([self.gcs_location], job_id,
+ self.table_reference,
+ bigquery_tools.ExportFileFormat.JSON,
+ include_header=False)
+ bq.wait_for_bq_job(job_ref)
+ metadata_list = FileSystems.match([self.gcs_location])[0].metadata_list
+
+ table = bq.get_table(self.table_reference.projectId,
+ self.table_reference.datasetId,
+ self.table_reference.tableId)
+
+ return table.schema, metadata_list
+
+
@deprecated(since='2.11.0', current="WriteToBigQuery")
class BigQuerySink(dataflow_io.NativeSink):
"""A sink based on a BigQuery table.
@@ -1274,3 +1473,139 @@
tableSpec)
res['table'] = DisplayDataItem(tableSpec, label='Table')
return res
+
+
+class _PassThroughThenCleanup(PTransform):
+ """A PTransform that invokes a DoFn after the input PCollection has been
+ processed.
+ """
+ def __init__(self, cleanup_dofn):
+ self.cleanup_dofn = cleanup_dofn
+
+ def expand(self, input):
+ class PassThrough(beam.DoFn):
+ def process(self, element):
+ yield element
+
+ output = input | beam.ParDo(PassThrough()).with_outputs('cleanup_signal',
+ main='main')
+ main_output = output['main']
+ cleanup_signal = output['cleanup_signal']
+
+ _ = (input.pipeline
+ | beam.Create([None])
+ | beam.ParDo(self.cleanup_dofn, beam.pvalue.AsSingleton(
+ cleanup_signal)))
+
+ return main_output
+
+
+@experimental()
+class _ReadFromBigQuery(PTransform):
+ """Read data from BigQuery.
+
+ This PTransform uses a BigQuery export job to take a snapshot of the table
+ on GCS, and then reads from each produced JSON file.
+
+ Do note that currently this source does not work with DirectRunner.
+
+ Args:
+ table (str, callable, ValueProvider): The ID of the table, or a callable
+ that returns it. The ID must contain only letters ``a-z``, ``A-Z``,
+ numbers ``0-9``, or underscores ``_``. If dataset argument is
+ :data:`None` then the table argument must contain the entire table
+ reference specified as: ``'DATASET.TABLE'``
+ or ``'PROJECT:DATASET.TABLE'``. If it's a callable, it must receive one
+ argument representing an element to be written to BigQuery, and return
+ a TableReference, or a string table name as specified above.
+ dataset (str): The ID of the dataset containing this table or
+ :data:`None` if the table reference is specified entirely by the table
+ argument.
+ project (str): The ID of the project containing this table.
+ query (str): A query to be used instead of arguments table, dataset, and
+ project.
+ validate (bool): If :data:`True`, various checks will be done when source
+ gets initialized (e.g., is table present?). This should be
+ :data:`True` for most scenarios in order to catch errors as early as
+ possible (pipeline construction instead of pipeline execution). It
+ should be :data:`False` if the table is created during pipeline
+ execution by a previous step.
+ coder (~apache_beam.coders.coders.Coder): The coder for the table
+ rows. If :data:`None`, then the default coder is
+ _JsonToDictCoder, which will interpret every row as a JSON
+ serialized dictionary.
+ use_standard_sql (bool): Specifies whether to use BigQuery's standard SQL
+ dialect for this query. The default value is :data:`False`.
+ If set to :data:`True`, the query will use BigQuery's updated SQL
+ dialect with improved standards compliance.
+ This parameter is ignored for table inputs.
+ flatten_results (bool): Flattens all nested and repeated fields in the
+ query results. The default value is :data:`True`.
+ kms_key (str): Experimental. Optional Cloud KMS key name for use when
+ creating new temporary tables.
+ gcs_location (str): The name of the Google Cloud Storage bucket where
+ the extracted table should be written as a string or
+ a :class:`~apache_beam.options.value_provider.ValueProvider`. If
+ :data:`None`, then the temp_location parameter is used.
+ """
+ def __init__(self, gcs_location=None, validate=False, *args, **kwargs):
+ if gcs_location:
+ if not isinstance(gcs_location, (str, unicode, ValueProvider)):
+ raise TypeError('%s: gcs_location must be of type string'
+ ' or ValueProvider; got %r instead'
+ % (self.__class__.__name__, type(gcs_location)))
+
+ if isinstance(gcs_location, (str, unicode)):
+ gcs_location = StaticValueProvider(str, gcs_location)
+ self.gcs_location = gcs_location
+ self.validate = validate
+
+ self._args = args
+ self._kwargs = kwargs
+
+ def _get_destination_uri(self, temp_location):
+ """Returns the fully qualified Google Cloud Storage URI where the
+ extracted table should be written.
+ """
+ file_pattern = 'bigquery-table-dump-*.json'
+
+ if self.gcs_location is not None:
+ gcs_base = self.gcs_location.get()
+ elif temp_location is not None:
+ gcs_base = temp_location
+ logging.debug("gcs_location is empty, using temp_location instead")
+ else:
+ raise ValueError('{} requires a GCS location to be provided'
+ .format(self.__class__.__name__))
+ if self.validate:
+ self._validate_gcs_location(gcs_base)
+
+ job_id = uuid.uuid4().hex
+ return FileSystems.join(gcs_base, job_id, file_pattern)
+
+ @staticmethod
+ def _validate_gcs_location(gcs_location):
+ if not gcs_location.startswith('gs://'):
+ raise ValueError('Invalid GCS location: {}'.format(gcs_location))
+
+ def expand(self, pcoll):
+ class RemoveJsonFiles(beam.DoFn):
+ def __init__(self, gcs_location):
+ self._gcs_location = gcs_location
+
+ def process(self, unused_element, signal):
+ match_result = FileSystems.match([self._gcs_location])[0].metadata_list
+ logging.debug("%s: matched %s files", self.__class__.__name__,
+ len(match_result))
+ paths = [x.path for x in match_result]
+ FileSystems.delete(paths)
+
+ temp_location = pcoll.pipeline.options.view_as(
+ GoogleCloudOptions).temp_location
+ gcs_location = self._get_destination_uri(temp_location)
+
+ return (pcoll
+ | beam.io.Read(_CustomBigQuerySource(gcs_location=gcs_location,
+ validate=self.validate,
+ *self._args, **self._kwargs))
+ | _PassThroughThenCleanup(RemoveJsonFiles(gcs_location)))
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
index 06525cd..5864a83 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
@@ -32,7 +32,6 @@
import hashlib
import logging
import random
-import time
import uuid
from future.utils import iteritems
@@ -493,11 +492,8 @@
Experimental; no backwards compatibility guarantees.
"""
- ALL_DONE = object()
- FAILED = object()
- WAITING = object()
- def __init__(self, test_client):
+ def __init__(self, test_client=None):
self.test_client = test_client
def start_bundle(self):
@@ -505,34 +501,10 @@
def process(self, element, dest_ids_list):
job_references = [elm[1] for elm in dest_ids_list]
-
- while True:
- status = self._check_job_states(job_references)
- if status == WaitForBQJobs.FAILED:
- raise Exception(
- 'BigQuery jobs failed. BQ error: %s', self._latest_error)
- elif status == WaitForBQJobs.ALL_DONE:
- return dest_ids_list # Pass the list of destination-jobs downstream
- time.sleep(10)
-
- def _check_job_states(self, job_references):
for ref in job_references:
- job = self.bq_wrapper.get_job(ref.projectId,
- ref.jobId,
- ref.location)
+ self.bq_wrapper.wait_for_bq_job(ref, sleep_duration_sec=10)
- _LOGGER.info("Job status: %s", job.status)
- if job.status.state == 'DONE' and job.status.errorResult:
- _LOGGER.warning("Job %s seems to have failed. Error Result: %s",
- ref.jobId, job.status.errorResult)
- self._latest_error = job.status
- return WaitForBQJobs.FAILED
- elif job.status.state == 'DONE':
- continue
- else:
- return WaitForBQJobs.WAITING
-
- return WaitForBQJobs.ALL_DONE
+ return dest_ids_list # Pass the list of destination-jobs downstream
class DeleteTablesFn(beam.DoFn):
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
index ff63eda..31ca577 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
@@ -26,6 +26,7 @@
import time
import unittest
from decimal import Decimal
+from functools import wraps
from future.utils import iteritems
from nose.plugins.attrib import attr
@@ -49,35 +50,73 @@
_LOGGER = logging.getLogger(__name__)
+def skip(runners):
+ if not isinstance(runners, list):
+ runners = [runners]
+
+ def inner(fn):
+ @wraps(fn)
+ def wrapped(self):
+ if self.runner_name in runners:
+ self.skipTest('This test doesn\'t work on these runners: {}'.format(
+ runners))
+ else:
+ return fn(self)
+ return wrapped
+ return inner
+
+
class BigQueryReadIntegrationTests(unittest.TestCase):
BIG_QUERY_DATASET_ID = 'python_read_table_'
- def setUp(self):
- self.test_pipeline = TestPipeline(is_integration_test=True)
- self.runner_name = type(self.test_pipeline.runner).__name__
- self.project = self.test_pipeline.get_option('project')
+ @classmethod
+ def setUpClass(cls):
+ cls.test_pipeline = TestPipeline(is_integration_test=True)
+ cls.args = cls.test_pipeline.get_full_options_as_args()
+ cls.runner_name = type(cls.test_pipeline.runner).__name__
+ cls.project = cls.test_pipeline.get_option('project')
- self.bigquery_client = BigQueryWrapper()
- self.dataset_id = '%s%s%d' % (self.BIG_QUERY_DATASET_ID,
- str(int(time.time())),
- random.randint(0, 10000))
- self.bigquery_client.get_or_create_dataset(self.project, self.dataset_id)
+ cls.bigquery_client = BigQueryWrapper()
+ cls.dataset_id = '%s%s%d' % (cls.BIG_QUERY_DATASET_ID,
+ str(int(time.time())),
+ random.randint(0, 10000))
+ cls.bigquery_client.get_or_create_dataset(cls.project, cls.dataset_id)
_LOGGER.info("Created dataset %s in project %s",
- self.dataset_id, self.project)
+ cls.dataset_id, cls.project)
- def tearDown(self):
+ @classmethod
+ def tearDownClass(cls):
request = bigquery.BigqueryDatasetsDeleteRequest(
- projectId=self.project, datasetId=self.dataset_id,
+ projectId=cls.project, datasetId=cls.dataset_id,
deleteContents=True)
try:
_LOGGER.info("Deleting dataset %s in project %s",
- self.dataset_id, self.project)
- self.bigquery_client.client.datasets.Delete(request)
+ cls.dataset_id, cls.project)
+ cls.bigquery_client.client.datasets.Delete(request)
except HttpError:
_LOGGER.debug('Failed to clean up dataset %s in project %s',
- self.dataset_id, self.project)
+ cls.dataset_id, cls.project)
- def create_table(self, tablename):
+
+class ReadTests(BigQueryReadIntegrationTests):
+ TABLE_DATA = [
+ {'number': 1, 'str': 'abc'},
+ {'number': 2, 'str': 'def'},
+ {'number': 3, 'str': u'你好'},
+ {'number': 4, 'str': u'привет'}
+ ]
+
+ @classmethod
+ def setUpClass(cls):
+ super(ReadTests, cls).setUpClass()
+ cls.table_name = 'python_write_table'
+ cls.create_table(cls.table_name)
+
+ table_id = '{}.{}'.format(cls.dataset_id, cls.table_name)
+ cls.query = 'SELECT number, str FROM `%s`' % table_id
+
+ @classmethod
+ def create_table(cls, table_name):
table_schema = bigquery.TableSchema()
table_field = bigquery.TableFieldSchema()
table_field.name = 'number'
@@ -89,23 +128,45 @@
table_schema.fields.append(table_field)
table = bigquery.Table(
tableReference=bigquery.TableReference(
- projectId=self.project,
- datasetId=self.dataset_id,
- tableId=tablename),
+ projectId=cls.project,
+ datasetId=cls.dataset_id,
+ tableId=table_name),
schema=table_schema)
request = bigquery.BigqueryTablesInsertRequest(
- projectId=self.project, datasetId=self.dataset_id, table=table)
- self.bigquery_client.client.tables.Insert(request)
- table_data = [
- {'number': 1, 'str': 'abc'},
- {'number': 2, 'str': 'def'},
- {'number': 3, 'str': u'你好'},
- {'number': 4, 'str': u'привет'}
- ]
- self.bigquery_client.insert_rows(
- self.project, self.dataset_id, tablename, table_data)
+ projectId=cls.project, datasetId=cls.dataset_id, table=table)
+ cls.bigquery_client.client.tables.Insert(request)
+ cls.bigquery_client.insert_rows(
+ cls.project, cls.dataset_id, table_name, cls.TABLE_DATA)
- def create_table_new_types(self, table_name):
+ @skip(['PortableRunner', 'FlinkRunner'])
+ @attr('IT')
+ def test_native_source(self):
+ with beam.Pipeline(argv=self.args) as p:
+ result = (p | 'read' >> beam.io.Read(beam.io.BigQuerySource(
+ query=self.query, use_standard_sql=True)))
+ assert_that(result, equal_to(self.TABLE_DATA))
+
+ @attr('IT')
+ def test_iobase_source(self):
+ with beam.Pipeline(argv=self.args) as p:
+ result = (p | 'read' >> beam.io._ReadFromBigQuery(
+ query=self.query, use_standard_sql=True, project=self.project))
+ assert_that(result, equal_to(self.TABLE_DATA))
+
+
+class ReadNewTypesTests(BigQueryReadIntegrationTests):
+ @classmethod
+ def setUpClass(cls):
+ super(ReadNewTypesTests, cls).setUpClass()
+ cls.table_name = 'python_new_types'
+ cls.create_table(cls.table_name)
+
+ table_id = '{}.{}'.format(cls.dataset_id, cls.table_name)
+ cls.query = 'SELECT float, numeric, bytes, date, time, datetime,' \
+ 'timestamp, geo FROM `%s`' % table_id
+
+ @classmethod
+ def create_table(cls, table_name):
table_schema = bigquery.TableSchema()
table_field = bigquery.TableFieldSchema()
table_field.name = 'float'
@@ -141,13 +202,13 @@
table_schema.fields.append(table_field)
table = bigquery.Table(
tableReference=bigquery.TableReference(
- projectId=self.project,
- datasetId=self.dataset_id,
+ projectId=cls.project,
+ datasetId=cls.dataset_id,
tableId=table_name),
schema=table_schema)
request = bigquery.BigqueryTablesInsertRequest(
- projectId=self.project, datasetId=self.dataset_id, table=table)
- self.bigquery_client.client.tables.Insert(request)
+ projectId=cls.project, datasetId=cls.dataset_id, table=table)
+ cls.bigquery_client.client.tables.Insert(request)
row_data = {
'float': 0.33, 'numeric': Decimal('10'), 'bytes':
base64.b64encode(b'\xab\xac').decode('utf-8'), 'date': '3000-12-31',
@@ -160,34 +221,10 @@
for key, value in iteritems(row_data):
table_data.append({key: value})
- self.bigquery_client.insert_rows(
- self.project, self.dataset_id, table_name, table_data)
+ cls.bigquery_client.insert_rows(
+ cls.project, cls.dataset_id, table_name, table_data)
- @attr('IT')
- def test_big_query_read(self):
- table_name = 'python_write_table'
- self.create_table(table_name)
- table_id = '{}.{}'.format(self.dataset_id, table_name)
-
- args = self.test_pipeline.get_full_options_as_args()
-
- with beam.Pipeline(argv=args) as p:
- result = (p | 'read' >> beam.io.Read(beam.io.BigQuerySource(
- query='SELECT number, str FROM `%s`' % table_id,
- use_standard_sql=True)))
- assert_that(result, equal_to([{'number': 1, 'str': 'abc'},
- {'number': 2, 'str': 'def'},
- {'number': 3, 'str': u'你好'},
- {'number': 4, 'str': u'привет'}]))
-
- @attr('IT')
- def test_big_query_read_new_types(self):
- table_name = 'python_new_types'
- self.create_table_new_types(table_name)
- table_id = '{}.{}'.format(self.dataset_id, table_name)
-
- args = self.test_pipeline.get_full_options_as_args()
-
+ def get_expected_data(self):
expected_row = {
'float': 0.33, 'numeric': Decimal('10'), 'bytes':
base64.b64encode(b'\xab\xac'), 'date': '3000-12-31',
@@ -203,12 +240,22 @@
row[key] = value
expected_data.append(row)
- with beam.Pipeline(argv=args) as p:
+ return expected_data
+
+ @skip(['PortableRunner', 'FlinkRunner'])
+ @attr('IT')
+ def test_native_source(self):
+ with beam.Pipeline(argv=self.args) as p:
result = (p | 'read' >> beam.io.Read(beam.io.BigQuerySource(
- query='SELECT float, numeric, bytes, date, time, datetime,'
- 'timestamp, geo FROM `%s`' % table_id,
- use_standard_sql=True)))
- assert_that(result, equal_to(expected_data))
+ query=self.query, use_standard_sql=True)))
+ assert_that(result, equal_to(self.get_expected_data()))
+
+ @attr('IT')
+ def test_iobase_source(self):
+ with beam.Pipeline(argv=self.args) as p:
+ result = (p | 'read' >> beam.io._ReadFromBigQuery(
+ query=self.query, use_standard_sql=True, project=self.project))
+ assert_that(result, equal_to(self.get_expected_data()))
if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py
index ac62774..2125260 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py
@@ -22,6 +22,7 @@
import json
import logging
import os
+import pickle
import random
import re
import time
@@ -35,11 +36,13 @@
from nose.plugins.attrib import attr
import apache_beam as beam
+from apache_beam.internal import pickler
from apache_beam.internal.gcp.json_value import to_json_value
from apache_beam.io.filebasedsink_test import _TestCaseWithTempDirCleanUp
from apache_beam.io.gcp import bigquery_tools
from apache_beam.io.gcp.bigquery import TableRowJsonCoder
from apache_beam.io.gcp.bigquery import WriteToBigQuery
+from apache_beam.io.gcp.bigquery import _JsonToDictCoder
from apache_beam.io.gcp.bigquery import _StreamToBigQuery
from apache_beam.io.gcp.bigquery_file_loads_test import _ELEMENTS
from apache_beam.io.gcp.bigquery_tools import JSON_COMPLIANCE_ERROR
@@ -51,6 +54,7 @@
from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultStreamingMatcher
from apache_beam.io.gcp.tests.bigquery_matcher import BigQueryTableMatcher
from apache_beam.options import value_provider
+from apache_beam.options.pipeline_options import GoogleCloudOptions
from apache_beam.options.pipeline_options import StandardOptions
from apache_beam.runners.dataflow.test_dataflow_runner import TestDataflowRunner
from apache_beam.runners.runner import PipelineState
@@ -245,6 +249,105 @@
@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
+class TestJsonToDictCoder(unittest.TestCase):
+
+ @staticmethod
+ def _make_schema(fields):
+ def _fill_schema(fields):
+ for field in fields:
+ table_field = bigquery.TableFieldSchema()
+ table_field.name, table_field.type, nested_fields = field
+ if nested_fields:
+ table_field.fields = list(_fill_schema(nested_fields))
+ yield table_field
+
+ schema = bigquery.TableSchema()
+ schema.fields = list(_fill_schema(fields))
+ return schema
+
+ def test_coder_is_pickable(self):
+ try:
+ schema = self._make_schema([
+ ('record', 'RECORD', [
+ ('float', 'FLOAT', []),
+ ]),
+ ('integer', 'INTEGER', []),
+ ])
+ coder = _JsonToDictCoder(schema)
+ pickler.loads(pickler.dumps(coder))
+ except pickle.PicklingError:
+ self.fail('{} is not pickable'.format(coder.__class__.__name__))
+
+ def test_values_are_converted(self):
+ input_row = b'{"float": "10.5", "string": "abc"}'
+ expected_row = {'float': 10.5, 'string': 'abc'}
+ schema = self._make_schema([
+ ('float', 'FLOAT', []),
+ ('string', 'STRING', [])
+ ])
+ coder = _JsonToDictCoder(schema)
+
+ actual = coder.decode(input_row)
+ self.assertEqual(expected_row, actual)
+
+ def test_null_fields_are_preserved(self):
+ input_row = b'{"float": "10.5"}'
+ expected_row = {'float': 10.5, 'string': None}
+ schema = self._make_schema([
+ ('float', 'FLOAT', []),
+ ('string', 'STRING', [])
+ ])
+ coder = _JsonToDictCoder(schema)
+
+ actual = coder.decode(input_row)
+ self.assertEqual(expected_row, actual)
+
+ def test_record_field_is_properly_converted(self):
+ input_row = b'{"record": {"float": "55.5"}, "integer": 10}'
+ expected_row = {'record': {'float': 55.5}, 'integer': 10}
+ schema = self._make_schema([
+ ('record', 'RECORD', [
+ ('float', 'FLOAT', []),
+ ]),
+ ('integer', 'INTEGER', []),
+ ])
+ coder = _JsonToDictCoder(schema)
+
+ actual = coder.decode(input_row)
+ self.assertEqual(expected_row, actual)
+
+
+@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
+class TestReadFromBigQuery(unittest.TestCase):
+
+ def test_exception_is_raised_when_gcs_location_cannot_be_specified(self):
+ with self.assertRaises(ValueError):
+ p = beam.Pipeline()
+ _ = p | beam.io._ReadFromBigQuery(project='project', dataset='dataset',
+ table='table')
+
+ @mock.patch('apache_beam.io.gcp.bigquery_tools.BigQueryWrapper')
+ def test_fallback_to_temp_location(self, BigQueryWrapper):
+ pipeline_options = beam.pipeline.PipelineOptions()
+ pipeline_options.view_as(GoogleCloudOptions).temp_location = 'gs://bucket'
+ try:
+ p = beam.Pipeline(options=pipeline_options)
+ _ = p | beam.io._ReadFromBigQuery(project='project', dataset='dataset',
+ table='table')
+ except ValueError:
+ self.fail('ValueError was raised unexpectedly')
+
+ def test_gcs_location_validation_works_properly(self):
+ with self.assertRaises(ValueError) as context:
+ p = beam.Pipeline()
+ _ = p | beam.io._ReadFromBigQuery(project='project', dataset='dataset',
+ table='table', validate=True,
+ gcs_location='fs://bad_location')
+ self.assertEqual('Invalid GCS location: fs://bad_location',
+ str(context.exception))
+
+
+@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
class TestBigQuerySink(unittest.TestCase):
def test_table_spec_display_data(self):
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py
index dfd477b..ba69bb0 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py
@@ -69,6 +69,19 @@
JSON_COMPLIANCE_ERROR = 'NAN, INF and -INF values are not JSON compliant.'
+class ExportFileFormat(object):
+ CSV = 'CSV'
+ JSON = 'NEWLINE_DELIMITED_JSON'
+ AVRO = 'AVRO'
+
+
+class ExportCompression(object):
+ GZIP = 'GZIP'
+ DEFLATE = 'DEFLATE'
+ SNAPPY = 'SNAPPY'
+ NONE = 'NONE'
+
+
def default_encoder(obj):
if isinstance(obj, decimal.Decimal):
return str(obj)
@@ -359,7 +372,7 @@
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _start_query_job(self, project_id, query, use_legacy_sql, flatten_results,
- job_id, dry_run=False):
+ job_id, dry_run=False, kms_key=None):
reference = bigquery.JobReference(jobId=job_id, projectId=project_id)
request = bigquery.BigqueryJobsInsertRequest(
projectId=project_id,
@@ -369,13 +382,46 @@
query=bigquery.JobConfigurationQuery(
query=query,
useLegacySql=use_legacy_sql,
- allowLargeResults=True,
- destinationTable=self._get_temp_table(project_id),
- flattenResults=flatten_results)),
+ allowLargeResults=not dry_run,
+ destinationTable=self._get_temp_table(project_id) if not
+ dry_run else None,
+ flattenResults=flatten_results,
+ destinationEncryptionConfiguration=bigquery
+ .EncryptionConfiguration(kmsKeyName=kms_key))),
jobReference=reference))
response = self.client.jobs.Insert(request)
- return response.jobReference.jobId, response.jobReference.location
+ return response
+
+ def wait_for_bq_job(self, job_reference, sleep_duration_sec=5,
+ max_retries=60):
+ """Poll job until it is DONE.
+
+ Args:
+ job_reference: bigquery.JobReference instance.
+ sleep_duration_sec: Specifies the delay in seconds between retries.
+ max_retries: The total number of times to retry. If equals to 0,
+ the function waits forever.
+
+ Raises:
+ `RuntimeError`: If the job is FAILED or the number of retries has been
+ reached.
+ """
+ retry = 0
+ while True:
+ retry += 1
+ job = self.get_job(job_reference.projectId, job_reference.jobId,
+ job_reference.location)
+ logging.info('Job status: %s', job.status.state)
+ if job.status.state == 'DONE' and job.status.errorResult:
+ raise RuntimeError('BigQuery job {} failed. Error Result: {}'.format(
+ job_reference.jobId, job.status.errorResult))
+ elif job.status.state == 'DONE':
+ return True
+ else:
+ time.sleep(sleep_duration_sec)
+ if max_retries != 0 and retry >= max_retries:
+ raise RuntimeError('The maximum number of retries has been reached')
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
@@ -601,6 +647,37 @@
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
+ def perform_extract_job(self, destination, job_id, table_reference,
+ destination_format, include_header=True,
+ compression=ExportCompression.NONE):
+ """Starts a job to export data from BigQuery.
+
+ Returns:
+ bigquery.JobReference with the information about the job that was started.
+ """
+ job_reference = bigquery.JobReference(jobId=job_id,
+ projectId=table_reference.projectId)
+ request = bigquery.BigqueryJobsInsertRequest(
+ projectId=table_reference.projectId,
+ job=bigquery.Job(
+ configuration=bigquery.JobConfiguration(
+ extract=bigquery.JobConfigurationExtract(
+ destinationUris=destination,
+ sourceTable=table_reference,
+ printHeader=include_header,
+ destinationFormat=destination_format,
+ compression=compression,
+ )
+ ),
+ jobReference=job_reference,
+ )
+ )
+ response = self.client.jobs.Insert(request)
+ return response.jobReference
+
+ @retry.with_exponential_backoff(
+ num_retries=MAX_RETRIES,
+ retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def get_or_create_table(
self, project_id, dataset_id, table_id, schema,
create_disposition, write_disposition, additional_create_parameters=None):
@@ -700,10 +777,12 @@
def run_query(self, project_id, query, use_legacy_sql, flatten_results,
dry_run=False):
- job_id, location = self._start_query_job(project_id, query,
- use_legacy_sql, flatten_results,
- job_id=uuid.uuid4().hex,
- dry_run=dry_run)
+ job = self._start_query_job(project_id, query, use_legacy_sql,
+ flatten_results, job_id=uuid.uuid4().hex,
+ dry_run=dry_run)
+ job_id = job.jobReference.jobId
+ location = job.jobReference.location
+
if dry_run:
# If this was a dry run then the fact that we get here means the
# query has no errors. The start_query_job would raise an error otherwise.
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py
index 7b04570..7cfea2f 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py
@@ -196,6 +196,35 @@
]), False, False)
self.assertEqual(new_table, 'table_id')
+ def test_wait_for_job_returns_true_when_job_is_done(self):
+ def make_response(state):
+ m = mock.Mock()
+ m.status.errorResult = None
+ m.status.state = state
+ return m
+
+ client, job_ref = mock.Mock(), mock.Mock()
+ wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client)
+ # Return 'DONE' the second time get_job is called.
+ wrapper.get_job = mock.Mock(side_effect=[make_response('RUNNING'),
+ make_response('DONE')])
+
+ result = wrapper.wait_for_bq_job(job_ref, sleep_duration_sec=0,
+ max_retries=5)
+ self.assertTrue(result)
+
+ def test_wait_for_job_retries_fail(self):
+ client, response, job_ref = mock.Mock(), mock.Mock(), mock.Mock()
+ response.status.state = 'RUNNING'
+ wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client)
+ # Return 'RUNNING' response forever.
+ wrapper.get_job = lambda *args: response
+
+ with self.assertRaises(RuntimeError) as context:
+ wrapper.wait_for_bq_job(job_ref, sleep_duration_sec=0, max_retries=5)
+ self.assertEqual('The maximum number of retries has been reached',
+ str(context.exception))
+
@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
class TestBigQueryReader(unittest.TestCase):
diff --git a/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher_test.py b/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher_test.py
index 6107b35..643fdc4 100644
--- a/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher_test.py
+++ b/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher_test.py
@@ -135,9 +135,9 @@
hc_assert_that(self.mock_presult, self.pubsub_matcher)
self.assertEqual(mock_sub.pull.call_count, 1)
self.assertCountEqual([b'c', b'd'], self.pubsub_matcher.messages)
- self.assertTrue(
- '\nExpected: Expected 1 messages.\n but: Got 2 messages.'
- in str(error.exception.args[0]))
+ self.assertIn(
+ '\nExpected: Expected 1 messages.\n but: Got 2 messages.',
+ str(error.exception.args[0]))
self.assertEqual(mock_sub.pull.call_count, 1)
self.assertEqual(mock_sub.acknowledge.call_count, 1)
@@ -161,9 +161,9 @@
with self.assertRaises(AssertionError) as error:
hc_assert_that(self.mock_presult, self.pubsub_matcher)
self.assertEqual(mock_sub.pull.call_count, 1)
- self.assertTrue(
- '\nExpected: Expected 1 messages.\n but: Got 2 messages.'
- in str(error.exception.args[0]))
+ self.assertIn(
+ '\nExpected: Expected 1 messages.\n but: Got 2 messages.',
+ str(error.exception.args[0]))
def test_message_count_matcher_above_fail(self, mock_get_sub, unused_mock):
self.init_counter_matcher(expected_msg_len=1)
diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index 33218fb..15a5d96 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -169,9 +169,8 @@
By default the options classes will use command line arguments to initialize
the options.
"""
- def __init__(self,
- flags=None, # type: Optional[List[str]]
- **kwargs):
+ def __init__(self, flags=None, **kwargs):
+ # type: (Optional[List[str]], **Any) -> None
"""Initialize an options class.
The initializer will traverse all subclasses, add all their argparse
@@ -186,6 +185,9 @@
**kwargs: Add overrides for arguments passed in flags.
"""
+ # Initializing logging configuration in case the user did not set it up.
+ logging.basicConfig()
+
# self._flags stores a list of not yet parsed arguments, typically,
# command-line flags. This list is shared across different views.
# See: view_as().
@@ -991,6 +993,28 @@
' cluster address. Requires Python 3.6+.')
+class SparkRunnerOptions(PipelineOptions):
+ @classmethod
+ def _add_argparse_args(cls, parser):
+ parser.add_argument('--spark_master_url',
+ default='local[4]',
+ help='Spark master URL (spark://HOST:PORT). '
+ 'Use "local" (single-threaded) or "local[*]" '
+ '(multi-threaded) to start a local cluster for '
+ 'the execution.')
+ parser.add_argument('--spark_job_server_jar',
+ help='Path or URL to a Beam Spark jobserver jar.')
+ parser.add_argument('--spark_submit_uber_jar',
+ default=False,
+ action='store_true',
+ help='Create and upload an uber jar to the Spark REST'
+ ' endpoint, rather than starting up a job server.'
+ ' Requires Python 3.6+.')
+ parser.add_argument('--spark_rest_url',
+ help='URL for the Spark REST endpoint. '
+ 'Only required when using spark_submit_uber_jar.')
+
+
class TestOptions(PipelineOptions):
@classmethod
diff --git a/sdks/python/apache_beam/options/value_provider_test.py b/sdks/python/apache_beam/options/value_provider_test.py
index 8f530a0..7ac8670 100644
--- a/sdks/python/apache_beam/options/value_provider_test.py
+++ b/sdks/python/apache_beam/options/value_provider_test.py
@@ -34,6 +34,11 @@
# <file name acronym>_non_vp_arg<number> for non-value-provider arguments.
# The number will grow per file as tests are added.
class ValueProviderTests(unittest.TestCase):
+ def setUp(self):
+ # Reset runtime options, since the is_accessible assertions require them to
+ # be uninitialized.
+ RuntimeValueProvider.set_runtime_options(None)
+
def test_static_value_provider_keyword_argument(self):
class UserDefinedOptions(PipelineOptions):
@classmethod
@@ -196,9 +201,6 @@
self.assertTrue(isinstance(RuntimeValueProvider.experiments, set))
self.assertTrue('feature_1' in RuntimeValueProvider.experiments)
self.assertTrue('feature_2' in RuntimeValueProvider.experiments)
- # Clean up runtime_options after this test case finish, otherwise, it'll
- # affect other cases since runtime_options is static attr
- RuntimeValueProvider.set_runtime_options(None)
def test_experiments_options_setup(self):
options = PipelineOptions(['--experiments', 'a', '--experiments', 'b,c'])
diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py
index d4c94c4..f42f44a 100644
--- a/sdks/python/apache_beam/pipeline.py
+++ b/sdks/python/apache_beam/pipeline.py
@@ -93,6 +93,7 @@
__all__ = ['Pipeline', 'PTransformOverride']
+
class Pipeline(object):
"""A pipeline object that manages a DAG of
:class:`~apache_beam.pvalue.PValue` s and their
@@ -108,11 +109,18 @@
(e.g. ``input | "label" >> my_tranform``).
"""
- def __init__(self,
- runner=None, # type: Optional[Union[str, PipelineRunner]]
- options=None, # type: Optional[PipelineOptions]
- argv=None # type: Optional[List[str]]
- ):
+ # TODO: BEAM-9001 - set environment ID in all transforms and allow runners to
+ # override.
+ @classmethod
+ def sdk_transforms_with_environment(cls):
+ from apache_beam.runners.portability import fn_api_runner_transforms
+ sets = [fn_api_runner_transforms.PAR_DO_URNS,
+ fn_api_runner_transforms.COMBINE_URNS,
+ frozenset([common_urns.primitives.ASSIGN_WINDOWS.urn])]
+ return frozenset().union(*sets)
+
+ def __init__(self, runner=None, options=None, argv=None):
+ # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None
"""Initialize a pipeline object.
Args:
@@ -133,6 +141,9 @@
~exceptions.ValueError: if either the runner or options argument is not
of the expected type.
"""
+ # Initializing logging configuration in case the user did not set it up.
+ logging.basicConfig()
+
if options is not None:
if isinstance(options, PipelineOptions):
self._options = options
@@ -810,7 +821,8 @@
parent,
transform, # type: ptransform.PTransform
full_label, # type: str
- inputs # type: Optional[Sequence[Union[pvalue.PBegin, pvalue.PCollection]]]
+ inputs, # type: Optional[Sequence[Union[pvalue.PBegin, pvalue.PCollection]]]
+ environment_id=None # type: Optional[str]
):
self.parent = parent
self.transform = transform
@@ -821,9 +833,11 @@
# any interference. This is particularly useful for composite transforms.
self.full_label = full_label
self.inputs = inputs or ()
+
self.side_inputs = () if transform is None else tuple(transform.side_inputs) # type: Tuple[pvalue.AsSideInput, ...]
self.outputs = {} # type: Dict[Union[str, int, None], pvalue.PValue]
self.parts = [] # type: List[AppliedPTransform]
+ self.environment_id = environment_id if environment_id else None # type: Optional[str]
def __repr__(self):
return "%s(%s, %s)" % (self.__class__.__name__, self.full_label,
@@ -972,15 +986,23 @@
return transform.to_runner_api(context, has_parts=bool(self.parts))
# Iterate over inputs and outputs by sorted key order, so that ids are
# consistently generated for multiple runs of the same pipeline.
+ transform_spec = transform_to_runner_api(self.transform, context)
+ environment_id = self.environment_id
+ transform_urn = transform_spec.urn if transform_spec else None
+ if (not environment_id and transform_urn and
+ (transform_urn in Pipeline.sdk_transforms_with_environment())):
+ environment_id = context.default_environment_id()
+
return beam_runner_api_pb2.PTransform(
unique_name=self.full_label,
- spec=transform_to_runner_api(self.transform, context),
+ spec=transform_spec,
subtransforms=[context.transforms.get_id(part, label=part.full_label)
for part in self.parts],
inputs={tag: context.pcollections.get_id(pc)
for tag, pc in sorted(self.named_inputs().items())},
outputs={str(tag): context.pcollections.get_id(out)
for tag, out in sorted(self.named_outputs().items())},
+ environment_id=environment_id,
# TODO(BEAM-366): Add display_data.
display_data=None)
@@ -1005,7 +1027,8 @@
parent=None,
transform=ptransform.PTransform.from_runner_api(proto.spec, context),
full_label=proto.unique_name,
- inputs=main_inputs)
+ inputs=main_inputs,
+ environment_id=proto.environment_id)
if result.transform and result.transform.side_inputs:
for si, pcoll in zip(result.transform.side_inputs, side_inputs):
si.pvalue = pcoll
diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py
index 1aa8913..5fa8ece 100644
--- a/sdks/python/apache_beam/pvalue.py
+++ b/sdks/python/apache_beam/pvalue.py
@@ -431,27 +431,23 @@
return beam_runner_api_pb2.SideInput(
access_pattern=beam_runner_api_pb2.FunctionSpec(
urn=self.access_pattern),
- view_fn=beam_runner_api_pb2.SdkFunctionSpec(
- environment_id=context.default_environment_id(),
- spec=beam_runner_api_pb2.FunctionSpec(
- urn=python_urns.PICKLED_VIEWFN,
- payload=pickler.dumps(self.view_fn))),
- window_mapping_fn=beam_runner_api_pb2.SdkFunctionSpec(
- environment_id=context.default_environment_id(),
- spec=beam_runner_api_pb2.FunctionSpec(
- urn=python_urns.PICKLED_WINDOW_MAPPING_FN,
- payload=pickler.dumps(self.window_mapping_fn))))
+ view_fn=beam_runner_api_pb2.FunctionSpec(
+ urn=python_urns.PICKLED_VIEWFN,
+ payload=pickler.dumps(self.view_fn)),
+ window_mapping_fn=beam_runner_api_pb2.FunctionSpec(
+ urn=python_urns.PICKLED_WINDOW_MAPPING_FN,
+ payload=pickler.dumps(self.window_mapping_fn)))
@staticmethod
def from_runner_api(proto, unused_context):
# type: (beam_runner_api_pb2.SideInput, PipelineContext) -> SideInputData
- assert proto.view_fn.spec.urn == python_urns.PICKLED_VIEWFN
- assert (proto.window_mapping_fn.spec.urn ==
+ assert proto.view_fn.urn == python_urns.PICKLED_VIEWFN
+ assert (proto.window_mapping_fn.urn ==
python_urns.PICKLED_WINDOW_MAPPING_FN)
return SideInputData(
proto.access_pattern.urn,
- pickler.loads(proto.window_mapping_fn.spec.payload),
- pickler.loads(proto.view_fn.spec.payload))
+ pickler.loads(proto.window_mapping_fn.payload),
+ pickler.loads(proto.view_fn.payload))
class AsSingleton(AsSideInput):
diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
index 0254366..6786760 100644
--- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
+++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
@@ -1238,7 +1238,7 @@
# TestStream source doesn't do any decoding of elements,
# so we won't set test_stream_payload.coder_id.
output_coder = transform._infer_output_coder() # pylint: disable=protected-access
- for event in transform.events:
+ for event in transform._events:
new_event = test_stream_payload.events.add()
if isinstance(event, ElementEvent):
for tv in event.timestamped_values:
diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py
index 111259d..621e8fa 100644
--- a/sdks/python/apache_beam/runners/dataflow/internal/names.py
+++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py
@@ -38,10 +38,10 @@
# Update this version to the next version whenever there is a change that will
# require changes to legacy Dataflow worker execution environment.
-BEAM_CONTAINER_VERSION = 'beam-master-20191112'
+BEAM_CONTAINER_VERSION = 'beam-master-20191220'
# Update this version to the next version whenever there is a change that
# requires changes to SDK harness container or SDK harness launcher.
-BEAM_FNAPI_CONTAINER_VERSION = 'beam-master-20191112'
+BEAM_FNAPI_CONTAINER_VERSION = 'beam-master-20191220'
# TODO(BEAM-5939): Remove these shared names once Dataflow worker is updated.
PICKLED_MAIN_SESSION_FILE = 'pickled_main_session'
diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py
index 4a66341..ef81a27 100644
--- a/sdks/python/apache_beam/runners/direct/direct_runner.py
+++ b/sdks/python/apache_beam/runners/direct/direct_runner.py
@@ -75,12 +75,53 @@
def is_fnapi_compatible(self):
return BundleBasedDirectRunner.is_fnapi_compatible()
+ def apply_TestStream(self, transform, pbegin, options):
+ """Expands the TestStream into the DirectRunner implementation.
+
+ Takes the TestStream transform and creates a _TestStream -> multiplexer ->
+ _WatermarkController.
+ """
+
+ from apache_beam.runners.direct.test_stream_impl import _TestStream
+ from apache_beam.runners.direct.test_stream_impl import _WatermarkController
+ from apache_beam import pvalue
+ assert isinstance(pbegin, pvalue.PBegin)
+
+ # If there is only one tag there is no need to add the multiplexer.
+ if len(transform.output_tags) == 1:
+ return (pbegin
+ | _TestStream(transform.output_tags, events=transform._events)
+ | _WatermarkController())
+
+ # This multiplexing the multiple output PCollections.
+ def mux(event):
+ if event.tag:
+ yield pvalue.TaggedOutput(event.tag, event)
+ else:
+ yield event
+ mux_output = (pbegin
+ | _TestStream(transform.output_tags, events=transform._events)
+ | 'TestStream Multiplexer' >> beam.ParDo(mux).with_outputs())
+
+ # Apply a way to control the watermark per output. It is necessary to
+ # have an individual _WatermarkController per PCollection because the
+ # calculation of the input watermark of a transform is based on the event
+ # timestamp of the elements flowing through it. Meaning, it is impossible
+ # to control the output watermarks of the individual PCollections solely
+ # on the event timestamps.
+ outputs = {}
+ for tag in transform.output_tags:
+ label = '_WatermarkController[{}]'.format(tag)
+ outputs[tag] = (mux_output[tag] | label >> _WatermarkController())
+
+ return outputs
+
def run_pipeline(self, pipeline, options):
from apache_beam.pipeline import PipelineVisitor
from apache_beam.runners.dataflow.native_io.iobase import NativeSource
from apache_beam.runners.dataflow.native_io.iobase import _NativeWrite
- from apache_beam.testing.test_stream import _TestStream
+ from apache_beam.runners.direct.test_stream_impl import _TestStream
class _FnApiRunnerSupportVisitor(PipelineVisitor):
"""Visitor determining if a Pipeline can be run on the FnApiRunner."""
@@ -360,7 +401,7 @@
from apache_beam.runners.direct.executor import Executor
from apache_beam.runners.direct.transform_evaluator import \
TransformEvaluatorRegistry
- from apache_beam.testing.test_stream import _TestStream
+ from apache_beam.runners.direct.test_stream_impl import _TestStream
# Performing configured PTransform overrides.
pipeline.replace_all(_get_transform_overrides(options))
diff --git a/sdks/python/apache_beam/runners/direct/test_stream_impl.py b/sdks/python/apache_beam/runners/direct/test_stream_impl.py
new file mode 100644
index 0000000..aa1405d
--- /dev/null
+++ b/sdks/python/apache_beam/runners/direct/test_stream_impl.py
@@ -0,0 +1,173 @@
+#
+# 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.
+#
+
+"""The TestStream implementation for the DirectRunner
+
+The DirectRunner implements TestStream as the _TestStream class which is used
+to store the events in memory, the _WatermarkController which is used to set the
+watermark and emit events, and the multiplexer which sends events to the correct
+tagged PCollection.
+"""
+
+from __future__ import absolute_import
+
+from apache_beam import coders
+from apache_beam import pvalue
+from apache_beam.testing.test_stream import WatermarkEvent
+from apache_beam.transforms import PTransform
+from apache_beam.transforms import core
+from apache_beam.transforms import window
+from apache_beam.utils import timestamp
+
+
+class _WatermarkController(PTransform):
+ """A runner-overridable PTransform Primitive to control the watermark.
+
+ Expected implementation behavior:
+ - If the instance recieves a WatermarkEvent, it sets its output watermark to
+ the specified value then drops the event.
+ - If the instance receives an ElementEvent, it emits all specified elements
+ to the Global Window with the event time set to the element's timestamp.
+ """
+ def get_windowing(self, _):
+ return core.Windowing(window.GlobalWindows())
+
+ def expand(self, pcoll):
+ return pvalue.PCollection.from_(pcoll)
+
+
+class _TestStream(PTransform):
+ """Test stream that generates events on an unbounded PCollection of elements.
+
+ Each event emits elements, advances the watermark or advances the processing
+ time. After all of the specified elements are emitted, ceases to produce
+ output.
+
+ Expected implementation behavior:
+ - If the instance receives a WatermarkEvent with the WATERMARK_CONTROL_TAG
+ then the instance sets its own watermark hold at the specified value and
+ drops the event.
+ - If the instance receives any other WatermarkEvent or ElementEvent, it
+ passes it to the consumer.
+ """
+
+ # This tag is used on WatermarkEvents to control the watermark at the root
+ # TestStream.
+ WATERMARK_CONTROL_TAG = '_TestStream_Watermark'
+
+ def __init__(self, output_tags, coder=coders.FastPrimitivesCoder(),
+ events=None):
+ assert coder is not None
+ self.coder = coder
+ self._raw_events = events
+ self._events = self._add_watermark_advancements(output_tags, events)
+
+ def _watermark_starts(self, output_tags):
+ """Sentinel values to hold the watermark of outputs to -inf.
+
+ The output watermarks of the output PCollections (fake unbounded sources) in
+ a TestStream are controlled by watermark holds. This sets the hold of each
+ output PCollection so that the individual holds can be controlled by the
+ given events.
+ """
+ return [WatermarkEvent(timestamp.MIN_TIMESTAMP, tag) for tag in output_tags]
+
+ def _watermark_stops(self, output_tags):
+ """Sentinel values to close the watermark of outputs."""
+ return [WatermarkEvent(timestamp.MAX_TIMESTAMP, tag) for tag in output_tags]
+
+ def _test_stream_start(self):
+ """Sentinel value to move the watermark hold of the TestStream to +inf.
+
+ This sets a hold to +inf such that the individual holds of the output
+ PCollections are allowed to modify their individial output watermarks with
+ their holds. This is because the calculation of the output watermark is a
+ min over all input watermarks.
+ """
+ return [WatermarkEvent(timestamp.MAX_TIMESTAMP - timestamp.TIME_GRANULARITY,
+ _TestStream.WATERMARK_CONTROL_TAG)]
+
+ def _test_stream_stop(self):
+ """Sentinel value to close the watermark of the TestStream."""
+ return [WatermarkEvent(timestamp.MAX_TIMESTAMP,
+ _TestStream.WATERMARK_CONTROL_TAG)]
+
+ def _test_stream_init(self):
+ """Sentinel value to hold the watermark of the TestStream to -inf.
+
+ This sets a hold to ensure that the output watermarks of the output
+ PCollections do not advance to +inf before their watermark holds are set.
+ """
+ return [WatermarkEvent(timestamp.MIN_TIMESTAMP,
+ _TestStream.WATERMARK_CONTROL_TAG)]
+
+ def _set_up(self, output_tags):
+ return (self._test_stream_init()
+ + self._watermark_starts(output_tags)
+ + self._test_stream_start())
+
+ def _tear_down(self, output_tags):
+ return self._watermark_stops(output_tags) + self._test_stream_stop()
+
+ def _add_watermark_advancements(self, output_tags, events):
+ """Adds watermark advancements to the given events.
+
+ The following watermark advancements can be done on the runner side.
+ However, it makes the logic on the runner side much more complicated than
+ it needs to be.
+
+ In order for watermarks to be properly advanced in a TestStream, a specific
+ sequence of watermark holds must be sent:
+
+ 1. Hold the root watermark at -inf (this prevents the pipeline from
+ immediately returning).
+ 2. Hold the watermarks at the WatermarkControllerss at -inf (this prevents
+ the pipeline from immediately returning).
+ 3. Advance the root watermark to +inf - 1 (this allows the downstream
+ WatermarkControllers to control their watermarks via holds).
+ 4. Advance watermarks as normal.
+ 5. Advance WatermarkController watermarks to +inf
+ 6. Advance root watermark to +inf.
+ """
+ if not events:
+ return []
+
+ return self._set_up(output_tags) + events + self._tear_down(output_tags)
+
+ def get_windowing(self, unused_inputs):
+ return core.Windowing(window.GlobalWindows())
+
+ def expand(self, pcoll):
+ return pvalue.PCollection(pcoll.pipeline, is_bounded=False)
+
+ def _infer_output_coder(self, input_type=None, input_coder=None):
+ return self.coder
+
+ def _events_from_script(self, index):
+ yield self._events[index]
+
+ def events(self, index):
+ return self._events_from_script(index)
+
+ def begin(self):
+ return 0
+
+ def end(self, index):
+ return index >= len(self._events)
+
+ def next(self, index):
+ return index + 1
diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
index 0d7ddaa..ff73aab 100644
--- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py
+++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
@@ -50,14 +50,14 @@
from apache_beam.runners.direct.sdf_direct_runner import ProcessElements
from apache_beam.runners.direct.sdf_direct_runner import ProcessFn
from apache_beam.runners.direct.sdf_direct_runner import SDFProcessElementInvoker
+from apache_beam.runners.direct.test_stream_impl import _TestStream
+from apache_beam.runners.direct.test_stream_impl import _WatermarkController
from apache_beam.runners.direct.util import KeyedWorkItem
from apache_beam.runners.direct.util import TransformResult
from apache_beam.runners.direct.watermark_manager import WatermarkManager
from apache_beam.testing.test_stream import ElementEvent
from apache_beam.testing.test_stream import ProcessingTimeEvent
from apache_beam.testing.test_stream import WatermarkEvent
-from apache_beam.testing.test_stream import _TestStream
-from apache_beam.testing.test_stream import _WatermarkController
from apache_beam.transforms import core
from apache_beam.transforms.trigger import InMemoryUnmergedState
from apache_beam.transforms.trigger import TimeDomain
diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job.py b/sdks/python/apache_beam/runners/interactive/background_caching_job.py
new file mode 100644
index 0000000..f157f9b
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/background_caching_job.py
@@ -0,0 +1,157 @@
+#
+# 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.
+#
+
+"""Module to build and run background caching job.
+
+For internal use only; no backwards-compatibility guarantees.
+
+A background caching job is a job that caches events for all unbounded sources
+of a given pipeline. With Interactive Beam, one such job is started when a
+pipeline run happens (which produces a main job in contrast to the background
+caching job) and meets the following conditions:
+
+ #. The pipeline contains unbounded sources.
+ #. No such background job is running.
+ #. No such background job has completed successfully and the cached events are
+ still valid (invalidated when unbounded sources change in the pipeline).
+
+Once started, the background caching job runs asynchronously until it hits some
+cache size limit. Meanwhile, the main job and future main jobs from the pipeline
+will run using the deterministic replay-able cached events until they are
+invalidated.
+"""
+
+from __future__ import absolute_import
+
+import apache_beam as beam
+from apache_beam import runners
+from apache_beam.runners.interactive import interactive_environment as ie
+
+
+def attempt_to_run_background_caching_job(runner, user_pipeline, options=None):
+ """Attempts to run a background caching job for a user-defined pipeline.
+
+ The pipeline result is automatically tracked by Interactive Beam in case
+ future cancellation/cleanup is needed.
+ """
+ if is_background_caching_job_needed(user_pipeline):
+ # Cancel non-terminal jobs if there is any before starting a new one.
+ attempt_to_cancel_background_caching_job(user_pipeline)
+ # Evict all caches if there is any.
+ ie.current_env().cleanup()
+ # TODO(BEAM-8335): refactor background caching job logic from
+ # pipeline_instrument module to this module and aggregate tests.
+ from apache_beam.runners.interactive import pipeline_instrument as instr
+ runner_pipeline = beam.pipeline.Pipeline.from_runner_api(
+ user_pipeline.to_runner_api(use_fake_coders=True),
+ runner,
+ options)
+ background_caching_job_result = beam.pipeline.Pipeline.from_runner_api(
+ instr.pin(runner_pipeline).background_caching_pipeline_proto(),
+ runner,
+ options).run()
+ ie.current_env().set_pipeline_result(user_pipeline,
+ background_caching_job_result,
+ is_main_job=False)
+
+
+def is_background_caching_job_needed(user_pipeline):
+ """Determines if a background caching job needs to be started."""
+ background_caching_job_result = ie.current_env().pipeline_result(
+ user_pipeline, is_main_job=False)
+ # Checks if the pipeline contains any source that needs to be cached.
+ return (has_source_to_cache(user_pipeline) and
+ # Checks if it's the first time running a job from the pipeline.
+ (not background_caching_job_result or
+ # Or checks if there is no previous job.
+ background_caching_job_result.state not in (
+ # DONE means a previous job has completed successfully and the
+ # cached events are still valid.
+ runners.runner.PipelineState.DONE,
+ # RUNNING means a previous job has been started and is still
+ # running.
+ runners.runner.PipelineState.RUNNING) or
+ # Or checks if we can invalidate the previous job.
+ is_source_to_cache_changed(user_pipeline)))
+
+
+def has_source_to_cache(user_pipeline):
+ """Determines if a user-defined pipeline contains any source that need to be
+ cached."""
+ from apache_beam.runners.interactive import pipeline_instrument as instr
+ # TODO(BEAM-8335): we temporarily only cache replaceable unbounded sources.
+ # Add logic for other cacheable sources here when they are available.
+ return instr.has_unbounded_sources(user_pipeline)
+
+
+def attempt_to_cancel_background_caching_job(user_pipeline):
+ """Attempts to cancel background caching job for a user-defined pipeline.
+
+ If no background caching job needs to be cancelled, NOOP. Otherwise, cancel
+ such job.
+ """
+ background_caching_job_result = ie.current_env().pipeline_result(
+ user_pipeline, is_main_job=False)
+ if (background_caching_job_result and
+ not ie.current_env().is_terminated(user_pipeline, is_main_job=False)):
+ background_caching_job_result.cancel()
+
+
+def is_source_to_cache_changed(user_pipeline):
+ """Determines if there is any change in the sources that need to be cached
+ used by the user-defined pipeline.
+
+ Due to the expensiveness of computations and for the simplicity of usage, this
+ function is not idempotent because Interactive Beam automatically discards
+ previously tracked signature of transforms and tracks the current signature of
+ transforms for the user-defined pipeline if there is any change.
+
+ When it's True, there is addition/deletion/mutation of source transforms that
+ requires a new background caching job.
+ """
+ # By default gets empty set if the user_pipeline is first time seen because
+ # we can treat it as adding transforms.
+ recorded_signature = ie.current_env().get_cached_source_signature(
+ user_pipeline)
+ current_signature = extract_source_to_cache_signature(user_pipeline)
+ is_changed = not current_signature.issubset(recorded_signature)
+ # The computation of extract_unbounded_source_signature is expensive, track on
+ # change by default.
+ if is_changed:
+ ie.current_env().set_cached_source_signature(user_pipeline,
+ current_signature)
+ return is_changed
+
+
+def extract_source_to_cache_signature(user_pipeline):
+ """Extracts a set of signature for sources that need to be cached in the
+ user-defined pipeline.
+
+ A signature is a str representation of urn and payload of a source.
+ """
+ from apache_beam.runners.interactive import pipeline_instrument as instr
+ # TODO(BEAM-8335): we temporarily only cache replaceable unbounded sources.
+ # Add logic for other cacheable sources here when they are available.
+ unbounded_sources_as_applied_transforms = instr.unbounded_sources(
+ user_pipeline)
+ unbounded_sources_as_ptransforms = set(
+ map(lambda x: x.transform, unbounded_sources_as_applied_transforms))
+ context, _ = user_pipeline.to_runner_api(
+ return_context=True, use_fake_coders=True)
+ signature = set(map(lambda transform: str(transform.to_runner_api(context)),
+ unbounded_sources_as_ptransforms))
+ return signature
diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
new file mode 100644
index 0000000..19e4889
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py
@@ -0,0 +1,248 @@
+#
+# 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.
+#
+
+"""Tests for apache_beam.runners.interactive.background_caching_job."""
+from __future__ import absolute_import
+
+import unittest
+
+import apache_beam as beam
+from apache_beam.pipeline import PipelineVisitor
+from apache_beam.runners import runner
+from apache_beam.runners.interactive import background_caching_job as bcj
+from apache_beam.runners.interactive import interactive_beam as ib
+from apache_beam.runners.interactive import interactive_environment as ie
+from apache_beam.runners.interactive import interactive_runner
+from apache_beam.runners.interactive.testing.mock_ipython import mock_get_ipython
+from apache_beam.testing.test_stream import TestStream
+from apache_beam.transforms.window import TimestampedValue
+
+# TODO(BEAM-8288): clean up the work-around of nose tests using Python2 without
+# unittest.mock module.
+try:
+ from unittest.mock import patch
+except ImportError:
+ from mock import patch
+
+_FOO_PUBSUB_SUB = 'projects/test-project/subscriptions/foo'
+_BAR_PUBSUB_SUB = 'projects/test-project/subscriptions/bar'
+
+
+def _build_a_test_stream_pipeline():
+ test_stream = (TestStream()
+ .advance_watermark_to(0)
+ .add_elements([TimestampedValue('a', 1)])
+ .advance_processing_time(5)
+ .advance_watermark_to_infinity())
+ p = beam.Pipeline(runner=interactive_runner.InteractiveRunner())
+ events = p | test_stream # pylint: disable=possibly-unused-variable
+ ib.watch(locals())
+ return p
+
+
+def _build_an_empty_stream_pipeline():
+ from apache_beam.options.pipeline_options import PipelineOptions
+ from apache_beam.options.pipeline_options import StandardOptions
+ pipeline_options = PipelineOptions()
+ pipeline_options.view_as(StandardOptions).streaming = True
+ p = beam.Pipeline(interactive_runner.InteractiveRunner(),
+ options=pipeline_options)
+ ib.watch({'pipeline': p})
+ return p
+
+
+@unittest.skipIf(not ie.current_env().is_interactive_ready,
+ '[interactive] dependency is not installed.')
+class BackgroundCachingJobTest(unittest.TestCase):
+
+ def tearDown(self):
+ ie.new_env()
+
+ # TODO(BEAM-8335): remove the patches when there are appropriate test sources
+ # that meet the boundedness checks.
+ @patch('apache_beam.runners.interactive.pipeline_instrument'
+ '.has_unbounded_sources', lambda x: True)
+ def test_background_caching_job_starts_when_none_such_job_exists(self):
+ p = _build_a_test_stream_pipeline()
+ p.run()
+ self.assertIsNotNone(
+ ie.current_env().pipeline_result(p, is_main_job=False))
+
+ @patch('apache_beam.runners.interactive.pipeline_instrument'
+ '.has_unbounded_sources', lambda x: False)
+ def test_background_caching_job_not_start_for_batch_pipeline(self):
+ p = _build_a_test_stream_pipeline()
+ p.run()
+ self.assertIsNone(
+ ie.current_env().pipeline_result(p, is_main_job=False))
+
+ @patch('apache_beam.runners.interactive.pipeline_instrument'
+ '.has_unbounded_sources', lambda x: True)
+ def test_background_caching_job_not_start_when_such_job_exists(self):
+ p = _build_a_test_stream_pipeline()
+ a_running_result = runner.PipelineResult(runner.PipelineState.RUNNING)
+ ie.current_env().set_pipeline_result(p, a_running_result, is_main_job=False)
+ main_job_result = p.run()
+ # No background caching job is started so result is still the running one.
+ self.assertIs(a_running_result,
+ ie.current_env().pipeline_result(p, is_main_job=False))
+ # A new main job is started so result of the main job is set.
+ self.assertIs(main_job_result,
+ ie.current_env().pipeline_result(p))
+
+ @patch('apache_beam.runners.interactive.pipeline_instrument'
+ '.has_unbounded_sources', lambda x: True)
+ def test_background_caching_job_not_start_when_such_job_is_done(self):
+ p = _build_a_test_stream_pipeline()
+ a_done_result = runner.PipelineResult(runner.PipelineState.DONE)
+ ie.current_env().set_pipeline_result(p, a_done_result, is_main_job=False)
+ main_job_result = p.run()
+ # No background caching job is started so result is still the running one.
+ self.assertIs(a_done_result,
+ ie.current_env().pipeline_result(p, is_main_job=False))
+ # A new main job is started so result of the main job is set.
+ self.assertIs(main_job_result,
+ ie.current_env().pipeline_result(p))
+
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
+ def test_source_to_cache_changed_when_pipeline_is_first_time_seen(self, cell):
+ with cell: # Cell 1
+ pipeline = _build_an_empty_stream_pipeline()
+
+ with cell: # Cell 2
+ read_foo = pipeline | 'Read' >> beam.io.ReadFromPubSub(
+ subscription=_FOO_PUBSUB_SUB)
+ ib.watch({'read_foo': read_foo})
+
+ self.assertTrue(bcj.is_source_to_cache_changed(pipeline))
+
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
+ def test_source_to_cache_changed_when_new_source_is_added(self, cell):
+ with cell: # Cell 1
+ pipeline = _build_an_empty_stream_pipeline()
+ read_foo = pipeline | 'Read' >> beam.io.ReadFromPubSub(
+ subscription=_FOO_PUBSUB_SUB)
+ ib.watch({'read_foo': read_foo})
+
+ # Sets the signature for current pipeline state.
+ ie.current_env().set_cached_source_signature(
+ pipeline, bcj.extract_source_to_cache_signature(pipeline))
+
+ with cell: # Cell 2
+ read_bar = pipeline | 'Read' >> beam.io.ReadFromPubSub(
+ subscription=_BAR_PUBSUB_SUB)
+ ib.watch({'read_bar': read_bar})
+
+ self.assertTrue(bcj.is_source_to_cache_changed(pipeline))
+
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
+ def test_source_to_cache_changed_when_source_is_altered(self, cell):
+ with cell: # Cell 1
+ pipeline = _build_an_empty_stream_pipeline()
+ transform = beam.io.ReadFromPubSub(subscription=_FOO_PUBSUB_SUB)
+ read_foo = pipeline | 'Read' >> transform
+ ib.watch({'read_foo': read_foo})
+
+ # Sets the signature for current pipeline state.
+ ie.current_env().set_cached_source_signature(
+ pipeline, bcj.extract_source_to_cache_signature(pipeline))
+
+ with cell: # Cell 2
+ from apache_beam.io.gcp.pubsub import _PubSubSource
+ # Alter the transform.
+ transform._source = _PubSubSource(subscription=_BAR_PUBSUB_SUB)
+
+ self.assertTrue(bcj.is_source_to_cache_changed(pipeline))
+
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
+ def test_source_to_cache_not_changed_for_same_source(self, cell):
+ with cell: # Cell 1
+ pipeline = _build_an_empty_stream_pipeline()
+ transform = beam.io.ReadFromPubSub(subscription=_FOO_PUBSUB_SUB)
+
+ with cell: # Cell 2
+ read_foo_1 = pipeline | 'Read' >> transform
+ ib.watch({'read_foo_1': read_foo_1})
+
+ # Sets the signature for current pipeline state.
+ ie.current_env().set_cached_source_signature(
+ pipeline, bcj.extract_source_to_cache_signature(pipeline))
+
+ with cell: # Cell 3
+ # Apply exactly the same transform and the same instance.
+ read_foo_2 = pipeline | 'Read' >> transform
+ ib.watch({'read_foo_2': read_foo_2})
+
+ self.assertFalse(bcj.is_source_to_cache_changed(pipeline))
+
+ with cell: # Cell 4
+ # Apply the same transform but represented in a different instance.
+ # The signature representing the urn and payload is still the same, so it
+ # is not treated as a new unbounded source.
+ read_foo_3 = pipeline | 'Read' >> beam.io.ReadFromPubSub(
+ subscription=_FOO_PUBSUB_SUB)
+ ib.watch({'read_foo_3': read_foo_3})
+
+ self.assertFalse(bcj.is_source_to_cache_changed(pipeline))
+
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
+ def test_source_to_cache_not_changed_when_source_is_removed(self, cell):
+ with cell: # Cell 1
+ pipeline = _build_an_empty_stream_pipeline()
+ foo_transform = beam.io.ReadFromPubSub(subscription=_FOO_PUBSUB_SUB)
+ bar_transform = beam.io.ReadFromPubSub(subscription=_BAR_PUBSUB_SUB)
+
+ with cell: # Cell 2
+ read_foo = pipeline | 'Read' >> foo_transform
+ ib.watch({'read_foo': read_foo})
+
+ signature_with_only_foo = bcj.extract_source_to_cache_signature(pipeline)
+
+ with cell: # Cell 3
+ read_bar = pipeline | 'Read' >> bar_transform
+ ib.watch({'read_bar': read_bar})
+
+ self.assertTrue(bcj.is_source_to_cache_changed(pipeline))
+ signature_with_foo_bar = ie.current_env().get_cached_source_signature(
+ pipeline)
+ self.assertNotEqual(signature_with_only_foo, signature_with_foo_bar)
+
+ class BarPruneVisitor(PipelineVisitor):
+
+ def enter_composite_transform(self, transform_node):
+ pruned_parts = list(transform_node.parts)
+ for part in transform_node.parts:
+ if part.transform is bar_transform:
+ pruned_parts.remove(part)
+ transform_node.parts = tuple(pruned_parts)
+ self.visit_transform(transform_node)
+
+ def visit_transform(self, transform_node):
+ if transform_node.transform is bar_transform:
+ transform_node.parent = None
+
+ v = BarPruneVisitor()
+ pipeline.visit(v)
+
+ signature_after_pruning_bar = bcj.extract_source_to_cache_signature(
+ pipeline)
+ self.assertEqual(signature_with_only_foo, signature_after_pruning_bar)
+ self.assertFalse(bcj.is_source_to_cache_changed(pipeline))
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py
index 2da7724..b2d330e 100644
--- a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py
+++ b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py
@@ -19,7 +19,6 @@
from __future__ import absolute_import
import sys
-import time
import unittest
import apache_beam as beam
@@ -32,9 +31,9 @@
# TODO(BEAM-8288): clean up the work-around of nose tests using Python2 without
# unittest.mock module.
try:
- from unittest.mock import patch
+ from unittest.mock import patch, ANY
except ImportError:
- from mock import patch
+ from mock import patch, ANY
try:
import timeloop
@@ -88,44 +87,46 @@
h.stop()
@patch('apache_beam.runners.interactive.display.pcoll_visualization'
- '.PCollectionVisualization.display_facets')
- def test_dynamic_plotting_update_same_display(self,
- mocked_display_facets):
- fake_pipeline_result = runner.PipelineResult(runner.PipelineState.RUNNING)
- ie.current_env().set_pipeline_result(self._p, fake_pipeline_result)
- # Starts async dynamic plotting that never ends in this test.
- h = pv.visualize(self._pcoll, dynamic_plotting_interval=0.001)
- # Blocking so the above async task can execute some iterations.
- time.sleep(1)
- # The first iteration doesn't provide updating_pv to display_facets.
- _, first_kwargs = mocked_display_facets.call_args_list[0]
- self.assertEqual(first_kwargs, {})
- # The following iterations use the same updating_pv to display_facets and so
- # on.
- _, second_kwargs = mocked_display_facets.call_args_list[1]
- updating_pv = second_kwargs['updating_pv']
- for call in mocked_display_facets.call_args_list[2:]:
- _, kwargs = call
- self.assertIs(kwargs['updating_pv'], updating_pv)
- h.stop()
+ '.PCollectionVisualization._display_dive')
+ @patch('apache_beam.runners.interactive.display.pcoll_visualization'
+ '.PCollectionVisualization._display_overview')
+ @patch('apache_beam.runners.interactive.display.pcoll_visualization'
+ '.PCollectionVisualization._display_dataframe')
+ def test_dynamic_plotting_updates_same_display(self,
+ mocked_display_dataframe,
+ mocked_display_overview,
+ mocked_display_dive):
+ original_pcollection_visualization = pv.PCollectionVisualization(
+ self._pcoll)
+ # Dynamic plotting always creates a new PCollectionVisualization.
+ new_pcollection_visualization = pv.PCollectionVisualization(self._pcoll)
+ # The display uses ANY data the moment display is invoked, and updates
+ # web elements with ids fetched from the given updating_pv.
+ new_pcollection_visualization.display_facets(
+ updating_pv=original_pcollection_visualization)
+ mocked_display_dataframe.assert_called_once_with(
+ ANY, original_pcollection_visualization._df_display_id)
+ mocked_display_overview.assert_called_once_with(
+ ANY, original_pcollection_visualization._overview_display_id)
+ mocked_display_dive.assert_called_once_with(
+ ANY, original_pcollection_visualization._dive_display_id)
- @patch('timeloop.Timeloop.stop')
- def test_auto_stop_dynamic_plotting_when_job_is_terminated(
- self,
- mocked_timeloop):
+ def test_auto_stop_dynamic_plotting_when_job_is_terminated(self):
fake_pipeline_result = runner.PipelineResult(runner.PipelineState.RUNNING)
- ie.current_env().set_pipeline_result(self._p, fake_pipeline_result)
- # Starts non-stopping async dynamic plotting until the job is terminated.
- pv.visualize(self._pcoll, dynamic_plotting_interval=0.001)
- # Blocking so the above async task can execute some iterations.
- time.sleep(1)
- mocked_timeloop.assert_not_called()
+ ie.current_env().set_pipeline_result(
+ self._p,
+ fake_pipeline_result,
+ is_main_job=True)
+ # When job is running, the dynamic plotting will not be stopped.
+ self.assertFalse(ie.current_env().is_terminated(self._p))
+
fake_pipeline_result = runner.PipelineResult(runner.PipelineState.DONE)
- ie.current_env().set_pipeline_result(self._p, fake_pipeline_result)
- # Blocking so the above async task can execute some iterations.
- time.sleep(1)
- # "assert_called" is new in Python 3.6.
- mocked_timeloop.assert_called()
+ ie.current_env().set_pipeline_result(
+ self._p,
+ fake_pipeline_result,
+ is_main_job=True)
+ # When job is done, the dynamic plotting will be stopped.
+ self.assertTrue(ie.current_env().is_terminated(self._p))
@patch('pandas.DataFrame.sample')
def test_display_plain_text_when_kernel_has_no_frontend(self,
diff --git a/sdks/python/apache_beam/runners/interactive/display/pipeline_graph_test.py b/sdks/python/apache_beam/runners/interactive/display/pipeline_graph_test.py
index e73dbd6..7e4aa13 100644
--- a/sdks/python/apache_beam/runners/interactive/display/pipeline_graph_test.py
+++ b/sdks/python/apache_beam/runners/interactive/display/pipeline_graph_test.py
@@ -91,24 +91,24 @@
'}\n'),
pipeline_graph.PipelineGraph(p).get_dot())
- @patch('IPython.get_ipython', mock_get_ipython)
- def test_get_dot_within_notebook(self):
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
+ def test_get_dot_within_notebook(self, cell):
# Assume a mocked ipython kernel and notebook frontend have been set up.
ie.current_env()._is_in_ipython = True
ie.current_env()._is_in_notebook = True
- with mock_get_ipython(): # Cell 1
+ with cell: # Cell 1
p = beam.Pipeline(ir.InteractiveRunner())
# Immediately track this local pipeline so that ipython prompts when
# applying transforms will be tracked and used for labels.
ib.watch(locals())
- with mock_get_ipython(): # Cell 2
+ with cell: # Cell 2
init_pcoll = p | 'Init' >> beam.Create(range(10))
- with mock_get_ipython(): # Cell 3
+ with cell: # Cell 3
squares = init_pcoll | 'Square' >> beam.Map(lambda x: x * x)
- with mock_get_ipython(): # Cell 4
+ with cell: # Cell 4
cubes = init_pcoll | 'Cube' >> beam.Map(lambda x: x ** 3)
# Tracks all PCollections defined so far.
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment.py b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
index 39d1445..31bb736 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
@@ -79,11 +79,17 @@
self._watching_set = set()
# Holds variables list of (Dict[str, object]).
self._watching_dict_list = []
- # Holds results of pipeline runs as Dict[Pipeline, PipelineResult].
+ # Holds results of main jobs as Dict[Pipeline, PipelineResult].
# Each key is a pipeline instance defined by the end user. The
# InteractiveRunner is responsible for populating this dictionary
# implicitly.
- self._pipeline_results = {}
+ self._main_pipeline_results = {}
+ # Holds results of background caching jobs as
+ # Dict[Pipeline, PipelineResult]. Each key is a pipeline instance defined by
+ # the end user. The InteractiveRunner is responsible for populating this
+ # dictionary implicitly when a background caching jobs is started.
+ self._background_caching_pipeline_results = {}
+ self._cached_source_signature = {}
self._tracked_user_pipelines = set()
# Always watch __main__ module.
self.watch('__main__')
@@ -199,31 +205,48 @@
"""Gets the cache manager held by current Interactive Environment."""
return self._cache_manager
- def set_pipeline_result(self, pipeline, result):
- """Sets the pipeline run result. Adds one if absent. Otherwise, replace."""
+ def set_pipeline_result(self, pipeline, result, is_main_job):
+ """Sets the pipeline run result. Adds one if absent. Otherwise, replace.
+
+ When is_main_job is True, set the result for the main job; otherwise, set
+ the result for the background caching job.
+ """
assert issubclass(type(pipeline), beam.Pipeline), (
'pipeline must be an instance of apache_beam.Pipeline or its subclass')
assert issubclass(type(result), runner.PipelineResult), (
'result must be an instance of '
'apache_beam.runners.runner.PipelineResult or its subclass')
- self._pipeline_results[pipeline] = result
+ if is_main_job:
+ self._main_pipeline_results[pipeline] = result
+ else:
+ self._background_caching_pipeline_results[pipeline] = result
- def evict_pipeline_result(self, pipeline):
+ def evict_pipeline_result(self, pipeline, is_main_job=True):
"""Evicts the tracking of given pipeline run. Noop if absent."""
- return self._pipeline_results.pop(pipeline, None)
+ if is_main_job:
+ return self._main_pipeline_results.pop(pipeline, None)
+ return self._background_caching_pipeline_results.pop(pipeline, None)
- def pipeline_result(self, pipeline):
+ def pipeline_result(self, pipeline, is_main_job=True):
"""Gets the pipeline run result. None if absent."""
- return self._pipeline_results.get(pipeline, None)
+ if is_main_job:
+ return self._main_pipeline_results.get(pipeline, None)
+ return self._background_caching_pipeline_results.get(pipeline, None)
- def is_terminated(self, pipeline):
+ def is_terminated(self, pipeline, is_main_job=True):
"""Queries if the most recent job (by executing the given pipeline) state
is in a terminal state. True if absent."""
- result = self.pipeline_result(pipeline)
+ result = self.pipeline_result(pipeline, is_main_job=is_main_job)
if result:
return runner.PipelineState.is_terminal(result.state)
return True
+ def set_cached_source_signature(self, pipeline, signature):
+ self._cached_source_signature[pipeline] = signature
+
+ def get_cached_source_signature(self, pipeline):
+ return self._cached_source_signature.get(pipeline, set())
+
def track_user_pipelines(self):
"""Record references to all user-defined pipeline instances watched in
current environment.
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
index 76c29b8..ca9ac68 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
@@ -109,7 +109,8 @@
with self.assertRaises(AssertionError) as ctx:
ie.current_env().set_pipeline_result(NotPipeline(),
runner.PipelineResult(
- runner.PipelineState.RUNNING))
+ runner.PipelineState.RUNNING),
+ is_main_job=True)
self.assertTrue('pipeline must be an instance of apache_beam.Pipeline '
'or its subclass' in ctx.exception)
@@ -118,7 +119,10 @@
pass
with self.assertRaises(AssertionError) as ctx:
- ie.current_env().set_pipeline_result(self._p, NotResult())
+ ie.current_env().set_pipeline_result(
+ self._p,
+ NotResult(),
+ is_main_job=True)
self.assertTrue('result must be an instance of '
'apache_beam.runners.runner.PipelineResult or its '
'subclass' in ctx.exception)
@@ -132,7 +136,10 @@
pipeline = PipelineSubClass()
pipeline_result = PipelineResultSubClass(runner.PipelineState.RUNNING)
- ie.current_env().set_pipeline_result(pipeline, pipeline_result)
+ ie.current_env().set_pipeline_result(
+ pipeline,
+ pipeline_result,
+ is_main_job=True)
self.assertIs(ie.current_env().pipeline_result(pipeline), pipeline_result)
def test_determine_terminal_state(self):
@@ -141,8 +148,10 @@
runner.PipelineState.CANCELLED,
runner.PipelineState.UPDATED,
runner.PipelineState.DRAINED):
- ie.current_env().set_pipeline_result(self._p, runner.PipelineResult(
- state))
+ ie.current_env().set_pipeline_result(
+ self._p,
+ runner.PipelineResult(state),
+ is_main_job=True)
self.assertTrue(ie.current_env().is_terminated(self._p))
for state in (runner.PipelineState.UNKNOWN,
runner.PipelineState.STARTING,
@@ -152,13 +161,18 @@
runner.PipelineState.PENDING,
runner.PipelineState.CANCELLING,
runner.PipelineState.UNRECOGNIZED):
- ie.current_env().set_pipeline_result(self._p, runner.PipelineResult(
- state))
+ ie.current_env().set_pipeline_result(
+ self._p,
+ runner.PipelineResult(state),
+ is_main_job=True)
self.assertFalse(ie.current_env().is_terminated(self._p))
def test_evict_pipeline_result(self):
pipeline_result = runner.PipelineResult(runner.PipelineState.DONE)
- ie.current_env().set_pipeline_result(self._p, pipeline_result)
+ ie.current_env().set_pipeline_result(
+ self._p,
+ pipeline_result,
+ is_main_job=True)
self.assertIs(ie.current_env().evict_pipeline_result(self._p),
pipeline_result)
self.assertIs(ie.current_env().pipeline_result(self._p), None)
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner.py b/sdks/python/apache_beam/runners/interactive/interactive_runner.py
index b0222c3..fd2b236 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_runner.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_runner.py
@@ -32,6 +32,7 @@
from apache_beam.runners.interactive import cache_manager as cache
from apache_beam.runners.interactive import interactive_environment as ie
from apache_beam.runners.interactive import pipeline_instrument as inst
+from apache_beam.runners.interactive import background_caching_job
from apache_beam.runners.interactive.display import pipeline_graph
# size of PCollection samples cached.
@@ -126,6 +127,17 @@
def run_pipeline(self, pipeline, options):
pipeline_instrument = inst.pin(pipeline, options)
+ # The user_pipeline analyzed might be None if the pipeline given has nothing
+ # to be cached and tracing back to the user defined pipeline is impossible.
+ # When it's None, there is no need to cache including the background
+ # caching job and no result to track since no background caching job is
+ # started at all.
+ user_pipeline = pipeline_instrument.user_pipeline
+ if user_pipeline:
+ # Should use the underlying runner and run asynchronously.
+ background_caching_job.attempt_to_run_background_caching_job(
+ self._underlying_runner, user_pipeline, options)
+
pipeline_to_execute = beam.pipeline.Pipeline.from_runner_api(
pipeline_instrument.instrumented_pipeline_proto(),
self._underlying_runner,
@@ -137,10 +149,19 @@
render_option=self._render_option)
a_pipeline_graph.display_graph()
- result = pipeline_to_execute.run()
- result.wait_until_finish()
+ main_job_result = PipelineResult(pipeline_to_execute.run(),
+ pipeline_instrument)
+ # In addition to this pipeline result setting, redundant result setting from
+ # outer scopes are also recommended since the user_pipeline might not be
+ # available from within this scope.
+ if user_pipeline:
+ ie.current_env().set_pipeline_result(
+ user_pipeline,
+ main_job_result,
+ is_main_job=True)
+ main_job_result.wait_until_finish()
- return PipelineResult(result, pipeline_instrument)
+ return main_job_result
class PipelineResult(beam.runners.runner.PipelineResult):
@@ -161,8 +182,7 @@
self._pipeline_instrument = pipeline_instrument
def wait_until_finish(self):
- # PipelineResult is not constructed until pipeline execution is finished.
- return
+ self._underlying_result.wait_until_finish()
def get(self, pcoll):
key = self._pipeline_instrument.cache_key(pcoll)
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py b/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py
index 36ebce8..9b5abf3 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_runner_test.py
@@ -29,11 +29,11 @@
import apache_beam as beam
from apache_beam.runners.direct import direct_runner
from apache_beam.runners.interactive import interactive_beam as ib
+from apache_beam.runners.interactive import interactive_environment as ie
from apache_beam.runners.interactive import interactive_runner
def print_with_message(msg):
-
def printer(elem):
print(msg, elem)
return elem
@@ -43,15 +43,20 @@
class InteractiveRunnerTest(unittest.TestCase):
+ def setUp(self):
+ ie.new_env()
+
def test_basic(self):
p = beam.Pipeline(
runner=interactive_runner.InteractiveRunner(
direct_runner.DirectRunner()))
+ ib.watch({'p': p})
p.run().wait_until_finish()
pc0 = (
p | 'read' >> beam.Create([1, 2, 3])
| 'Print1.1' >> beam.Map(print_with_message('Run1.1')))
pc = pc0 | 'Print1.2' >> beam.Map(print_with_message('Run1.2'))
+ ib.watch(locals())
p.run().wait_until_finish()
_ = pc | 'Print2' >> beam.Map(print_with_message('Run2'))
p.run().wait_until_finish()
@@ -59,7 +64,6 @@
p.run().wait_until_finish()
def test_wordcount(self):
-
class WordExtractingDoFn(beam.DoFn):
def process(self, element):
diff --git a/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py b/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py
index b9255e4..084e2ac 100644
--- a/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py
+++ b/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py
@@ -100,6 +100,11 @@
# (Dict[str, AppliedPTransform]).
self._cached_pcoll_read = {}
+ # Reference to the user defined pipeline instance based on the given
+ # pipeline. The class never mutates it.
+ # Note: the original pipeline is not the user pipeline.
+ self._user_pipeline = None
+
def instrumented_pipeline_proto(self):
"""Always returns a new instance of portable instrumented proto."""
return self._pipeline.to_runner_api(use_fake_coders=True)
@@ -258,6 +263,20 @@
"""Returns a snapshot of the pipeline before instrumentation."""
return self._pipeline_snap
+ @property
+ def user_pipeline(self):
+ """Returns a reference to the pipeline instance defined by the user. If a
+ pipeline has no cacheable PCollection and the user pipeline cannot be
+ found, return None indicating there is nothing to be cached in the user
+ pipeline.
+
+ The pipeline given for instrumenting and mutated in this class is not
+ necessarily the pipeline instance defined by the user. From the watched
+ scopes, this class figures out what the user pipeline instance is.
+ This metadata can be used for tracking pipeline results.
+ """
+ return self._user_pipeline
+
def instrument(self):
"""Instruments original pipeline with cache.
@@ -330,6 +349,10 @@
cacheable_key = self._pin._cacheable_key(pcoll)
if (cacheable_key in self._pin.cacheables and
self._pin.cacheables[cacheable_key]['pcoll'] != pcoll):
+ if not self._pin._user_pipeline:
+ # Retrieve a reference to the user defined pipeline instance.
+ self._pin._user_pipeline = self._pin.cacheables[cacheable_key][
+ 'pcoll'].pipeline
self._pin.cacheables[cacheable_key]['pcoll'] = pcoll
v = PreprocessVisitor(self)
diff --git a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py
index 09b646e..f06c271 100644
--- a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py
+++ b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py
@@ -273,6 +273,23 @@
p_origin.visit(v)
assert_pipeline_equal(self, p_origin, p_copy)
+ def test_find_out_correct_user_pipeline(self):
+ # This is the user pipeline instance we care in the watched scope.
+ user_pipeline, _, _ = self._example_pipeline()
+ # This is a new runner pipeline instance with the same pipeline graph to
+ # what the user_pipeline represents.
+ runner_pipeline = beam.pipeline.Pipeline.from_runner_api(
+ user_pipeline.to_runner_api(use_fake_coders=True),
+ user_pipeline.runner,
+ options=None)
+ # This is a totally irrelevant user pipeline in the watched scope.
+ irrelevant_user_pipeline = beam.Pipeline(
+ interactive_runner.InteractiveRunner())
+ ib.watch({'irrelevant_user_pipeline': irrelevant_user_pipeline})
+ # Build instrument from the runner pipeline.
+ pipeline_instrument = instr.pin(runner_pipeline)
+ self.assertTrue(pipeline_instrument.user_pipeline is user_pipeline)
+
if __name__ == '__main__':
unittest.main()
diff --git a/sdks/python/apache_beam/runners/interactive/testing/mock_ipython.py b/sdks/python/apache_beam/runners/interactive/testing/mock_ipython.py
index ee3acd8..fc11e4d 100644
--- a/sdks/python/apache_beam/runners/interactive/testing/mock_ipython.py
+++ b/sdks/python/apache_beam/runners/interactive/testing/mock_ipython.py
@@ -15,9 +15,6 @@
# limitations under the License.
#
-# Mocked object returned by invoking get_ipython() in an ipython environment.
-_mocked_get_ipython = None
-
def mock_get_ipython():
"""Mock an ipython environment w/o setting up real ipython kernel.
@@ -27,17 +24,20 @@
Examples::
- # Usage, before each test function, append:
- @patch('IPython.get_ipython', mock_get_ipython)
+ # Usage, before each test function, prepend:
+ @patch('IPython.get_ipython', new_callable=mock_get_ipython)
- # Group lines of code into a cell:
- with mock_get_ipython():
+ # In the test function's signature, add an argument for the patch, e.g.:
+ def some_test(self, cell):
+
+ # Group lines of code into a cell using the argument:
+ with cell:
# arbitrary python code
# ...
# arbitrary python code
# Next cell with prompt increased by one:
- with mock_get_ipython(): # Auto-incremental
+ with cell: # Auto-incremental
# arbitrary python code
# ...
# arbitrary python code
@@ -48,6 +48,9 @@
def __init__(self):
self._execution_count = 0
+ def __call__(self):
+ return self
+
@property
def execution_count(self):
"""Execution count always starts from 1 and is constant within a cell."""
@@ -61,7 +64,4 @@
"""Marks exiting of a cell/prompt."""
pass
- global _mocked_get_ipython
- if not _mocked_get_ipython:
- _mocked_get_ipython = MockedGetIpython()
- return _mocked_get_ipython
+ return MockedGetIpython()
diff --git a/sdks/python/apache_beam/runners/portability/abstract_job_service.py b/sdks/python/apache_beam/runners/portability/abstract_job_service.py
index e285257..df7f9d22 100644
--- a/sdks/python/apache_beam/runners/portability/abstract_job_service.py
+++ b/sdks/python/apache_beam/runners/portability/abstract_job_service.py
@@ -17,25 +17,34 @@
from __future__ import absolute_import
import itertools
+import json
import logging
+import shutil
+import tempfile
import uuid
+import zipfile
from builtins import object
+from concurrent import futures
from typing import TYPE_CHECKING
from typing import Dict
from typing import Iterator
from typing import Optional
from typing import Union
+import grpc
+from google.protobuf import json_format
from google.protobuf import timestamp_pb2
+from apache_beam.portability.api import beam_artifact_api_pb2_grpc
from apache_beam.portability.api import beam_job_api_pb2
from apache_beam.portability.api import beam_job_api_pb2_grpc
+from apache_beam.portability.api import endpoints_pb2
+from apache_beam.runners.portability import artifact_service
from apache_beam.utils.timestamp import Timestamp
if TYPE_CHECKING:
from google.protobuf import struct_pb2 # pylint: disable=ungrouped-imports
from apache_beam.portability.api import beam_runner_api_pb2
- from apache_beam.portability.api import endpoints_pb2
_LOGGER = logging.getLogger(__name__)
@@ -260,3 +269,70 @@
job_name=self._job_name,
pipeline_options=self._pipeline_options,
state=self.state)
+
+
+class UberJarBeamJob(AbstractBeamJob):
+ """Abstract baseclass for creating a Beam job. The resulting job will be
+ packaged and run in an executable uber jar."""
+
+ # These must agree with those defined in PortablePipelineJarUtils.java.
+ PIPELINE_FOLDER = 'BEAM-PIPELINE'
+ PIPELINE_MANIFEST = PIPELINE_FOLDER + '/pipeline-manifest.json'
+
+ # We only stage a single pipeline in the jar.
+ PIPELINE_NAME = 'pipeline'
+ PIPELINE_PATH = '/'.join(
+ [PIPELINE_FOLDER, PIPELINE_NAME, "pipeline.json"])
+ PIPELINE_OPTIONS_PATH = '/'.join(
+ [PIPELINE_FOLDER, PIPELINE_NAME, 'pipeline-options.json'])
+ ARTIFACT_MANIFEST_PATH = '/'.join(
+ [PIPELINE_FOLDER, PIPELINE_NAME, 'artifact-manifest.json'])
+ ARTIFACT_FOLDER = '/'.join([PIPELINE_FOLDER, PIPELINE_NAME, 'artifacts'])
+
+ def __init__(
+ self, executable_jar, job_id, job_name, pipeline, options,
+ artifact_port=0):
+ super(UberJarBeamJob, self).__init__(job_id, job_name, pipeline, options)
+ self._executable_jar = executable_jar
+ self._jar_uploaded = False
+ self._artifact_port = artifact_port
+
+ def prepare(self):
+ # Copy the executable jar, injecting the pipeline and options as resources.
+ with tempfile.NamedTemporaryFile(suffix='.jar') as tout:
+ self._jar = tout.name
+ shutil.copy(self._executable_jar, self._jar)
+ with zipfile.ZipFile(self._jar, 'a', compression=zipfile.ZIP_DEFLATED) as z:
+ with z.open(self.PIPELINE_PATH, 'w') as fout:
+ fout.write(json_format.MessageToJson(
+ self._pipeline_proto).encode('utf-8'))
+ with z.open(self.PIPELINE_OPTIONS_PATH, 'w') as fout:
+ fout.write(json_format.MessageToJson(
+ self._pipeline_options).encode('utf-8'))
+ with z.open(self.PIPELINE_MANIFEST, 'w') as fout:
+ fout.write(json.dumps(
+ {'defaultJobName': self.PIPELINE_NAME}).encode('utf-8'))
+ self._start_artifact_service(self._jar, self._artifact_port)
+
+ def _start_artifact_service(self, jar, requested_port):
+ self._artifact_staging_service = artifact_service.ZipFileArtifactService(
+ jar, self.ARTIFACT_FOLDER)
+ self._artifact_staging_server = grpc.server(futures.ThreadPoolExecutor())
+ port = self._artifact_staging_server.add_insecure_port(
+ '[::]:%s' % requested_port)
+ beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
+ self._artifact_staging_service, self._artifact_staging_server)
+ self._artifact_staging_endpoint = endpoints_pb2.ApiServiceDescriptor(
+ url='localhost:%d' % port)
+ self._artifact_staging_server.start()
+ _LOGGER.info('Artifact server started on port %s', port)
+ return port
+
+ def _stop_artifact_service(self):
+ self._artifact_staging_server.stop(1)
+ self._artifact_staging_service.close()
+ self._artifact_manifest_location = (
+ self._artifact_staging_service.retrieval_token(self._job_id))
+
+ def artifact_staging_endpoint(self):
+ return self._artifact_staging_endpoint
diff --git a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
index b318971..8a035f8 100644
--- a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
+++ b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
@@ -20,25 +20,17 @@
from __future__ import absolute_import
from __future__ import print_function
-import json
import logging
import os
-import shutil
import tempfile
import time
import zipfile
-from concurrent import futures
-import grpc
import requests
-from google.protobuf import json_format
from apache_beam.options import pipeline_options
-from apache_beam.portability.api import beam_artifact_api_pb2_grpc
from apache_beam.portability.api import beam_job_api_pb2
-from apache_beam.portability.api import endpoints_pb2
from apache_beam.runners.portability import abstract_job_service
-from apache_beam.runners.portability import artifact_service
from apache_beam.runners.portability import job_server
_LOGGER = logging.getLogger(__name__)
@@ -89,77 +81,24 @@
artifact_port=self._artifact_port)
-class FlinkBeamJob(abstract_job_service.AbstractBeamJob):
+class FlinkBeamJob(abstract_job_service.UberJarBeamJob):
"""Runs a single Beam job on Flink by staging all contents into a Jar
and uploading it via the Flink Rest API."""
- # These must agree with those defined in PortablePipelineJarUtils.java.
- PIPELINE_FOLDER = 'BEAM-PIPELINE'
- PIPELINE_MANIFEST = PIPELINE_FOLDER + '/pipeline-manifest.json'
-
- # We only stage a single pipeline in the jar.
- PIPELINE_NAME = 'pipeline'
- PIPELINE_PATH = '/'.join(
- [PIPELINE_FOLDER, PIPELINE_NAME, "pipeline.json"])
- PIPELINE_OPTIONS_PATH = '/'.join(
- [PIPELINE_FOLDER, PIPELINE_NAME, 'pipeline-options.json'])
- ARTIFACT_MANIFEST_PATH = '/'.join(
- [PIPELINE_FOLDER, PIPELINE_NAME, 'artifact-manifest.json'])
- ARTIFACT_FOLDER = '/'.join([PIPELINE_FOLDER, PIPELINE_NAME, 'artifacts'])
-
def __init__(
self, master_url, executable_jar, job_id, job_name, pipeline, options,
artifact_port=0):
- super(FlinkBeamJob, self).__init__(job_id, job_name, pipeline, options)
+ super(FlinkBeamJob, self).__init__(
+ executable_jar, job_id, job_name, pipeline, options,
+ artifact_port=artifact_port)
self._master_url = master_url
- self._executable_jar = executable_jar
- self._jar_uploaded = False
- self._artifact_port = artifact_port
-
- def prepare(self):
- # Copy the executable jar, injecting the pipeline and options as resources.
- with tempfile.NamedTemporaryFile(suffix='.jar') as tout:
- self._jar = tout.name
- shutil.copy(self._executable_jar, self._jar)
- with zipfile.ZipFile(self._jar, 'a', compression=zipfile.ZIP_DEFLATED) as z:
- with z.open(self.PIPELINE_PATH, 'w') as fout:
- fout.write(json_format.MessageToJson(
- self._pipeline_proto).encode('utf-8'))
- with z.open(self.PIPELINE_OPTIONS_PATH, 'w') as fout:
- fout.write(json_format.MessageToJson(
- self._pipeline_options).encode('utf-8'))
- with z.open(self.PIPELINE_MANIFEST, 'w') as fout:
- fout.write(json.dumps(
- {'defaultJobName': self.PIPELINE_NAME}).encode('utf-8'))
- self._start_artifact_service(self._jar, self._artifact_port)
-
- def _start_artifact_service(self, jar, requested_port):
- self._artifact_staging_service = artifact_service.ZipFileArtifactService(
- jar, self.ARTIFACT_FOLDER)
- self._artifact_staging_server = grpc.server(futures.ThreadPoolExecutor())
- port = self._artifact_staging_server.add_insecure_port(
- '[::]:%s' % requested_port)
- beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
- self._artifact_staging_service, self._artifact_staging_server)
- self._artifact_staging_endpoint = endpoints_pb2.ApiServiceDescriptor(
- url='localhost:%d' % port)
- self._artifact_staging_server.start()
- _LOGGER.info('Artifact server started on port %s', port)
- return port
-
- def _stop_artifact_service(self):
- self._artifact_staging_server.stop(1)
- self._artifact_staging_service.close()
- self._artifact_manifest_location = (
- self._artifact_staging_service.retrieval_token(self._job_id))
-
- def artifact_staging_endpoint(self):
- return self._artifact_staging_endpoint
def request(self, method, path, expected_status=200, **kwargs):
- response = method('%s/%s' % (self._master_url, path), **kwargs)
+ url = '%s/%s' % (self._master_url, path)
+ response = method(url, **kwargs)
if response.status_code != expected_status:
- raise RuntimeError(response.text)
+ raise RuntimeError("Request to %s failed with status %d: %s" %
+ (url, response.status_code, response.text))
if response.text:
return response.json()
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
index c03b6a9..7c45847 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
@@ -126,6 +126,9 @@
# The cache is disabled in production for other runners.
STATE_CACHE_SIZE = 100
+# Time-based flush is enabled in the fn_api_runner by default.
+DATA_BUFFER_TIME_LIMIT_MS = 1000
+
_LOGGER = logging.getLogger(__name__)
@@ -1442,7 +1445,8 @@
beam_artifact_api_pb2_grpc.add_ArtifactRetrievalServiceServicer_to_server(
service, self.control_server)
- self.data_plane_handler = data_plane.BeamFnDataServicer()
+ self.data_plane_handler = data_plane.BeamFnDataServicer(
+ DATA_BUFFER_TIME_LIMIT_MS)
beam_fn_api_pb2_grpc.add_BeamFnDataServicer_to_server(
self.data_plane_handler, self.data_server)
@@ -1583,16 +1587,20 @@
# type: (...) -> None
super(EmbeddedGrpcWorkerHandler, self).__init__(state, provision_info,
grpc_server)
- if payload:
- state_cache_size = payload.decode('ascii')
- self._state_cache_size = int(state_cache_size)
- else:
- self._state_cache_size = STATE_CACHE_SIZE
+
+ from apache_beam.transforms.environments import EmbeddedPythonGrpcEnvironment
+ config = EmbeddedPythonGrpcEnvironment.parse_config(
+ payload.decode('utf-8'))
+ self._state_cache_size = config.get('state_cache_size') or STATE_CACHE_SIZE
+ self._data_buffer_time_limit_ms = \
+ config.get('data_buffer_time_limit_ms') or DATA_BUFFER_TIME_LIMIT_MS
def start_worker(self):
# type: () -> None
self.worker = sdk_worker.SdkHarness(
- self.control_address, state_cache_size=self._state_cache_size,
+ self.control_address,
+ state_cache_size=self._state_cache_size,
+ data_buffer_time_limit_ms=self._data_buffer_time_limit_ms,
worker_id=self.worker_id)
self.worker_thread = threading.Thread(
name='run_worker', target=self.worker.run)
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
index 6f76fd1..e678f65 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
@@ -598,7 +598,8 @@
with self.create_pipeline() as p:
big = (p
| beam.Create(['a', 'a', 'b'])
- | beam.Map(lambda x: (x, x * data_plane._DEFAULT_FLUSH_THRESHOLD)))
+ | beam.Map(lambda x: (
+ x, x * data_plane._DEFAULT_SIZE_FLUSH_THRESHOLD)))
side_input_res = (
big
@@ -1154,7 +1155,8 @@
return beam.Pipeline(
runner=fn_api_runner.FnApiRunner(
default_environment=environments.EmbeddedPythonGrpcEnvironment(
- state_cache_size=0)))
+ state_cache_size=0,
+ data_buffer_time_limit_ms=0)))
class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py
index 3a5a63a..6ff8689 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py
@@ -58,10 +58,12 @@
common_urns.primitives.PAR_DO.urn, # After SDF expansion.
])
+
COMBINE_URNS = frozenset([
common_urns.composites.COMBINE_PER_KEY.urn,
])
+
PAR_DO_URNS = frozenset([
common_urns.primitives.PAR_DO.urn,
common_urns.sdf_components.PAIR_WITH_RESTRICTION.urn,
@@ -71,6 +73,7 @@
common_urns.sdf_components.PROCESS_ELEMENTS.urn,
])
+
IMPULSE_BUFFER = b'impulse'
@@ -114,17 +117,11 @@
@staticmethod
def _extract_environment(transform):
+
# type: (beam_runner_api_pb2.PTransform) -> Optional[str]
- if transform.spec.urn in PAR_DO_URNS:
- pardo_payload = proto_utils.parse_Bytes(
- transform.spec.payload, beam_runner_api_pb2.ParDoPayload)
- return pardo_payload.do_fn.environment_id
- elif transform.spec.urn in COMBINE_URNS:
- combine_payload = proto_utils.parse_Bytes(
- transform.spec.payload, beam_runner_api_pb2.CombinePayload)
- return combine_payload.combine_fn.environment_id
- else:
- return None
+ environment = transform.environment_id
+ if environment:
+ return environment
@staticmethod
def _merge_environments(env1, env2):
@@ -295,7 +292,7 @@
payload=payload.SerializeToString()),
inputs=named_inputs,
outputs={'output_%d' % ix: pcoll
- for ix, pcoll in enumerate(external_outputs)})
+ for ix, pcoll in enumerate(external_outputs)},)
def memoize_on_instance(f):
@@ -681,13 +678,7 @@
context.components.pcollections[
only_element(list(combine_per_key_transform.inputs.values()))
].windowing_strategy_id]
- if windowing.output_time != beam_runner_api_pb2.OutputTime.END_OF_WINDOW:
- # This depends on the spec of PartialGroupByKey.
- return False
- elif not is_compatible_with_combiner_lifting(windowing.trigger):
- return False
- else:
- return True
+ return is_compatible_with_combiner_lifting(windowing.trigger)
def make_stage(base_stage, transform):
# type: (Stage, beam_runner_api_pb2.PTransform) -> Stage
@@ -771,7 +762,8 @@
.COMBINE_PER_KEY_PRECOMBINE.urn,
payload=transform.spec.payload),
inputs=transform.inputs,
- outputs={'out': precombined_pcoll_id}))
+ outputs={'out': precombined_pcoll_id},
+ environment_id=transform.environment_id))
yield make_stage(
stage,
@@ -791,7 +783,8 @@
.COMBINE_PER_KEY_MERGE_ACCUMULATORS.urn,
payload=transform.spec.payload),
inputs={'in': grouped_pcoll_id},
- outputs={'out': merged_pcoll_id}))
+ outputs={'out': merged_pcoll_id},
+ environment_id=transform.environment_id))
yield make_stage(
stage,
@@ -802,7 +795,8 @@
.COMBINE_PER_KEY_EXTRACT_OUTPUTS.urn,
payload=transform.spec.payload),
inputs={'in': merged_pcoll_id},
- outputs=transform.outputs))
+ outputs=transform.outputs,
+ environment_id=transform.environment_id))
def unlifted_stages(stage):
transform = stage.transforms[0]
@@ -1043,7 +1037,8 @@
inputs={local_in: pcoll_in},
outputs={'out': transcoded_pcollection},
spec=beam_runner_api_pb2.FunctionSpec(
- urn=bundle_processor.IDENTITY_DOFN_URN))],
+ urn=bundle_processor.IDENTITY_DOFN_URN),
+ environment_id=transform.environment_id)],
downstream_side_inputs=frozenset(),
must_follow=stage.must_follow)
pcollections[transcoded_pcollection].CopyFrom(
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py b/sdks/python/apache_beam/runners/portability/portable_runner.py
index e29a345..0221068 100644
--- a/sdks/python/apache_beam/runners/portability/portable_runner.py
+++ b/sdks/python/apache_beam/runners/portability/portable_runner.py
@@ -151,6 +151,8 @@
portable_options.environment_config, server = (
worker_pool_main.BeamFnExternalWorkerPoolServicer.start(
state_cache_size=sdk_worker_main._get_state_cache_size(options),
+ data_buffer_time_limit_ms=
+ sdk_worker_main._get_data_buffer_time_limit_ms(options),
use_process=use_loopback_process_worker))
cleanup_callbacks = [functools.partial(server.stop, 1)]
else:
@@ -417,6 +419,7 @@
def wait_until_finish(self):
def read_messages():
+ previous_state = -1
for message in self._message_stream:
if message.HasField('message_response'):
logging.log(
@@ -424,10 +427,12 @@
"%s",
message.message_response.message_text)
else:
- _LOGGER.info(
- "Job state changed to %s",
- self._runner_api_state_to_pipeline_state(
- message.state_response.state))
+ current_state = message.state_response.state
+ if current_state != previous_state:
+ _LOGGER.info(
+ "Job state changed to %s",
+ self._runner_api_state_to_pipeline_state(current_state))
+ previous_state = current_state
self._messages.append(message)
t = threading.Thread(target=read_messages, name='wait_until_finish_read')
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner_test.py b/sdks/python/apache_beam/runners/portability/portable_runner_test.py
index f8b6cf8..81e8cce 100644
--- a/sdks/python/apache_beam/runners/portability/portable_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/portable_runner_test.py
@@ -185,6 +185,9 @@
python_urns.EMBEDDED_PYTHON)
# Enable caching (disabled by default)
options.view_as(DebugOptions).add_experiment('state_cache_size=100')
+ # Enable time-based data buffer (disabled by default)
+ options.view_as(DebugOptions).add_experiment(
+ 'data_buffer_time_limit_ms=1000')
return options
def create_pipeline(self):
@@ -240,6 +243,8 @@
options = super(PortableRunnerOptimized, self).create_options()
options.view_as(DebugOptions).add_experiment('pre_optimize=all')
options.view_as(DebugOptions).add_experiment('state_cache_size=100')
+ options.view_as(DebugOptions).add_experiment(
+ 'data_buffer_time_limit_ms=1000')
return options
@@ -249,7 +254,7 @@
def setUpClass(cls):
cls._worker_address, cls._worker_server = (
worker_pool_main.BeamFnExternalWorkerPoolServicer.start(
- state_cache_size=100))
+ state_cache_size=100, data_buffer_time_limit_ms=1000))
@classmethod
def tearDownClass(cls):
@@ -274,6 +279,9 @@
sys.executable.encode('ascii')).decode('utf-8')
# Enable caching (disabled by default)
options.view_as(DebugOptions).add_experiment('state_cache_size=100')
+ # Enable time-based data buffer (disabled by default)
+ options.view_as(DebugOptions).add_experiment(
+ 'data_buffer_time_limit_ms=1000')
return options
@classmethod
diff --git a/sdks/python/apache_beam/runners/portability/spark_runner.py b/sdks/python/apache_beam/runners/portability/spark_runner.py
index 8c3939e..ce33acd 100644
--- a/sdks/python/apache_beam/runners/portability/spark_runner.py
+++ b/sdks/python/apache_beam/runners/portability/spark_runner.py
@@ -21,10 +21,12 @@
from __future__ import print_function
import re
+import sys
from apache_beam.options import pipeline_options
from apache_beam.runners.portability import job_server
from apache_beam.runners.portability import portable_runner
+from apache_beam.runners.portability import spark_uber_jar_job_server
# https://spark.apache.org/docs/latest/submitting-applications.html#master-urls
LOCAL_MASTER_PATTERN = r'^local(\[.+\])?$'
@@ -32,7 +34,7 @@
class SparkRunner(portable_runner.PortableRunner):
def run_pipeline(self, pipeline, options):
- spark_options = options.view_as(SparkRunnerOptions)
+ spark_options = options.view_as(pipeline_options.SparkRunnerOptions)
portable_options = options.view_as(pipeline_options.PortableOptions)
if (re.match(LOCAL_MASTER_PATTERN, spark_options.spark_master_url)
and not portable_options.environment_type
@@ -41,27 +43,23 @@
return super(SparkRunner, self).run_pipeline(pipeline, options)
def default_job_server(self, options):
- # TODO(BEAM-8139) submit a Spark jar to a cluster
+ spark_options = options.view_as(pipeline_options.SparkRunnerOptions)
+ if spark_options.spark_submit_uber_jar:
+ if sys.version_info < (3, 6):
+ raise ValueError(
+ 'spark_submit_uber_jar requires Python 3.6+, current version %s'
+ % sys.version)
+ if not spark_options.spark_rest_url:
+ raise ValueError('Option spark_rest_url must be set.')
+ return spark_uber_jar_job_server.SparkUberJarJobServer(
+ spark_options.spark_rest_url, options)
return job_server.StopOnExitJobServer(SparkJarJobServer(options))
-class SparkRunnerOptions(pipeline_options.PipelineOptions):
- @classmethod
- def _add_argparse_args(cls, parser):
- parser.add_argument('--spark_master_url',
- default='local[4]',
- help='Spark master URL (spark://HOST:PORT). '
- 'Use "local" (single-threaded) or "local[*]" '
- '(multi-threaded) to start a local cluster for '
- 'the execution.')
- parser.add_argument('--spark_job_server_jar',
- help='Path or URL to a Beam Spark jobserver jar.')
-
-
class SparkJarJobServer(job_server.JavaJarJobServer):
def __init__(self, options):
super(SparkJarJobServer, self).__init__(options)
- options = options.view_as(SparkRunnerOptions)
+ options = options.view_as(pipeline_options.SparkRunnerOptions)
self._jar = options.spark_job_server_jar
self._master_url = options.spark_master_url
diff --git a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py
new file mode 100644
index 0000000..88f7cb7
--- /dev/null
+++ b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py
@@ -0,0 +1,255 @@
+#
+# 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.
+#
+
+"""A job server submitting portable pipelines as uber jars to Spark."""
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+import itertools
+import logging
+import tempfile
+import time
+import zipfile
+
+import requests
+
+from apache_beam.options import pipeline_options
+from apache_beam.portability.api import beam_job_api_pb2
+from apache_beam.runners.portability import abstract_job_service
+from apache_beam.runners.portability import job_server
+from apache_beam.utils.timestamp import Timestamp
+
+_LOGGER = logging.getLogger(__name__)
+
+
+class SparkUberJarJobServer(abstract_job_service.AbstractJobServiceServicer):
+ """A Job server which submits a self-contained Jar to a Spark cluster.
+
+ The jar contains the Beam pipeline definition, dependencies, and
+ the pipeline artifacts.
+ """
+
+ def __init__(self, rest_url, options):
+ super(SparkUberJarJobServer, self).__init__()
+ self._rest_url = rest_url
+ self._executable_jar = (options.view_as(pipeline_options.SparkRunnerOptions)
+ .spark_job_server_jar)
+ self._artifact_port = (options.view_as(pipeline_options.JobServerOptions)
+ .artifact_port)
+ self._temp_dir = tempfile.mkdtemp(prefix='apache-beam-spark')
+
+ def start(self):
+ return self
+
+ def stop(self):
+ pass
+
+ def executable_jar(self):
+ url = (self._executable_jar or
+ job_server.JavaJarJobServer.path_to_beam_jar(
+ 'runners:spark:job-server:shadowJar'))
+ return job_server.JavaJarJobServer.local_jar(url)
+
+ def create_beam_job(self, job_id, job_name, pipeline, options):
+ return SparkBeamJob(
+ self._rest_url,
+ self.executable_jar(),
+ job_id,
+ job_name,
+ pipeline,
+ options,
+ artifact_port=self._artifact_port)
+
+
+class SparkBeamJob(abstract_job_service.UberJarBeamJob):
+ """Runs a single Beam job on Spark by staging all contents into a Jar
+ and uploading it via the Spark Rest API.
+
+ Note that the Spark Rest API is not enabled by default. It must be enabled by
+ setting the configuration property spark.master.rest.enabled to true."""
+
+ def __init__(
+ self, rest_url, executable_jar, job_id, job_name, pipeline, options,
+ artifact_port=0):
+ super(SparkBeamJob, self).__init__(
+ executable_jar, job_id, job_name, pipeline, options,
+ artifact_port=artifact_port)
+ self._rest_url = rest_url
+ # Message history is a superset of state history.
+ self._message_history = self._state_history[:]
+
+ def request(self, method, path, expected_status=200, **kwargs):
+ url = '%s/%s' % (self._rest_url, path)
+ response = method(url, **kwargs)
+ if response.status_code != expected_status:
+ raise RuntimeError("Request to %s failed with status %d: %s" %
+ (url, response.status_code, response.text))
+ if response.text:
+ return response.json()
+
+ def get(self, path, **kwargs):
+ return self.request(requests.get, path, **kwargs)
+
+ def post(self, path, **kwargs):
+ return self.request(requests.post, path, **kwargs)
+
+ def delete(self, path, **kwargs):
+ return self.request(requests.delete, path, **kwargs)
+
+ def _get_server_spark_version(self):
+ # Spark REST API doesn't seem to offer a dedicated endpoint for getting the
+ # version, but it does include the version in all responses, even errors.
+ return self.get('', expected_status=400)['serverSparkVersion']
+
+ def _get_client_spark_version_from_properties(self, jar):
+ """Parse Spark version from spark-version-info.properties file in the jar.
+ https://github.com/apache/spark/blob/dddfeca175bdce5294debe00d4a993daef92ca60/build/spark-build-info#L30
+ """
+ with zipfile.ZipFile(jar, 'a', compression=zipfile.ZIP_DEFLATED) as z:
+ with z.open('spark-version-info.properties') as fin:
+ for line in fin.read().decode('utf-8').splitlines():
+ split = list(map(lambda s: s.strip(), line.split('=')))
+ if len(split) == 2 and split[0] == 'version' and split[1] != '':
+ return split[1]
+ raise ValueError(
+ 'Property "version" not found in spark-version-info.properties.')
+
+ def _get_client_spark_version(self, jar):
+ try:
+ return self._get_client_spark_version_from_properties(jar)
+ except Exception as e:
+ _LOGGER.debug(e)
+ server_version = self._get_server_spark_version()
+ _LOGGER.warning('Unable to parse Spark version from '
+ 'spark-version-info.properties. Defaulting to %s' %
+ server_version)
+ return server_version
+
+ def _create_submission_request(self, jar, job_name):
+ jar_url = "file:%s" % jar
+ return {
+ "action": "CreateSubmissionRequest",
+ "appArgs": [],
+ "appResource": jar_url,
+ "clientSparkVersion": self._get_client_spark_version(jar),
+ "environmentVariables": {},
+ "mainClass": "org.apache.beam.runners.spark.SparkPipelineRunner",
+ "sparkProperties": {
+ "spark.jars": jar_url,
+ "spark.app.name": job_name,
+ "spark.submit.deployMode": "cluster",
+ }
+ }
+
+ def run(self):
+ self._stop_artifact_service()
+ # Move the artifact manifest to the expected location.
+ with zipfile.ZipFile(self._jar, 'a', compression=zipfile.ZIP_DEFLATED) as z:
+ with z.open(self._artifact_manifest_location) as fin:
+ manifest_contents = fin.read()
+ with z.open(self.ARTIFACT_MANIFEST_PATH, 'w') as fout:
+ fout.write(manifest_contents)
+
+ # Upload the jar and start the job.
+ self._spark_submission_id = self.post(
+ 'v1/submissions/create',
+ json=self._create_submission_request(self._jar, self._job_name)
+ )['submissionId']
+ _LOGGER.info('Submitted Spark job with ID %s' % self._spark_submission_id)
+
+ def cancel(self):
+ self.post('v1/submissions/kill/%s' % self._spark_submission_id)
+
+ @staticmethod
+ def _get_beam_state(spark_response):
+ return {
+ 'SUBMITTED': beam_job_api_pb2.JobState.STARTING,
+ 'RUNNING': beam_job_api_pb2.JobState.RUNNING,
+ 'FINISHED': beam_job_api_pb2.JobState.DONE,
+ 'RELAUNCHING': beam_job_api_pb2.JobState.RUNNING,
+ 'UNKNOWN': beam_job_api_pb2.JobState.UNSPECIFIED,
+ 'KILLED': beam_job_api_pb2.JobState.CANCELLED,
+ 'FAILED': beam_job_api_pb2.JobState.FAILED,
+ 'ERROR': beam_job_api_pb2.JobState.FAILED,
+ }.get(spark_response['driverState'], beam_job_api_pb2.JobState.UNSPECIFIED)
+
+ def _get_spark_status(self):
+ return self.get('v1/submissions/status/%s' % self._spark_submission_id)
+
+ def get_state(self):
+ response = self._get_spark_status()
+ state = self._get_beam_state(response)
+ timestamp = self.set_state(state)
+ if timestamp is None:
+ # State has not changed since last check. Use previous timestamp.
+ return super(SparkBeamJob, self).get_state()
+ else:
+ return state, timestamp
+
+ def _with_message_history(self, message_stream):
+ return itertools.chain(self._message_history[:], message_stream)
+
+ def _get_message_iter(self):
+ """Returns an iterator of messages from the Spark server.
+ Note that while message history is de-duped, this function's returned
+ iterator may contain duplicate values."""
+ sleep_secs = 1.0
+ message_ix = 0
+ while True:
+ response = self._get_spark_status()
+ state = self._get_beam_state(response)
+ timestamp = Timestamp.now()
+ message = None
+ if 'message' in response:
+ importance = (
+ beam_job_api_pb2.JobMessage.MessageImportance.JOB_MESSAGE_ERROR if
+ state == beam_job_api_pb2.JobState.FAILED else
+ beam_job_api_pb2.JobMessage.MessageImportance.JOB_MESSAGE_BASIC)
+ message = beam_job_api_pb2.JobMessage(
+ message_id='message%d' % message_ix,
+ time=str(int(timestamp)),
+ importance=importance,
+ message_text=response['message'])
+ yield message
+ message_ix += 1
+ # TODO(BEAM-8983) In the event of a failure, query
+ # additional info from Spark master and/or workers.
+ check_timestamp = self.set_state(state)
+ if check_timestamp is not None:
+ if message:
+ self._message_history.append(message)
+ self._message_history.append((state, check_timestamp))
+ yield state, timestamp
+ sleep_secs = min(60, sleep_secs * 1.2)
+ time.sleep(sleep_secs)
+
+ def get_state_stream(self):
+ for msg in self._with_message_history(self._get_message_iter()):
+ if isinstance(msg, tuple):
+ state, timestamp = msg
+ yield state, timestamp
+ if self.is_terminal_state(state):
+ break
+
+ def get_message_stream(self):
+ for msg in self._with_message_history(self._get_message_iter()):
+ yield msg
+ if isinstance(msg, tuple):
+ state, _ = msg
+ if self.is_terminal_state(state):
+ break
diff --git a/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py
new file mode 100644
index 0000000..7106403
--- /dev/null
+++ b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py
@@ -0,0 +1,209 @@
+#
+# 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.
+#
+from __future__ import absolute_import
+from __future__ import print_function
+
+import contextlib
+import logging
+import os
+import sys
+import tempfile
+import unittest
+import zipfile
+
+import freezegun
+import grpc
+import requests_mock
+
+from apache_beam.options import pipeline_options
+from apache_beam.portability.api import beam_artifact_api_pb2
+from apache_beam.portability.api import beam_artifact_api_pb2_grpc
+from apache_beam.portability.api import beam_job_api_pb2
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability import spark_uber_jar_job_server
+
+
+@contextlib.contextmanager
+def temp_name(*args, **kwargs):
+ with tempfile.NamedTemporaryFile(*args, **kwargs) as t:
+ name = t.name
+ yield name
+ if os.path.exists(name):
+ os.unlink(name)
+
+def spark_job():
+ return spark_uber_jar_job_server.SparkBeamJob(
+ 'http://host:6066', '', '', '', '', '',
+ pipeline_options.SparkRunnerOptions())
+
+
+@unittest.skipIf(sys.version_info < (3, 6), "Requires Python 3.6+")
+class SparkUberJarJobServerTest(unittest.TestCase):
+
+ @requests_mock.mock()
+ def test_get_server_spark_version(self, http_mock):
+ http_mock.get('http://host:6066', json={
+ "action": "ErrorResponse",
+ "message": "Missing protocol version. Please submit requests through "
+ "http://[host]:[port]/v1/submissions/...",
+ "serverSparkVersion": "1.2.3"
+ }, status_code=400)
+ self.assertEqual(spark_job()._get_server_spark_version(), "1.2.3")
+
+ def test_get_client_spark_version_from_properties(self):
+ with temp_name(suffix='fake.jar') as fake_jar:
+ with zipfile.ZipFile(fake_jar, 'w') as zip:
+ with zip.open('spark-version-info.properties', 'w') as fout:
+ fout.write(b'version=4.5.6')
+ self.assertEqual(spark_job().
+ _get_client_spark_version_from_properties(fake_jar),
+ "4.5.6")
+
+ def test_get_client_spark_version_from_properties_no_properties_file(self):
+ with self.assertRaises(KeyError):
+ with temp_name(suffix='fake.jar') as fake_jar:
+ with zipfile.ZipFile(fake_jar, 'w') as zip:
+ # Write some other file to the jar.
+ with zip.open('FakeClass.class', 'w') as fout:
+ fout.write(b'[original_contents]')
+ spark_job()._get_client_spark_version_from_properties(fake_jar)
+
+ def test_get_client_spark_version_from_properties_missing_version(self):
+ with self.assertRaises(ValueError):
+ with temp_name(suffix='fake.jar') as fake_jar:
+ with zipfile.ZipFile(fake_jar, 'w') as zip:
+ with zip.open('spark-version-info.properties', 'w') as fout:
+ fout.write(b'version=')
+ spark_job()._get_client_spark_version_from_properties(fake_jar)
+
+ @requests_mock.mock()
+ @freezegun.freeze_time("1970-01-01")
+ def test_end_to_end(self, http_mock):
+ submission_id = "submission-id"
+ worker_host_port = "workerhost:12345"
+ worker_id = "worker-id"
+ server_spark_version = "1.2.3"
+
+ def spark_submission_status_response(state):
+ return {
+ 'json': {
+ "action": "SubmissionStatusResponse",
+ "driverState": state,
+ "serverSparkVersion": server_spark_version,
+ "submissionId": submission_id,
+ "success": "true",
+ "workerHostPort": worker_host_port,
+ "workerId": worker_id
+ }
+ }
+
+ with temp_name(suffix='fake.jar') as fake_jar:
+ with zipfile.ZipFile(fake_jar, 'w') as zip:
+ with zip.open('spark-version-info.properties', 'w') as fout:
+ fout.write(b'version=4.5.6')
+
+ options = pipeline_options.SparkRunnerOptions()
+ options.spark_job_server_jar = fake_jar
+ job_server = spark_uber_jar_job_server.SparkUberJarJobServer(
+ 'http://host:6066', options)
+
+ # Prepare the job.
+ prepare_response = job_server.Prepare(
+ beam_job_api_pb2.PrepareJobRequest(
+ job_name='job',
+ pipeline=beam_runner_api_pb2.Pipeline()))
+ channel = grpc.insecure_channel(
+ prepare_response.artifact_staging_endpoint.url)
+ retrieval_token = beam_artifact_api_pb2_grpc.ArtifactStagingServiceStub(
+ channel).CommitManifest(
+ beam_artifact_api_pb2.CommitManifestRequest(
+ staging_session_token=prepare_response.staging_session_token,
+ manifest=beam_artifact_api_pb2.Manifest())
+ ).retrieval_token
+ channel.close()
+
+ # Now actually run the job.
+ http_mock.post(
+ 'http://host:6066/v1/submissions/create',
+ json={
+ "action": "CreateSubmissionResponse",
+ "message": "Driver successfully submitted as submission-id",
+ "serverSparkVersion": "1.2.3",
+ "submissionId": "submission-id",
+ "success": "true"
+ })
+ job_server.Run(
+ beam_job_api_pb2.RunJobRequest(
+ preparation_id=prepare_response.preparation_id,
+ retrieval_token=retrieval_token))
+
+ # Check the status until the job is "done" and get all error messages.
+ http_mock.get(
+ 'http://host:6066/v1/submissions/status/submission-id',
+ [spark_submission_status_response('RUNNING'),
+ spark_submission_status_response('RUNNING'),
+ {
+ 'json': {
+ "action": "SubmissionStatusResponse",
+ "driverState": "ERROR",
+ "message": "oops",
+ "serverSparkVersion": "1.2.3",
+ "submissionId": submission_id,
+ "success": "true",
+ "workerHostPort": worker_host_port,
+ "workerId": worker_id
+ }}])
+
+ state_stream = job_server.GetStateStream(
+ beam_job_api_pb2.GetJobStateRequest(
+ job_id=prepare_response.preparation_id))
+
+ self.assertEqual(
+ [s.state for s in state_stream],
+ [beam_job_api_pb2.JobState.STOPPED,
+ beam_job_api_pb2.JobState.RUNNING,
+ beam_job_api_pb2.JobState.RUNNING,
+ beam_job_api_pb2.JobState.FAILED])
+
+ message_stream = job_server.GetMessageStream(
+ beam_job_api_pb2.JobMessagesRequest(
+ job_id=prepare_response.preparation_id))
+
+ def get_item(x):
+ if x.HasField('message_response'):
+ return x.message_response
+ else:
+ return x.state_response.state
+
+ self.assertEqual(
+ [get_item(m) for m in message_stream],
+ [
+ beam_job_api_pb2.JobState.STOPPED,
+ beam_job_api_pb2.JobState.RUNNING,
+ beam_job_api_pb2.JobMessage(
+ message_id='message0',
+ time='0',
+ importance=beam_job_api_pb2.JobMessage.MessageImportance
+ .JOB_MESSAGE_ERROR,
+ message_text="oops"),
+ beam_job_api_pb2.JobState.FAILED,
+ ])
+
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ unittest.main()
diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py
index 6398810..95c67cf 100644
--- a/sdks/python/apache_beam/runners/worker/bundle_processor.py
+++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py
@@ -1098,6 +1098,12 @@
# type: (beam_runner_api_pb2.PTransform) -> coders.Coder
return only_element(list(self.get_input_coders(transform_proto).values()))
+ def get_input_windowing(self, transform_proto):
+ pcoll_id = only_element(transform_proto.inputs.values())
+ windowing_strategy_id = self.descriptor.pcollections[
+ pcoll_id].windowing_strategy_id
+ return self.context.windowing_strategies.get_by_id(windowing_strategy_id)
+
# TODO(robertwb): Update all operations to take these in the constructor.
@staticmethod
def augment_oldstyle_op(op, # type: OperationT
@@ -1280,8 +1286,8 @@
common_urns.sdf_components.PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS.urn,
beam_runner_api_pb2.ParDoPayload)
def create(factory, transform_id, transform_proto, parameter, consumers):
- assert parameter.do_fn.spec.urn == python_urns.PICKLED_DOFN_INFO
- serialized_fn = parameter.do_fn.spec.payload
+ assert parameter.do_fn.urn == python_urns.PICKLED_DOFN_INFO
+ serialized_fn = parameter.do_fn.payload
return _create_pardo_operation(
factory, transform_id, transform_proto, consumers,
serialized_fn, parameter,
@@ -1292,7 +1298,7 @@
proxy_dofn,
factory, transform_id, transform_proto, parameter, consumers):
- dofn_data = pickler.loads(parameter.do_fn.spec.payload)
+ dofn_data = pickler.loads(parameter.do_fn.payload)
dofn = dofn_data[0]
restriction_provider = common.DoFnSignature(dofn).get_restriction_provider()
serialized_fn = pickler.dumps(
@@ -1305,8 +1311,8 @@
@BeamTransformFactory.register_urn(
common_urns.primitives.PAR_DO.urn, beam_runner_api_pb2.ParDoPayload)
def create(factory, transform_id, transform_proto, parameter, consumers):
- assert parameter.do_fn.spec.urn == python_urns.PICKLED_DOFN_INFO
- serialized_fn = parameter.do_fn.spec.payload
+ assert parameter.do_fn.urn == python_urns.PICKLED_DOFN_INFO
+ serialized_fn = parameter.do_fn.payload
return _create_pardo_operation(
factory, transform_id, transform_proto, consumers,
serialized_fn, parameter)
@@ -1485,7 +1491,8 @@
None,
[factory.get_only_output_coder(transform_proto)]),
factory.counter_factory,
- factory.state_sampler),
+ factory.state_sampler,
+ factory.get_input_windowing(transform_proto)),
transform_proto.unique_name,
consumers)
@@ -1550,10 +1557,10 @@
@BeamTransformFactory.register_urn(
common_urns.primitives.MAP_WINDOWS.urn,
- beam_runner_api_pb2.SdkFunctionSpec)
+ beam_runner_api_pb2.FunctionSpec)
def create(factory, transform_id, transform_proto, mapping_fn_spec, consumers):
- assert mapping_fn_spec.spec.urn == python_urns.PICKLED_WINDOW_MAPPING_FN
- window_mapping_fn = pickler.loads(mapping_fn_spec.spec.payload)
+ assert mapping_fn_spec.urn == python_urns.PICKLED_WINDOW_MAPPING_FN
+ window_mapping_fn = pickler.loads(mapping_fn_spec.payload)
class MapWindows(beam.DoFn):
diff --git a/sdks/python/apache_beam/runners/worker/data_plane.py b/sdks/python/apache_beam/runners/worker/data_plane.py
index 3e87e73..6fb00f3 100644
--- a/sdks/python/apache_beam/runners/worker/data_plane.py
+++ b/sdks/python/apache_beam/runners/worker/data_plane.py
@@ -27,6 +27,7 @@
import queue
import sys
import threading
+import time
from builtins import object
from builtins import range
from typing import TYPE_CHECKING
@@ -56,7 +57,8 @@
_LOGGER = logging.getLogger(__name__)
-_DEFAULT_FLUSH_THRESHOLD = 10 << 20 # 10MB
+_DEFAULT_SIZE_FLUSH_THRESHOLD = 10 << 20 # 10MB
+_DEFAULT_TIME_FLUSH_THRESHOLD_MS = 0 # disable time-based flush by default
if TYPE_CHECKING:
@@ -69,25 +71,117 @@
class ClosableOutputStream(OutputStream):
"""A Outputstream for use with CoderImpls that has a close() method."""
- def __init__(self,
- close_callback=None, # type: Optional[Callable[[bytes], None]]
- flush_callback=None, # type: Optional[Callable[[bytes], None]]
- flush_threshold=_DEFAULT_FLUSH_THRESHOLD):
+ def __init__(self, close_callback=None):
super(ClosableOutputStream, self).__init__()
self._close_callback = close_callback
- self._flush_callback = flush_callback
- self._flush_threshold = flush_threshold
-
- # This must be called explicitly to avoid flushing partial elements.
- def maybe_flush(self):
- if self._flush_callback and self.size() > self._flush_threshold:
- self._flush_callback(self.get())
- self._clear()
def close(self):
if self._close_callback:
self._close_callback(self.get())
+ @staticmethod
+ def create(close_callback,
+ flush_callback,
+ data_buffer_time_limit_ms):
+ if data_buffer_time_limit_ms > 0:
+ return TimeBasedBufferingClosableOutputStream(
+ close_callback,
+ flush_callback=flush_callback,
+ time_flush_threshold_ms=data_buffer_time_limit_ms)
+ else:
+ return SizeBasedBufferingClosableOutputStream(
+ close_callback, flush_callback=flush_callback)
+
+
+class SizeBasedBufferingClosableOutputStream(ClosableOutputStream):
+ """A size-based buffering OutputStream."""
+
+ def __init__(self,
+ close_callback=None, # type: Optional[Callable[[bytes], None]]
+ flush_callback=None, # type: Optional[Callable[[bytes], None]]
+ size_flush_threshold=_DEFAULT_SIZE_FLUSH_THRESHOLD):
+ super(SizeBasedBufferingClosableOutputStream, self).__init__(close_callback)
+ self._flush_callback = flush_callback
+ self._size_flush_threshold = size_flush_threshold
+
+ # This must be called explicitly to avoid flushing partial elements.
+ def maybe_flush(self):
+ if self.size() > self._size_flush_threshold:
+ self.flush()
+
+ def flush(self):
+ if self._flush_callback:
+ self._flush_callback(self.get())
+ self._clear()
+
+
+class TimeBasedBufferingClosableOutputStream(
+ SizeBasedBufferingClosableOutputStream):
+ """A buffering OutputStream with both time-based and size-based."""
+
+ def __init__(self,
+ close_callback=None,
+ flush_callback=None,
+ size_flush_threshold=_DEFAULT_SIZE_FLUSH_THRESHOLD,
+ time_flush_threshold_ms=_DEFAULT_TIME_FLUSH_THRESHOLD_MS):
+ super(TimeBasedBufferingClosableOutputStream, self).__init__(
+ close_callback, flush_callback, size_flush_threshold)
+ assert time_flush_threshold_ms > 0
+ self._time_flush_threshold_ms = time_flush_threshold_ms
+ self._flush_lock = threading.Lock()
+ self._schedule_lock = threading.Lock()
+ self._closed = False
+ self._schedule_periodic_flush()
+
+ def flush(self):
+ with self._flush_lock:
+ super(TimeBasedBufferingClosableOutputStream, self).flush()
+
+ def close(self):
+ with self._schedule_lock:
+ self._closed = True
+ if self._periodic_flusher:
+ self._periodic_flusher.cancel()
+ self._periodic_flusher = None
+ super(TimeBasedBufferingClosableOutputStream, self).close()
+
+ def _schedule_periodic_flush(self):
+ def _flush():
+ with self._schedule_lock:
+ if not self._closed:
+ self.flush()
+
+ self._periodic_flusher = PeriodicThread(
+ self._time_flush_threshold_ms / 1000.0, _flush)
+ self._periodic_flusher.daemon = True
+ self._periodic_flusher.start()
+
+
+class PeriodicThread(threading.Thread):
+ """Call a function periodically with the specified number of seconds"""
+
+ def __init__(self,
+ interval,
+ function,
+ args=None,
+ kwargs=None):
+ threading.Thread.__init__(self)
+ self._interval = interval
+ self._function = function
+ self._args = args if args is not None else []
+ self._kwargs = kwargs if kwargs is not None else {}
+ self._finished = threading.Event()
+
+ def run(self):
+ next_call = time.time() + self._interval
+ while not self._finished.wait(next_call - time.time()):
+ next_call = next_call + self._interval
+ self._function(*self._args, **self._kwargs)
+
+ def cancel(self):
+ """Stop the thread if it hasn't finished yet."""
+ self._finished.set()
+
class DataChannel(with_metaclass(abc.ABCMeta, object)): # type: ignore[misc]
"""Represents a channel for reading and writing data over the data plane.
@@ -163,10 +257,12 @@
The inverse() method returns the other side of a instance.
"""
- def __init__(self, inverse=None):
- # type: (Optional[InMemoryDataChannel]) -> None
+ def __init__(self, inverse=None, data_buffer_time_limit_ms=0):
+ # type: (Optional[InMemoryDataChannel], Optional[int]) -> None
self._inputs = [] # type: List[beam_fn_api_pb2.Elements.Data]
- self._inverse = inverse or InMemoryDataChannel(self)
+ self._data_buffer_time_limit_ms = data_buffer_time_limit_ms
+ self._inverse = inverse or InMemoryDataChannel(
+ self, data_buffer_time_limit_ms=data_buffer_time_limit_ms)
def inverse(self):
# type: () -> InMemoryDataChannel
@@ -195,8 +291,10 @@
instruction_id=instruction_id,
transform_id=transform_id,
data=data))
- return ClosableOutputStream(
- add_to_inverse_output, flush_callback=add_to_inverse_output)
+ return ClosableOutputStream.create(
+ add_to_inverse_output,
+ add_to_inverse_output,
+ self._data_buffer_time_limit_ms)
def close(self):
pass
@@ -207,7 +305,9 @@
_WRITES_FINISHED = object()
- def __init__(self):
+ def __init__(self, data_buffer_time_limit_ms=0):
+ # type: (Optional[int]) -> None
+ self._data_buffer_time_limit_ms = data_buffer_time_limit_ms
self._to_send = queue.Queue() # type: queue.Queue[beam_fn_api_pb2.Elements.Data]
self._received = collections.defaultdict(lambda: queue.Queue(maxsize=5)) # type: DefaultDict[str, queue.Queue[beam_fn_api_pb2.Elements.Data]]
self._receive_lock = threading.Lock()
@@ -292,8 +392,11 @@
instruction_id=instruction_id,
transform_id=transform_id,
data=b''))
- return ClosableOutputStream(
- close_callback, flush_callback=add_to_send_queue)
+
+ return ClosableOutputStream.create(
+ close_callback,
+ add_to_send_queue,
+ self._data_buffer_time_limit_ms)
def _write_outputs(self):
# type: () -> Iterator[beam_fn_api_pb2.Elements]
@@ -340,20 +443,23 @@
"""A DataChannel wrapping the client side of a BeamFnData connection."""
def __init__(self,
- data_stub # type: beam_fn_api_pb2_grpc.BeamFnDataStub
- ):
+ data_stub, # type: beam_fn_api_pb2_grpc.BeamFnDataStub
+ data_buffer_time_limit_ms=0 # type: Optional[int]
+ ):
# type: (...) -> None
- super(GrpcClientDataChannel, self).__init__()
+ super(GrpcClientDataChannel, self).__init__(data_buffer_time_limit_ms)
self.set_inputs(data_stub.Data(self._write_outputs()))
class BeamFnDataServicer(beam_fn_api_pb2_grpc.BeamFnDataServicer):
"""Implementation of BeamFnDataServicer for any number of clients"""
- def __init__(self):
+ def __init__(self,
+ data_buffer_time_limit_ms=0 # type: Optional[int]
+ ):
self._lock = threading.Lock()
self._connections_by_worker_id = collections.defaultdict(
- _GrpcDataChannel) # type: DefaultDict[str, _GrpcDataChannel]
+ lambda: _GrpcDataChannel(data_buffer_time_limit_ms)) # type: DefaultDict[str, _GrpcDataChannel]
def get_conn_by_worker_id(self, worker_id):
# type: (str) -> _GrpcDataChannel
@@ -396,13 +502,15 @@
def __init__(self,
credentials=None,
- worker_id=None # type: Optional[str]
- ):
+ worker_id=None, # type: Optional[str]
+ data_buffer_time_limit_ms=0 # type: Optional[int]
+ ):
# type: (...) -> None
self._data_channel_cache = {} # type: Dict[str, GrpcClientDataChannel]
self._lock = threading.Lock()
self._credentials = None
self._worker_id = worker_id
+ self._data_buffer_time_limit_ms = data_buffer_time_limit_ms
if credentials is not None:
_LOGGER.info('Using secure channel creds.')
self._credentials = credentials
@@ -430,7 +538,8 @@
grpc_channel = grpc.intercept_channel(
grpc_channel, WorkerIdInterceptor(self._worker_id))
self._data_channel_cache[url] = GrpcClientDataChannel(
- beam_fn_api_pb2_grpc.BeamFnDataStub(grpc_channel))
+ beam_fn_api_pb2_grpc.BeamFnDataStub(grpc_channel),
+ self._data_buffer_time_limit_ms)
return self._data_channel_cache[url]
diff --git a/sdks/python/apache_beam/runners/worker/data_plane_test.py b/sdks/python/apache_beam/runners/worker/data_plane_test.py
index 900532b..677758b 100644
--- a/sdks/python/apache_beam/runners/worker/data_plane_test.py
+++ b/sdks/python/apache_beam/runners/worker/data_plane_test.py
@@ -21,6 +21,7 @@
from __future__ import division
from __future__ import print_function
+import itertools
import logging
import sys
import threading
@@ -62,7 +63,18 @@
@timeout(5)
def test_grpc_data_channel(self):
- data_servicer = data_plane.BeamFnDataServicer()
+ self._grpc_data_channel_test()
+
+ @timeout(5)
+ def test_time_based_flush_grpc_data_channel(self):
+ self._grpc_data_channel_test(True)
+
+ def _grpc_data_channel_test(self, time_based_flush=False):
+ if time_based_flush:
+ data_servicer = data_plane.BeamFnDataServicer(
+ data_buffer_time_limit_ms=100)
+ else:
+ data_servicer = data_plane.BeamFnDataServicer()
worker_id = 'worker_0'
data_channel_service = \
data_servicer.get_conn_by_worker_id(worker_id)
@@ -78,10 +90,15 @@
grpc_channel = grpc.intercept_channel(
grpc_channel, WorkerIdInterceptor(worker_id))
data_channel_stub = beam_fn_api_pb2_grpc.BeamFnDataStub(grpc_channel)
- data_channel_client = data_plane.GrpcClientDataChannel(data_channel_stub)
+ if time_based_flush:
+ data_channel_client = data_plane.GrpcClientDataChannel(
+ data_channel_stub, data_buffer_time_limit_ms=100)
+ else:
+ data_channel_client = data_plane.GrpcClientDataChannel(data_channel_stub)
try:
- self._data_channel_test(data_channel_service, data_channel_client)
+ self._data_channel_test(
+ data_channel_service, data_channel_client, time_based_flush)
finally:
data_channel_client.close()
data_channel_service.close()
@@ -92,22 +109,25 @@
channel = data_plane.InMemoryDataChannel()
self._data_channel_test(channel, channel.inverse())
- def _data_channel_test(self, server, client):
- self._data_channel_test_one_direction(server, client)
- self._data_channel_test_one_direction(client, server)
+ def _data_channel_test(self, server, client, time_based_flush=False):
+ self._data_channel_test_one_direction(server, client, time_based_flush)
+ self._data_channel_test_one_direction(client, server, time_based_flush)
- def _data_channel_test_one_direction(self, from_channel, to_channel):
+ def _data_channel_test_one_direction(
+ self, from_channel, to_channel, time_based_flush):
def send(instruction_id, transform_id, data):
stream = from_channel.output_stream(instruction_id, transform_id)
stream.write(data)
- stream.close()
+ if not time_based_flush:
+ stream.close()
transform_1 = '1'
transform_2 = '2'
# Single write.
send('0', transform_1, b'abc')
self.assertEqual(
- list(to_channel.input_elements('0', [transform_1])),
+ list(itertools.islice(
+ to_channel.input_elements('0', [transform_1]), 1)),
[beam_fn_api_pb2.Elements.Data(
instruction_id='0',
transform_id=transform_1,
@@ -117,14 +137,16 @@
send('1', transform_1, b'abc')
send('2', transform_1, b'def')
self.assertEqual(
- list(to_channel.input_elements('1', [transform_1])),
+ list(itertools.islice(
+ to_channel.input_elements('1', [transform_1]), 1)),
[beam_fn_api_pb2.Elements.Data(
instruction_id='1',
transform_id=transform_1,
data=b'abc')])
send('2', transform_2, b'ghi')
self.assertEqual(
- list(to_channel.input_elements('2', [transform_1, transform_2])),
+ list(itertools.islice(
+ to_channel.input_elements('2', [transform_1, transform_2]), 2)),
[beam_fn_api_pb2.Elements.Data(
instruction_id='2',
transform_id=transform_1,
diff --git a/sdks/python/apache_beam/runners/worker/log_handler.py b/sdks/python/apache_beam/runners/worker/log_handler.py
index 1ec798e..4ecd342 100644
--- a/sdks/python/apache_beam/runners/worker/log_handler.py
+++ b/sdks/python/apache_beam/runners/worker/log_handler.py
@@ -162,7 +162,9 @@
# This case is unlikely and the chance of reconnection and successful
# transmission of logs is also very less as the process is terminating.
# I choose not to handle this case to avoid un-necessary code complexity.
- while self._alive:
+
+ alive = True # Force at least one connection attempt.
+ while alive:
# Loop for reconnection.
log_control_iterator = self.connect()
if self._dropped_logs > 0:
@@ -181,3 +183,4 @@
file=sys.stderr)
# Wait a bit before trying a reconnect
time.sleep(0.5) # 0.5 seconds
+ alive = self._alive
diff --git a/sdks/python/apache_beam/runners/worker/operations.pxd b/sdks/python/apache_beam/runners/worker/operations.pxd
index 68da490..672fc4c 100644
--- a/sdks/python/apache_beam/runners/worker/operations.pxd
+++ b/sdks/python/apache_beam/runners/worker/operations.pxd
@@ -111,11 +111,13 @@
cdef public object combine_fn
cdef public object combine_fn_add_input
cdef public object combine_fn_compact
+ cdef public bint is_default_windowing
+ cdef public object timestamp_combiner
cdef dict table
cdef long max_keys
cdef long key_count
- cpdef output_key(self, tuple wkey, value)
+ cpdef output_key(self, wkey, value, timestamp)
cdef class FlattenOperation(Operation):
diff --git a/sdks/python/apache_beam/runners/worker/operations.py b/sdks/python/apache_beam/runners/worker/operations.py
index 33751f3..b4601f1 100644
--- a/sdks/python/apache_beam/runners/worker/operations.py
+++ b/sdks/python/apache_beam/runners/worker/operations.py
@@ -30,7 +30,6 @@
from builtins import object
from builtins import zip
from typing import TYPE_CHECKING
-from typing import Any
from typing import DefaultDict
from typing import Dict
from typing import FrozenSet
@@ -38,7 +37,6 @@
from typing import Iterator
from typing import List
from typing import Optional
-from typing import Tuple
from typing import Union
from apache_beam import pvalue
@@ -58,6 +56,7 @@
from apache_beam.transforms import combiners
from apache_beam.transforms import core
from apache_beam.transforms import userstate
+from apache_beam.transforms import window
from apache_beam.transforms.combiners import PhasedCombineFnExecutor
from apache_beam.transforms.combiners import curry_combine_fn
from apache_beam.transforms.window import GlobalWindows
@@ -888,7 +887,8 @@
class PGBKCVOperation(Operation):
- def __init__(self, name_context, spec, counter_factory, state_sampler):
+ def __init__(
+ self, name_context, spec, counter_factory, state_sampler, windowing=None):
super(PGBKCVOperation, self).__init__(
name_context, spec, counter_factory, state_sampler)
# Combiners do not accept deferred side-inputs (the ignored fourth
@@ -904,6 +904,15 @@
self.combine_fn_compact = None
else:
self.combine_fn_compact = self.combine_fn.compact
+ if windowing:
+ self.is_default_windowing = windowing.is_default()
+ tsc_type = windowing.timestamp_combiner
+ self.timestamp_combiner = (
+ None if tsc_type == window.TimestampCombiner.OUTPUT_AT_EOW
+ else window.TimestampCombiner.get_impl(tsc_type, windowing.windowfn))
+ else:
+ self.is_default_windowing = False # unknown
+ self.timestamp_combiner = None
# Optimization for the (known tiny accumulator, often wide keyspace)
# combine functions.
# TODO(b/36567833): Bound by in-memory size rather than key count.
@@ -923,8 +932,8 @@
key, value = wkv.value
# pylint: disable=unidiomatic-typecheck
# Optimization for the global window case.
- if len(wkv.windows) == 1 and type(wkv.windows[0]) is _global_window_type:
- wkey = 0, key # type: Tuple[Hashable, Any]
+ if self.is_default_windowing:
+ wkey = key # type: Hashable
else:
wkey = tuple(wkv.windows), key
entry = self.table.get(wkey, None)
@@ -935,7 +944,7 @@
# TODO(robertwb): Use an LRU cache?
for old_wkey, old_wvalue in self.table.items():
old_wkeys.append(old_wkey) # Can't mutate while iterating.
- self.output_key(old_wkey, old_wvalue[0])
+ self.output_key(old_wkey, old_wvalue[0], old_wvalue[1])
self.key_count -= 1
if self.key_count <= target:
break
@@ -944,26 +953,33 @@
self.key_count += 1
# We save the accumulator as a one element list so we can efficiently
# mutate when new values are added without searching the cache again.
- entry = self.table[wkey] = [self.combine_fn.create_accumulator()]
+ entry = self.table[wkey] = [self.combine_fn.create_accumulator(), None]
+ if not self.is_default_windowing:
+ # Conditional as the timestamp attribute is lazily initialized.
+ entry[1] = wkv.timestamp
entry[0] = self.combine_fn_add_input(entry[0], value)
+ if not self.is_default_windowing and self.timestamp_combiner:
+ entry[1] = self.timestamp_combiner.combine(entry[1], wkv.timestamp)
def finish(self):
for wkey, value in self.table.items():
- self.output_key(wkey, value[0])
+ self.output_key(wkey, value[0], value[1])
self.table = {}
self.key_count = 0
- def output_key(self, wkey, accumulator):
- windows, key = wkey
+ def output_key(self, wkey, accumulator, timestamp):
if self.combine_fn_compact is None:
value = accumulator
else:
value = self.combine_fn_compact(accumulator)
- if windows == 0:
- self.output(_globally_windowed_value.with_value((key, value)))
+
+ if self.is_default_windowing:
+ self.output(_globally_windowed_value.with_value((wkey, value)))
else:
- self.output(
- WindowedValue((key, value), windows[0].max_timestamp(), windows))
+ windows, key = wkey
+ if self.timestamp_combiner is None:
+ timestamp = windows[0].max_timestamp()
+ self.output(WindowedValue((key, value), timestamp, windows))
class FlattenOperation(Operation):
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py
index a9853ca..3eefb17 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py
@@ -29,6 +29,7 @@
import threading
import traceback
from builtins import object
+from concurrent import futures
from typing import TYPE_CHECKING
from typing import Callable
from typing import DefaultDict
@@ -74,8 +75,10 @@
worker_id=None, # type: Optional[str]
# Caching is disabled by default
state_cache_size=0,
+ # time-based data buffering is disabled by default
+ data_buffer_time_limit_ms=0,
profiler_factory=None # type: Optional[Callable[..., Profile]]
- ):
+ ):
self._alive = True
self._worker_index = 0
self._worker_id = worker_id
@@ -94,7 +97,7 @@
self._control_channel = grpc.intercept_channel(
self._control_channel, WorkerIdInterceptor(self._worker_id))
self._data_channel_factory = data_plane.GrpcClientDataChannelFactory(
- credentials, self._worker_id)
+ credentials, self._worker_id, data_buffer_time_limit_ms)
self._state_handler_factory = GrpcStateHandlerFactory(self._state_cache,
credentials)
self._profiler_factory = profiler_factory
@@ -104,6 +107,10 @@
state_handler_factory=self._state_handler_factory,
data_channel_factory=self._data_channel_factory,
fns=self._fns)
+
+ # TODO(BEAM-8998) use common UnboundedThreadPoolExecutor to process bundle
+ # progress once dataflow runner's excessive progress polling is removed.
+ self._report_progress_executor = futures.ThreadPoolExecutor(max_workers=1)
self._worker_thread_pool = UnboundedThreadPoolExecutor()
self._responses = queue.Queue() # type: queue.Queue[beam_fn_api_pb2.InstructionResponse]
_LOGGER.info('Initializing SDKHarness with unbounded number of workers.')
@@ -199,7 +206,7 @@
'Unknown process bundle instruction {}').format(
instruction_id)), request)
- self._worker_thread_pool.submit(task)
+ self._report_progress_executor.submit(task)
def _request_finalize_bundle(self, request):
# type: (beam_fn_api_pb2.InstructionRequest) -> None
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
index 71ed8e5..99921b8 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
@@ -54,7 +54,8 @@
for t in threading.enumerate():
lines.append('--- Thread #%s name: %s ---\n' % (t.ident, t.name))
- lines.append(''.join(traceback.format_stack(frames[t.ident])))
+ if t.ident in frames:
+ lines.append(''.join(traceback.format_stack(frames[t.ident])))
return lines
@@ -151,6 +152,8 @@
control_address=service_descriptor.url,
worker_id=_worker_id,
state_cache_size=_get_state_cache_size(sdk_pipeline_options),
+ data_buffer_time_limit_ms=_get_data_buffer_time_limit_ms(
+ sdk_pipeline_options),
profiler_factory=profiler.Profile.factory_from_options(
sdk_pipeline_options.view_as(ProfilingOptions))
).run()
@@ -200,6 +203,29 @@
return 0
+def _get_data_buffer_time_limit_ms(pipeline_options):
+ """Defines the time limt of the outbound data buffering.
+
+ Note: data_buffer_time_limit_ms is an experimental flag and might
+ not be available in future releases.
+
+ Returns:
+ an int indicating the time limit in milliseconds of the the outbound
+ data buffering. Default is 0 (disabled)
+ """
+ experiments = pipeline_options.view_as(DebugOptions).experiments
+ experiments = experiments if experiments else []
+
+ for experiment in experiments:
+ # There should only be 1 match so returning from the loop
+ if re.match(r'data_buffer_time_limit_ms=', experiment):
+ return int(
+ re.match(
+ r'data_buffer_time_limit_ms=(?P<data_buffer_time_limit_ms>.*)',
+ experiment).group('data_buffer_time_limit_ms'))
+ return 0
+
+
def _load_main_session(semi_persistent_directory):
"""Loads a pickled main session from the path specified."""
if semi_persistent_directory:
diff --git a/sdks/python/apache_beam/runners/worker/worker_pool_main.py b/sdks/python/apache_beam/runners/worker/worker_pool_main.py
index 1aad3ed..117f38b 100644
--- a/sdks/python/apache_beam/runners/worker/worker_pool_main.py
+++ b/sdks/python/apache_beam/runners/worker/worker_pool_main.py
@@ -55,11 +55,13 @@
def __init__(self,
use_process=False,
container_executable=None, # type: Optional[str]
- state_cache_size=0
+ state_cache_size=0,
+ data_buffer_time_limit_ms=0
):
self._use_process = use_process
self._container_executable = container_executable
self._state_cache_size = state_cache_size
+ self._data_buffer_time_limit_ms = data_buffer_time_limit_ms
self._worker_processes = {} # type: Dict[str, subprocess.Popen]
@classmethod
@@ -67,15 +69,17 @@
use_process=False,
port=0,
state_cache_size=0,
+ data_buffer_time_limit_ms=-1,
container_executable=None # type: Optional[str]
- ):
+ ):
# type: (...) -> Tuple[str, grpc.Server]
worker_server = grpc.server(UnboundedThreadPoolExecutor())
worker_address = 'localhost:%s' % worker_server.add_insecure_port(
'[::]:%s' % port)
worker_pool = cls(use_process=use_process,
container_executable=container_executable,
- state_cache_size=state_cache_size)
+ state_cache_size=state_cache_size,
+ data_buffer_time_limit_ms=data_buffer_time_limit_ms)
beam_fn_api_pb2_grpc.add_BeamFnExternalWorkerPoolServicer_to_server(
worker_pool,
worker_server)
@@ -103,11 +107,13 @@
'"%s",'
'worker_id="%s",'
'state_cache_size=%d'
+ 'data_buffer_time_limit_ms=%d'
')'
'.run()' % (
start_worker_request.control_endpoint.url,
start_worker_request.worker_id,
- self._state_cache_size)]
+ self._state_cache_size,
+ self._data_buffer_time_limit_ms)]
if self._container_executable:
# command as per container spec
# the executable is responsible to handle concurrency
@@ -132,7 +138,8 @@
worker = sdk_worker.SdkHarness(
start_worker_request.control_endpoint.url,
worker_id=start_worker_request.worker_id,
- state_cache_size=self._state_cache_size)
+ state_cache_size=self._state_cache_size,
+ data_buffer_time_limit_ms=self._data_buffer_time_limit_ms)
worker_thread = threading.Thread(
name='run_worker_%s' % start_worker_request.worker_id,
target=worker.run)
diff --git a/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml b/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
index cac0c74..b2d4e9a 100644
--- a/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
+++ b/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
@@ -30,6 +30,7 @@
window_fn: FixedWindows(10)
trigger_fn: Default
timestamp_combiner: OUTPUT_AT_EOW
+allowed_lateness: 100
transcript:
- input: [1, 2, 3, 10, 11, 25]
- watermark: 100
@@ -42,6 +43,26 @@
- {window: [0, 9], values: [1, 2, 3, 7], timestamp: 9, late: true}
---
+name: fixed_drop_late_data_after_allowed_lateness
+window_fn: FixedWindows(10)
+trigger_fn: AfterWatermark(early=AfterCount(3), late=AfterCount(1))
+timestamp_combiner: OUTPUT_AT_EOW
+allowed_lateness: 20
+accumulation_mode: accumulating
+transcript:
+ - input: [1, 2, 10, 11, 80, 81]
+ - watermark: 100
+ - expect:
+ - {window: [0, 9], values: [1, 2], timestamp: 9, final: false}
+ - {window: [10, 19], values: [10, 11], timestamp: 19}
+ - {window: [80, 89], values: [80, 81], timestamp: 89, late: false}
+ - input: [7, 8] # no output
+ - input: [17, 18] # no output
+ - input: [82]
+ - expect:
+ - {window: [80, 89], values: [80, 81, 82], timestamp: 89, late: true}
+
+---
name: timestamp_combiner_earliest
window_fn: FixedWindows(10)
trigger_fn: Default
@@ -118,6 +139,7 @@
- SwitchingDirectRunner
window_fn: Sessions(10)
trigger_fn: AfterWatermark(early=AfterCount(2), late=AfterCount(3))
+allowed_lateness: 100
timestamp_combiner: OUTPUT_AT_EOW
transcript:
- input: [1, 2, 3]
diff --git a/sdks/python/apache_beam/testing/load_tests/build.gradle b/sdks/python/apache_beam/testing/load_tests/build.gradle
index 8eba17f..543d8c4 100644
--- a/sdks/python/apache_beam/testing/load_tests/build.gradle
+++ b/sdks/python/apache_beam/testing/load_tests/build.gradle
@@ -24,8 +24,6 @@
distTarBall project(path: ":sdks:python", configuration: "distTarBall")
}
-pythonVersion = '2.7'
-
description = "Apache Beam :: SDKs :: Python :: Load Tests"
def mainClassProperty = "loadTest.mainClass"
@@ -59,4 +57,4 @@
def parseOptions(String option) {
option.replace('\"', '\\"')
-}
\ No newline at end of file
+}
diff --git a/sdks/python/apache_beam/testing/test_stream.py b/sdks/python/apache_beam/testing/test_stream.py
index 8429f20..ff54fe2 100644
--- a/sdks/python/apache_beam/testing/test_stream.py
+++ b/sdks/python/apache_beam/testing/test_stream.py
@@ -28,7 +28,6 @@
from future.utils import with_metaclass
-import apache_beam as beam
from apache_beam import coders
from apache_beam import pvalue
from apache_beam.portability import common_urns
@@ -171,47 +170,24 @@
output.
"""
- def __init__(self, coder=coders.FastPrimitivesCoder(), events=()):
+ def __init__(self, coder=coders.FastPrimitivesCoder(), events=None):
super(TestStream, self).__init__()
assert coder is not None
self.coder = coder
self.watermarks = {None: timestamp.MIN_TIMESTAMP}
- self._events = list(events)
+ self._events = [] if events is None else list(events)
self.output_tags = set()
def get_windowing(self, unused_inputs):
return core.Windowing(window.GlobalWindows())
+ def _infer_output_coder(self, input_type=None, input_coder=None):
+ return self.coder
+
def expand(self, pbegin):
assert isinstance(pbegin, pvalue.PBegin)
self.pipeline = pbegin.pipeline
-
- # This multiplexing the multiple output PCollections.
- def mux(event):
- if event.tag:
- yield pvalue.TaggedOutput(event.tag, event)
- else:
- yield event
- mux_output = (pbegin
- | _TestStream(self.output_tags, events=self._events)
- | 'TestStream Multiplexer' >> beam.ParDo(mux).with_outputs())
-
- # Apply a way to control the watermark per output. It is necessary to
- # have an individual _WatermarkController per PCollection because the
- # calculation of the input watermark of a transform is based on the event
- # timestamp of the elements flowing through it. Meaning, it is impossible
- # to control the output watermarks of the individual PCollections solely
- # on the event timestamps.
- outputs = {}
- for tag in self.output_tags:
- label = '_WatermarkController[{}]'.format(tag)
- outputs[tag] = (mux_output[tag] | label >> _WatermarkController())
-
- # Downstream consumers expect a PCollection if there is only a single
- # output.
- if len(outputs) == 1:
- return list(outputs.values())[0]
- return outputs
+ return pvalue.PCollection(self.pipeline, is_bounded=False)
def _add(self, event):
if isinstance(event, ElementEvent):
@@ -303,141 +279,3 @@
return TestStream(
coder=coder,
events=[Event.from_runner_api(e, coder) for e in payload.events])
-
-
-class _WatermarkController(PTransform):
- """A runner-overridable PTransform Primitive to control the watermark.
-
- Expected implementation behavior:
- - If the instance recieves a WatermarkEvent, it sets its output watermark to
- the specified value then drops the event.
- - If the instance receives an ElementEvent, it emits all specified elements
- to the Global Window with the event time set to the element's timestamp.
- """
- def get_windowing(self, _):
- return core.Windowing(window.GlobalWindows())
-
- def expand(self, pcoll):
- return pvalue.PCollection.from_(pcoll)
-
-
-class _TestStream(PTransform):
- """Test stream that generates events on an unbounded PCollection of elements.
-
- Each event emits elements, advances the watermark or advances the processing
- time. After all of the specified elements are emitted, ceases to produce
- output.
-
- Expected implementation behavior:
- - If the instance receives a WatermarkEvent with the WATERMARK_CONTROL_TAG
- then the instance sets its own watermark hold at the specified value and
- drops the event.
- - If the instance receives any other WatermarkEvent or ElementEvent, it
- passes it to the consumer.
- """
-
- # This tag is used on WatermarkEvents to control the watermark at the root
- # TestStream.
- WATERMARK_CONTROL_TAG = '_TestStream_Watermark'
-
- def __init__(self, output_tags, coder=coders.FastPrimitivesCoder(),
- events=None):
- assert coder is not None
- self.coder = coder
- self._events = self._add_watermark_advancements(output_tags, events)
-
- def _watermark_starts(self, output_tags):
- """Sentinel values to hold the watermark of outputs to -inf.
-
- The output watermarks of the output PCollections (fake unbounded sources) in
- a TestStream are controlled by watermark holds. This sets the hold of each
- output PCollection so that the individual holds can be controlled by the
- given events.
- """
- return [WatermarkEvent(timestamp.MIN_TIMESTAMP, tag) for tag in output_tags]
-
- def _watermark_stops(self, output_tags):
- """Sentinel values to close the watermark of outputs."""
- return [WatermarkEvent(timestamp.MAX_TIMESTAMP, tag) for tag in output_tags]
-
- def _test_stream_start(self):
- """Sentinel value to move the watermark hold of the TestStream to +inf.
-
- This sets a hold to +inf such that the individual holds of the output
- PCollections are allowed to modify their individial output watermarks with
- their holds. This is because the calculation of the output watermark is a
- min over all input watermarks.
- """
- return [WatermarkEvent(timestamp.MAX_TIMESTAMP - timestamp.TIME_GRANULARITY,
- _TestStream.WATERMARK_CONTROL_TAG)]
-
- def _test_stream_stop(self):
- """Sentinel value to close the watermark of the TestStream."""
- return [WatermarkEvent(timestamp.MAX_TIMESTAMP,
- _TestStream.WATERMARK_CONTROL_TAG)]
-
- def _test_stream_init(self):
- """Sentinel value to hold the watermark of the TestStream to -inf.
-
- This sets a hold to ensure that the output watermarks of the output
- PCollections do not advance to +inf before their watermark holds are set.
- """
- return [WatermarkEvent(timestamp.MIN_TIMESTAMP,
- _TestStream.WATERMARK_CONTROL_TAG)]
-
- def _set_up(self, output_tags):
- return (self._test_stream_init()
- + self._watermark_starts(output_tags)
- + self._test_stream_start())
-
- def _tear_down(self, output_tags):
- return self._watermark_stops(output_tags) + self._test_stream_stop()
-
- def _add_watermark_advancements(self, output_tags, events):
- """Adds watermark advancements to the given events.
-
- The following watermark advancements can be done on the runner side.
- However, it makes the logic on the runner side much more complicated than
- it needs to be.
-
- In order for watermarks to be properly advanced in a TestStream, a specific
- sequence of watermark holds must be sent:
-
- 1. Hold the root watermark at -inf (this prevents the pipeline from
- immediately returning).
- 2. Hold the watermarks at the WatermarkControllerss at -inf (this prevents
- the pipeline from immediately returning).
- 3. Advance the root watermark to +inf - 1 (this allows the downstream
- WatermarkControllers to control their watermarks via holds).
- 4. Advance watermarks as normal.
- 5. Advance WatermarkController watermarks to +inf
- 6. Advance root watermark to +inf.
- """
- if not events:
- return []
-
- return self._set_up(output_tags) + events + self._tear_down(output_tags)
-
- def get_windowing(self, unused_inputs):
- return core.Windowing(window.GlobalWindows())
-
- def expand(self, pcoll):
- return pvalue.PCollection(pcoll.pipeline, is_bounded=False)
-
- def _infer_output_coder(self, input_type=None, input_coder=None):
- return self.coder
-
- def _events_from_script(self, index):
- yield self._events[index]
-
- def events(self, index):
- return self._events_from_script(index)
-
- def begin(self):
- return 0
-
- def end(self, index):
- return index >= len(self._events)
-
- def next(self, index):
- return index + 1
diff --git a/sdks/python/apache_beam/testing/test_stream_it_test.py b/sdks/python/apache_beam/testing/test_stream_it_test.py
new file mode 100644
index 0000000..8e724f6
--- /dev/null
+++ b/sdks/python/apache_beam/testing/test_stream_it_test.py
@@ -0,0 +1,237 @@
+#
+# 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.
+#
+
+"""Integration tests for the test_stream module."""
+
+from __future__ import absolute_import
+
+import unittest
+from functools import wraps
+
+from nose.plugins.attrib import attr
+
+import apache_beam as beam
+from apache_beam.options.pipeline_options import StandardOptions
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.test_stream import TestStream
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+from apache_beam.testing.util import equal_to_per_window
+from apache_beam.transforms import trigger
+from apache_beam.transforms import window
+from apache_beam.transforms.window import FixedWindows
+from apache_beam.transforms.window import TimestampedValue
+from apache_beam.utils import timestamp
+from apache_beam.utils.timestamp import Timestamp
+
+
+def supported(runners):
+ if not isinstance(runners, list):
+ runners = [runners]
+
+ def inner(fn):
+ @wraps(fn)
+ def wrapped(self):
+ if self.runner_name not in runners:
+ self.skipTest('The "{}", does not support the TestStream transform. Supported runners: {}'.format(
+ self.runner_name, runners))
+ else:
+ return fn(self)
+ return wrapped
+ return inner
+
+
+class TestStreamIntegrationTests(unittest.TestCase):
+
+ @classmethod
+ def setUpClass(cls):
+ cls.test_pipeline = TestPipeline(is_integration_test=True)
+ cls.args = cls.test_pipeline.get_full_options_as_args()
+ cls.runner_name = type(cls.test_pipeline.runner).__name__
+ cls.project = cls.test_pipeline.get_option('project')
+
+ @supported(['DirectRunner', 'SwitchingDirectRunner'])
+ @attr('IT')
+ def test_basic_execution(self):
+ test_stream = (TestStream()
+ .advance_watermark_to(10)
+ .add_elements(['a', 'b', 'c'])
+ .advance_watermark_to(20)
+ .add_elements(['d'])
+ .add_elements(['e'])
+ .advance_processing_time(10)
+ .advance_watermark_to(300)
+ .add_elements([TimestampedValue('late', 12)])
+ .add_elements([TimestampedValue('last', 310)])
+ .advance_watermark_to_infinity())
+
+ class RecordFn(beam.DoFn):
+
+ def process(self, element=beam.DoFn.ElementParam,
+ timestamp=beam.DoFn.TimestampParam):
+ yield (element, timestamp)
+
+ with beam.Pipeline(argv=self.args) as p:
+ my_record_fn = RecordFn()
+ records = p | test_stream | beam.ParDo(my_record_fn)
+
+ assert_that(records, equal_to([
+ ('a', timestamp.Timestamp(10)),
+ ('b', timestamp.Timestamp(10)),
+ ('c', timestamp.Timestamp(10)),
+ ('d', timestamp.Timestamp(20)),
+ ('e', timestamp.Timestamp(20)),
+ ('late', timestamp.Timestamp(12)),
+ ('last', timestamp.Timestamp(310)),]))
+
+ @supported(['DirectRunner', 'SwitchingDirectRunner'])
+ @attr('IT')
+ def test_multiple_outputs(self):
+ """Tests that the TestStream supports emitting to multiple PCollections."""
+ letters_elements = [
+ TimestampedValue('a', 6),
+ TimestampedValue('b', 7),
+ TimestampedValue('c', 8),
+ ]
+ numbers_elements = [
+ TimestampedValue('1', 11),
+ TimestampedValue('2', 12),
+ TimestampedValue('3', 13),
+ ]
+ test_stream = (TestStream()
+ .advance_watermark_to(5, tag='letters')
+ .add_elements(letters_elements, tag='letters')
+ .advance_watermark_to(10, tag='numbers')
+ .add_elements(numbers_elements, tag='numbers'))
+
+ class RecordFn(beam.DoFn):
+ def process(self, element=beam.DoFn.ElementParam,
+ timestamp=beam.DoFn.TimestampParam):
+ yield (element, timestamp)
+
+ options = StandardOptions(streaming=True)
+ p = TestPipeline(is_integration_test=True, options=options)
+
+ main = p | test_stream
+ letters = main['letters'] | 'record letters' >> beam.ParDo(RecordFn())
+ numbers = main['numbers'] | 'record numbers' >> beam.ParDo(RecordFn())
+
+ assert_that(letters, equal_to([
+ ('a', Timestamp(6)),
+ ('b', Timestamp(7)),
+ ('c', Timestamp(8))]), label='assert letters')
+
+ assert_that(numbers, equal_to([
+ ('1', Timestamp(11)),
+ ('2', Timestamp(12)),
+ ('3', Timestamp(13))]), label='assert numbers')
+
+ p.run()
+
+ @supported(['DirectRunner', 'SwitchingDirectRunner'])
+ @attr('IT')
+ def test_multiple_outputs_with_watermark_advancement(self):
+ """Tests that the TestStream can independently control output watermarks."""
+
+ # Purposely set the watermark of numbers to 20 then letters to 5 to test
+ # that the watermark advancement is per PCollection.
+ #
+ # This creates two PCollections, (a, b, c) and (1, 2, 3). These will be
+ # emitted at different times so that they will have different windows. The
+ # watermark advancement is checked by checking their windows. If the
+ # watermark does not advance, then the windows will be [-inf, -inf). If the
+ # windows do not advance separately, then the PCollections will both
+ # windowed in [15, 30).
+ letters_elements = [
+ TimestampedValue('a', 6),
+ TimestampedValue('b', 7),
+ TimestampedValue('c', 8),
+ ]
+ numbers_elements = [
+ TimestampedValue('1', 21),
+ TimestampedValue('2', 22),
+ TimestampedValue('3', 23),
+ ]
+ test_stream = (TestStream()
+ .advance_watermark_to(0, tag='letters')
+ .advance_watermark_to(0, tag='numbers')
+ .advance_watermark_to(20, tag='numbers')
+ .advance_watermark_to(5, tag='letters')
+ .add_elements(letters_elements, tag='letters')
+ .advance_watermark_to(10, tag='letters')
+ .add_elements(numbers_elements, tag='numbers')
+ .advance_watermark_to(30, tag='numbers'))
+
+ options = StandardOptions(streaming=True)
+ p = TestPipeline(is_integration_test=True, options=options)
+
+ main = p | test_stream
+
+ # Use an AfterWatermark trigger with an early firing to test that the
+ # watermark is advancing properly and that the element is being emitted in
+ # the correct window.
+ letters = (main['letters']
+ | 'letter windows' >> beam.WindowInto(
+ FixedWindows(15),
+ trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)),
+ accumulation_mode=trigger.AccumulationMode.DISCARDING)
+ | 'letter with key' >> beam.Map(lambda x: ('k', x))
+ | 'letter gbk' >> beam.GroupByKey())
+
+ numbers = (main['numbers']
+ | 'number windows' >> beam.WindowInto(
+ FixedWindows(15),
+ trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)),
+ accumulation_mode=trigger.AccumulationMode.DISCARDING)
+ | 'number with key' >> beam.Map(lambda x: ('k', x))
+ | 'number gbk' >> beam.GroupByKey())
+
+ # The letters were emitted when the watermark was at 5, thus we expect to
+ # see the elements in the [0, 15) window. We used an early trigger to make
+ # sure that the ON_TIME empty pane was also emitted with a TestStream.
+ # This pane has no data because of the early trigger causes the elements to
+ # fire before the end of the window and because the accumulation mode
+ # discards any data after the trigger fired.
+ expected_letters = {
+ window.IntervalWindow(0, 15): [
+ ('k', ['a', 'b', 'c']),
+ ('k', []),
+ ],
+ }
+
+ # Same here, except the numbers were emitted at watermark = 20, thus they
+ # are in the [15, 30) window.
+ expected_numbers = {
+ window.IntervalWindow(15, 30): [
+ ('k', ['1', '2', '3']),
+ ('k', []),
+ ],
+ }
+ assert_that(
+ letters,
+ equal_to_per_window(expected_letters),
+ label='letters assert per window')
+ assert_that(
+ numbers,
+ equal_to_per_window(expected_numbers),
+ label='numbers assert per window')
+
+ p.run()
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/sdks/python/apache_beam/testing/test_stream_service.py b/sdks/python/apache_beam/testing/test_stream_service.py
new file mode 100644
index 0000000..8819073
--- /dev/null
+++ b/sdks/python/apache_beam/testing/test_stream_service.py
@@ -0,0 +1,54 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
+
+from concurrent.futures import ThreadPoolExecutor
+
+import grpc
+
+from apache_beam.portability.api import beam_runner_api_pb2_grpc
+from apache_beam.portability.api.beam_runner_api_pb2_grpc import TestStreamServiceServicer
+
+
+class TestStreamServiceController(TestStreamServiceServicer):
+ def __init__(self, events, endpoint=None):
+ self._server = grpc.server(ThreadPoolExecutor(max_workers=10))
+
+ if endpoint:
+ self.endpoint = endpoint
+ self._server.add_insecure_port(self.endpoint)
+ else:
+ port = self._server.add_insecure_port('[::]:0')
+ self.endpoint = '[::]:{}'.format(port)
+
+ beam_runner_api_pb2_grpc.add_TestStreamServiceServicer_to_server(
+ self, self._server)
+ self._events = events
+
+ def start(self):
+ self._server.start()
+
+ def stop(self):
+ self._server.stop(0)
+ self._server.wait_for_termination()
+
+ def Events(self, request, context):
+ """Streams back all of the events from the streaming cache."""
+
+ for e in self._events:
+ yield e
diff --git a/sdks/python/apache_beam/testing/test_stream_service_test.py b/sdks/python/apache_beam/testing/test_stream_service_test.py
new file mode 100644
index 0000000..2b25100
--- /dev/null
+++ b/sdks/python/apache_beam/testing/test_stream_service_test.py
@@ -0,0 +1,96 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
+
+import unittest
+
+import grpc
+
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.portability.api import beam_runner_api_pb2_grpc
+from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileHeader
+from apache_beam.portability.api.beam_interactive_api_pb2 import TestStreamFileRecord
+from apache_beam.portability.api.beam_runner_api_pb2 import TestStreamPayload
+from apache_beam.testing.test_stream_service import TestStreamServiceController
+
+# Nose automatically detects tests if they match a regex. Here, it mistakens
+# these protos as tests. For more info see the Nose docs at:
+# https://nose.readthedocs.io/en/latest/writing_tests.html
+TestStreamPayload.__test__ = False
+TestStreamFileHeader.__test__ = False
+TestStreamFileRecord.__test__ = False
+
+
+class TestStreamServiceTest(unittest.TestCase):
+ def events(self):
+ events = []
+ for i in range(10):
+ e = TestStreamPayload.Event()
+ e.element_event.elements.append(
+ TestStreamPayload.TimestampedElement(timestamp=i))
+ events.append(e)
+ return events
+
+ def setUp(self):
+ self.controller = TestStreamServiceController(self.events())
+ self.controller.start()
+
+ channel = grpc.insecure_channel(self.controller.endpoint)
+ self.stub = beam_runner_api_pb2_grpc.TestStreamServiceStub(channel)
+
+ def tearDown(self):
+ self.controller.stop()
+
+ def test_normal_run(self):
+ r = self.stub.Events(beam_runner_api_pb2.EventsRequest())
+ events = [e for e in r]
+ expected_events = [e for e in self.events()]
+
+ self.assertEqual(events, expected_events)
+
+ def test_multiple_sessions(self):
+ resp_a = self.stub.Events(beam_runner_api_pb2.EventsRequest())
+ resp_b = self.stub.Events(beam_runner_api_pb2.EventsRequest())
+
+ events_a = []
+ events_b = []
+
+ done = False
+ while not done:
+ a_is_done = False
+ b_is_done = False
+ try:
+ events_a.append(next(resp_a))
+ except StopIteration:
+ a_is_done = True
+
+ try:
+ events_b.append(next(resp_b))
+ except StopIteration:
+ b_is_done = True
+
+ done = a_is_done and b_is_done
+
+ expected_events = [e for e in self.events()]
+
+ self.assertEqual(events_a, expected_events)
+ self.assertEqual(events_b, expected_events)
+
+
+if __name__ == '__main__':
+ unittest.main()
diff --git a/sdks/python/apache_beam/testing/test_stream_test.py b/sdks/python/apache_beam/testing/test_stream_test.py
index 26b54bd..bfadb5e 100644
--- a/sdks/python/apache_beam/testing/test_stream_test.py
+++ b/sdks/python/apache_beam/testing/test_stream_test.py
@@ -275,7 +275,7 @@
p = TestPipeline(options=options)
records = (p
| test_stream
- | beam.WindowInto(FixedWindows(15))
+ | beam.WindowInto(FixedWindows(15), allowed_lateness=300)
| beam.Map(lambda x: ('k', x))
| beam.GroupByKey())
diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 9e2bda5..25cc91f 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -64,6 +64,7 @@
from apache_beam.typehints.typehints import is_consistent_with
from apache_beam.utils import timestamp
from apache_beam.utils import urns
+from apache_beam.utils.timestamp import Duration
if typing.TYPE_CHECKING:
from google.protobuf import message # pylint: disable=ungrouped-imports
@@ -610,7 +611,7 @@
**kwargs: other keyword arguments.
Returns:
- An Iterable of output elements.
+ An Iterable of output elements or None.
"""
raise NotImplementedError
@@ -1303,11 +1304,7 @@
return (
common_urns.primitives.PAR_DO.urn,
beam_runner_api_pb2.ParDoPayload(
- do_fn=beam_runner_api_pb2.SdkFunctionSpec(
- environment_id=context.default_environment_id(),
- spec=beam_runner_api_pb2.FunctionSpec(
- urn=python_urns.PICKLED_DOFN_INFO,
- payload=picked_pardo_fn_data)),
+ do_fn=beam_runner_api_pb2.FunctionSpec(urn=python_urns.PICKLED_DOFN_INFO, payload=picked_pardo_fn_data),
splittable=is_splittable,
restriction_coder_id=restriction_coder_id,
state_specs={spec.name: spec.to_runner_api(context)
@@ -1326,9 +1323,9 @@
@PTransform.register_urn(
common_urns.primitives.PAR_DO.urn, beam_runner_api_pb2.ParDoPayload)
def from_runner_api_parameter(pardo_payload, context):
- assert pardo_payload.do_fn.spec.urn == python_urns.PICKLED_DOFN_INFO
+ assert pardo_payload.do_fn.urn == python_urns.PICKLED_DOFN_INFO
fn, args, kwargs, si_tags_and_types, windowing = pickler.loads(
- pardo_payload.do_fn.spec.payload)
+ pardo_payload.do_fn.payload)
if si_tags_and_types:
raise NotImplementedError('explicit side input data')
elif windowing:
@@ -2273,7 +2270,21 @@
triggerfn=None, # type: typing.Optional[TriggerFn]
accumulation_mode=None, # typing.Optional[beam_runner_api_pb2.AccumulationMode]
timestamp_combiner=None, # typing.Optional[beam_runner_api_pb2.OutputTime]
- ):
+ allowed_lateness=0, # type: typing.Union[int, float]
+ ):
+ """Class representing the window strategy.
+
+ Args:
+ windowfn: Window assign function.
+ triggerfn: Trigger function.
+ accumulation_mode: a AccumulationMode, controls what to do with data
+ when a trigger fires multiple times.
+ timestamp_combiner: a TimestampCombiner, determines how output
+ timestamps of grouping operations are assigned.
+ allowed_lateness: Maximum delay in seconds after end of window
+ allowed for any late data to be processed without being discarded
+ directly.
+ """
global AccumulationMode, DefaultTrigger # pylint: disable=global-variable-not-assigned
# pylint: disable=wrong-import-order, wrong-import-position
from apache_beam.transforms.trigger import AccumulationMode, DefaultTrigger
@@ -2293,13 +2304,15 @@
self.windowfn = windowfn
self.triggerfn = triggerfn
self.accumulation_mode = accumulation_mode
+ self.allowed_lateness = Duration.of(allowed_lateness)
self.timestamp_combiner = (
timestamp_combiner or TimestampCombiner.OUTPUT_AT_EOW)
self._is_default = (
self.windowfn == GlobalWindows() and
self.triggerfn == DefaultTrigger() and
self.accumulation_mode == AccumulationMode.DISCARDING and
- self.timestamp_combiner == TimestampCombiner.OUTPUT_AT_EOW)
+ self.timestamp_combiner == TimestampCombiner.OUTPUT_AT_EOW and
+ self.allowed_lateness == 0)
def __repr__(self):
return "Windowing(%s, %s, %s, %s)" % (self.windowfn, self.triggerfn,
@@ -2314,7 +2327,8 @@
self.windowfn == other.windowfn
and self.triggerfn == other.triggerfn
and self.accumulation_mode == other.accumulation_mode
- and self.timestamp_combiner == other.timestamp_combiner)
+ and self.timestamp_combiner == other.timestamp_combiner
+ and self.allowed_lateness == other.allowed_lateness)
return False
def __ne__(self, other):
@@ -2322,7 +2336,8 @@
return not self == other
def __hash__(self):
- return hash((self.windowfn, self.accumulation_mode,
+ return hash((self.windowfn, self.triggerfn, self.accumulation_mode,
+ self.allowed_lateness,
self.timestamp_combiner))
def is_default(self):
@@ -2344,7 +2359,8 @@
# TODO(robertwb): Support EMIT_IF_NONEMPTY
closing_behavior=beam_runner_api_pb2.ClosingBehavior.EMIT_ALWAYS,
OnTimeBehavior=beam_runner_api_pb2.OnTimeBehavior.FIRE_ALWAYS,
- allowed_lateness=0)
+ allowed_lateness=self.allowed_lateness.micros // 1000,
+ environment_id=context.default_environment_id())
@staticmethod
def from_runner_api(proto, context):
@@ -2354,7 +2370,8 @@
windowfn=WindowFn.from_runner_api(proto.window_fn, context),
triggerfn=TriggerFn.from_runner_api(proto.trigger, context),
accumulation_mode=proto.accumulation_mode,
- timestamp_combiner=proto.output_time)
+ timestamp_combiner=proto.output_time,
+ allowed_lateness=Duration(micros=proto.allowed_lateness * 1000))
@typehints.with_input_types(T)
@@ -2386,8 +2403,8 @@
windowfn, # type: typing.Union[Windowing, WindowFn]
trigger=None, # type: typing.Optional[TriggerFn]
accumulation_mode=None,
- timestamp_combiner=None
- ):
+ timestamp_combiner=None,
+ allowed_lateness=0):
"""Initializes a WindowInto transform.
Args:
@@ -2409,7 +2426,8 @@
timestamp_combiner = timestamp_combiner or windowing.timestamp_combiner
self.windowing = Windowing(
- windowfn, trigger, accumulation_mode, timestamp_combiner)
+ windowfn, trigger, accumulation_mode, timestamp_combiner,
+ allowed_lateness)
super(WindowInto, self).__init__(self.WindowIntoFn(self.windowing))
def get_windowing(self, unused_inputs):
diff --git a/sdks/python/apache_beam/transforms/environments.py b/sdks/python/apache_beam/transforms/environments.py
index 6f67266..dce3b13 100644
--- a/sdks/python/apache_beam/transforms/environments.py
+++ b/sdks/python/apache_beam/transforms/environments.py
@@ -39,6 +39,11 @@
'SubprocessSDKEnvironment', 'RunnerAPIEnvironmentHolder']
+def looks_like_json(s):
+ import re
+ return re.match(r'\s*\{.*\}\s*$', s)
+
+
class Environment(object):
"""Abstract base class for environments.
@@ -263,10 +268,6 @@
@classmethod
def from_options(cls, options):
- def looks_like_json(environment_config):
- import re
- return re.match(r'\s*\{.*\}\s*$', environment_config)
-
if looks_like_json(options.environment_config):
config = json.loads(options.environment_config)
url = config.get('url')
@@ -308,50 +309,77 @@
@Environment.register_urn(python_urns.EMBEDDED_PYTHON_GRPC, bytes)
class EmbeddedPythonGrpcEnvironment(Environment):
- def __init__(self, state_cache_size=None):
+ def __init__(self, state_cache_size=None, data_buffer_time_limit_ms=None):
self.state_cache_size = state_cache_size
+ self.data_buffer_time_limit_ms = data_buffer_time_limit_ms
def __eq__(self, other):
return self.__class__ == other.__class__ \
- and self.state_cache_size == other.state_cache_size
+ and self.state_cache_size == other.state_cache_size \
+ and self.data_buffer_time_limit_ms == other.data_buffer_time_limit_ms
def __ne__(self, other):
# TODO(BEAM-5949): Needed for Python 2 compatibility.
return not self == other
def __hash__(self):
- return hash((self.__class__, self.state_cache_size))
+ return hash((self.__class__, self.state_cache_size,
+ self.data_buffer_time_limit_ms))
def __repr__(self):
repr_parts = []
if not self.state_cache_size is None:
repr_parts.append('state_cache_size=%d' % self.state_cache_size)
+ if not self.data_buffer_time_limit_ms is None:
+ repr_parts.append(
+ 'data_buffer_time_limit_ms=%d' % self.data_buffer_time_limit_ms)
return 'EmbeddedPythonGrpcEnvironment(%s)' % ','.join(repr_parts)
def to_runner_api_parameter(self, context):
- if self.state_cache_size is None:
- payload = b''
- else:
- payload = b'%d' % self.state_cache_size
+ params = {}
+ if self.state_cache_size is not None:
+ params['state_cache_size'] = self.state_cache_size
+ if self.data_buffer_time_limit_ms is not None:
+ params['data_buffer_time_limit_ms'] = self.data_buffer_time_limit_ms
+ payload = json.dumps(params).encode('utf-8')
return python_urns.EMBEDDED_PYTHON_GRPC, payload
@staticmethod
def from_runner_api_parameter(payload, context):
if payload:
- state_cache_size = payload.decode('utf-8')
+ config = EmbeddedPythonGrpcEnvironment.parse_config(
+ payload.decode('utf-8'))
return EmbeddedPythonGrpcEnvironment(
- state_cache_size=int(state_cache_size))
+ state_cache_size=config.get('state_cache_size'),
+ data_buffer_time_limit_ms=config.get('data_buffer_time_limit_ms'))
else:
return EmbeddedPythonGrpcEnvironment()
@classmethod
def from_options(cls, options):
if options.environment_config:
- state_cache_size = options.environment_config
- return cls(state_cache_size=state_cache_size)
+ config = EmbeddedPythonGrpcEnvironment.parse_config(
+ options.environment_config)
+ return cls(state_cache_size=config.get('state_cache_size'),
+ data_buffer_time_limit_ms=config.get(
+ 'data_buffer_time_limit_ms'))
else:
return cls()
+ @staticmethod
+ def parse_config(s):
+ if looks_like_json(s):
+ config_dict = json.loads(s)
+ if 'state_cache_size' in config_dict:
+ config_dict['state_cache_size'] = int(config_dict['state_cache_size'])
+
+ if 'data_buffer_time_limit_ms' in config_dict:
+ config_dict['data_buffer_time_limit_ms'] = \
+ int(config_dict['data_buffer_time_limit_ms'])
+ return config_dict
+ else:
+ return {'state_cache_size': int(s)}
+
@Environment.register_urn(python_urns.SUBPROCESS_SDK, bytes)
class SubprocessSDKEnvironment(Environment):
diff --git a/sdks/python/apache_beam/transforms/environments_test.py b/sdks/python/apache_beam/transforms/environments_test.py
index 46868e8..b28af24 100644
--- a/sdks/python/apache_beam/transforms/environments_test.py
+++ b/sdks/python/apache_beam/transforms/environments_test.py
@@ -46,7 +46,8 @@
ExternalEnvironment('localhost:8080', params={'k1': 'v1'}),
EmbeddedPythonEnvironment(),
EmbeddedPythonGrpcEnvironment(),
- EmbeddedPythonGrpcEnvironment(state_cache_size=0),
+ EmbeddedPythonGrpcEnvironment(
+ state_cache_size=0, data_buffer_time_limit_ms=0),
SubprocessSDKEnvironment(command_string=u'foö')):
context = pipeline_context.PipelineContext()
self.assertEqual(
diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py
index fdec9b7..bf45802 100644
--- a/sdks/python/apache_beam/transforms/external.py
+++ b/sdks/python/apache_beam/transforms/external.py
@@ -408,7 +408,8 @@
inputs={tag: pcoll_renames.get(pcoll, pcoll)
for tag, pcoll in proto.inputs.items()},
outputs={tag: pcoll_renames.get(pcoll, pcoll)
- for tag, pcoll in proto.outputs.items()})
+ for tag, pcoll in proto.outputs.items()},
+ environment_id=proto.environment_id)
context.transforms.put_proto(id, new_proto)
return beam_runner_api_pb2.PTransform(
@@ -418,7 +419,8 @@
inputs=self._expanded_transform.inputs,
outputs={
tag: pcoll_renames.get(pcoll, pcoll)
- for tag, pcoll in self._expanded_transform.outputs.items()})
+ for tag, pcoll in self._expanded_transform.outputs.items()},
+ environment_id=self._expanded_transform.environment_id)
class JavaJarExpansionService(object):
diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py
index bbee4e5..dcc3d96 100644
--- a/sdks/python/apache_beam/transforms/ptransform.py
+++ b/sdks/python/apache_beam/transforms/ptransform.py
@@ -92,6 +92,7 @@
'label_from_callable',
]
+
T = TypeVar('T')
PTransformT = TypeVar('PTransformT', bound='PTransform')
ConstructorFn = Callable[
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index ffb245c..fac509e 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -347,14 +347,14 @@
pipeline = TestPipeline()
result = (pipeline
- | 'Start' >> beam.Create([x for x in range(3)])
+ | 'Start' >> beam.Create([1])
| beam.ParDo(MyDoFn())
| WindowInto(windowfn)
| 'create tuple' >> beam.Map(
lambda v, t=beam.DoFn.TimestampParam, w=beam.DoFn.WindowParam:
(v, t, w.start, w.end)))
- expected_process = [('process'+ str(x), Timestamp(5), Timestamp(4),
- Timestamp(6)) for x in range(3)]
+ expected_process = [('process1', Timestamp(5), Timestamp(4),
+ Timestamp(6))]
expected_finish = [('finish', Timestamp(1), Timestamp(0), Timestamp(2))]
assert_that(result, equal_to(expected_process + expected_finish))
@@ -592,7 +592,8 @@
assert_that(result, equal_to(input))
pipeline.run()
- @attr('ValidatesRunner')
+ # TODO(BEAM-9002): Does not work in streaming mode on Dataflow.
+ @attr('ValidatesRunner', 'sickbay-streaming')
def test_flatten_same_pcollections(self):
pipeline = TestPipeline()
pc = pipeline | beam.Create(['a', 'b'])
diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py
index 6106c26..65bd4c7 100644
--- a/sdks/python/apache_beam/transforms/trigger.py
+++ b/sdks/python/apache_beam/transforms/trigger.py
@@ -1128,6 +1128,7 @@
def __init__(self, windowing, clock):
self.clock = clock
+ self.allowed_lateness = windowing.allowed_lateness
self.window_fn = windowing.windowfn
self.timestamp_combiner_impl = TimestampCombiner.get_impl(
windowing.timestamp_combiner, self.window_fn)
@@ -1147,6 +1148,9 @@
windows_to_elements = collections.defaultdict(list)
for wv in windowed_values:
for window in wv.windows:
+ # ignore expired windows
+ if input_watermark > window.end + self.allowed_lateness:
+ continue
windows_to_elements[window].append((wv.value, wv.timestamp))
# First handle merging.
@@ -1241,7 +1245,6 @@
nonspeculative_index = state.get_state(
window, self.NONSPECULATIVE_INDEX)
state.add_state(window, self.NONSPECULATIVE_INDEX, 1)
- windowed_value.PaneInfoTiming.LATE
_LOGGER.warning('Watermark moved backwards in time '
'or late data moved window end forward.')
else:
diff --git a/sdks/python/apache_beam/transforms/trigger_test.py b/sdks/python/apache_beam/transforms/trigger_test.py
index 58b29e0..bdc8e37 100644
--- a/sdks/python/apache_beam/transforms/trigger_test.py
+++ b/sdks/python/apache_beam/transforms/trigger_test.py
@@ -36,6 +36,7 @@
from apache_beam import coders
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import StandardOptions
+from apache_beam.portability import common_urns
from apache_beam.runners import pipeline_context
from apache_beam.runners.direct.clock import TestClock
from apache_beam.testing.test_pipeline import TestPipeline
@@ -66,6 +67,7 @@
from apache_beam.transforms.window import WindowFn
from apache_beam.utils.timestamp import MAX_TIMESTAMP
from apache_beam.utils.timestamp import MIN_TIMESTAMP
+from apache_beam.utils.timestamp import Duration
from apache_beam.utils.windowed_value import PaneInfoTiming
@@ -118,8 +120,11 @@
bundles, late_bundles,
expected_panes):
actual_panes = collections.defaultdict(list)
+ allowed_lateness = Duration(micros=int(
+ common_urns.constants.MAX_TIMESTAMP_MILLIS.constant)*1000)
driver = GeneralTriggerDriver(
- Windowing(window_fn, trigger_fn, accumulation_mode), TestClock())
+ Windowing(window_fn, trigger_fn, accumulation_mode,
+ allowed_lateness=allowed_lateness), TestClock())
state = InMemoryUnmergedState()
for bundle in bundles:
@@ -422,19 +427,6 @@
class TriggerPipelineTest(unittest.TestCase):
- def setUp(self):
- # Use state on the TestCase class, since other references would be pickled
- # into a closure and not have the desired side effects.
- TriggerPipelineTest.all_records = []
-
- def record_dofn(self):
- class RecordDoFn(beam.DoFn):
-
- def process(self, element):
- TriggerPipelineTest.all_records.append(element)
-
- return RecordDoFn()
-
def test_after_count(self):
with TestPipeline() as p:
def construct_timestamped(k_t):
@@ -471,29 +463,28 @@
if i % 5 == 0:
ts.advance_watermark_to(i)
ts.advance_processing_time(5)
+ ts.advance_watermark_to_infinity()
options = PipelineOptions()
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
- _ = (p
- | ts
- | beam.WindowInto(
- FixedWindows(10),
- accumulation_mode=trigger.AccumulationMode.ACCUMULATING,
- trigger=AfterWatermark(
- early=AfterAll(
- AfterCount(1), AfterProcessingTime(5))
- ))
- | beam.GroupByKey()
- | beam.FlatMap(lambda x: x[1])
- | beam.ParDo(self.record_dofn()))
+ records = (p
+ | ts
+ | beam.WindowInto(
+ FixedWindows(10),
+ accumulation_mode=trigger.AccumulationMode.ACCUMULATING,
+ trigger=AfterWatermark(
+ early=AfterAll(
+ AfterCount(1), AfterProcessingTime(5))
+ ))
+ | beam.GroupByKey()
+ | beam.FlatMap(lambda x: x[1]))
# The trigger should fire twice. Once after 5 seconds, and once after 10.
# The firings should accumulate the output.
first_firing = [str(i) for i in elements if i <= 5]
second_firing = [str(i) for i in elements]
- self.assertListEqual(first_firing + second_firing,
- TriggerPipelineTest.all_records)
+ assert_that(records, equal_to(first_firing + second_firing))
class TranscriptTest(unittest.TestCase):
@@ -604,6 +595,7 @@
timestamp_combiner = getattr(
TimestampCombiner,
spec.get('timestamp_combiner', 'OUTPUT_AT_EOW').upper())
+ allowed_lateness = spec.get('allowed_lateness', 0.000)
def only_element(xs):
x, = list(xs)
@@ -613,7 +605,7 @@
self._execute(
window_fn, trigger_fn, accumulation_mode, timestamp_combiner,
- transcript, spec)
+ allowed_lateness, transcript, spec)
def _windowed_value_info(windowed_value):
@@ -690,11 +682,11 @@
def _execute(
self, window_fn, trigger_fn, accumulation_mode, timestamp_combiner,
- transcript, unused_spec):
+ allowed_lateness, transcript, unused_spec):
driver = GeneralTriggerDriver(
- Windowing(window_fn, trigger_fn, accumulation_mode, timestamp_combiner),
- TestClock())
+ Windowing(window_fn, trigger_fn, accumulation_mode,
+ timestamp_combiner, allowed_lateness), TestClock())
state = InMemoryUnmergedState()
output = []
watermark = MIN_TIMESTAMP
@@ -722,7 +714,8 @@
for t in params]
output = [
_windowed_value_info(wv)
- for wv in driver.process_elements(state, bundle, watermark)]
+ for wv in driver.process_elements(state, bundle, watermark,
+ watermark)]
fire_timers()
elif action == 'watermark':
@@ -756,7 +749,7 @@
def _execute(
self, window_fn, trigger_fn, accumulation_mode, timestamp_combiner,
- transcript, spec):
+ allowed_lateness, transcript, spec):
runner_name = TestPipeline().runner.__class__.__name__
if runner_name in spec.get('broken_on', ()):
@@ -816,6 +809,7 @@
else:
raise ValueError('Unexpected action: %s' % action)
test_stream.add_elements([json.dumps(('expect', []))])
+ test_stream.advance_watermark_to_infinity()
read_test_stream = test_stream | beam.Map(json.loads)
@@ -894,7 +888,8 @@
window_fn,
trigger=trigger_fn,
accumulation_mode=accumulation_mode,
- timestamp_combiner=timestamp_combiner)
+ timestamp_combiner=timestamp_combiner,
+ allowed_lateness=allowed_lateness)
| aggregation
| beam.MapTuple(_windowed_value_info_map_fn)
# Place outputs back into the global window to allow flattening
@@ -934,7 +929,7 @@
def _execute(
self, window_fn, trigger_fn, accumulation_mode, timestamp_combiner,
- transcript, spec):
+ allowed_lateness, transcript, spec):
if timestamp_combiner == TimestampCombiner.OUTPUT_AT_EARLIEST_TRANSFORMED:
self.skipTest(
'Non-fnapi timestamp combiner: %s' % spec.get('timestamp_combiner'))
@@ -984,7 +979,8 @@
window_fn,
trigger=trigger_fn,
accumulation_mode=accumulation_mode,
- timestamp_combiner=timestamp_combiner))
+ timestamp_combiner=timestamp_combiner,
+ allowed_lateness=allowed_lateness))
grouped = input_pc | 'Grouped' >> (
beam.GroupByKey()
diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py
index 58cf243..cf2e3b1 100644
--- a/sdks/python/apache_beam/transforms/util_test.py
+++ b/sdks/python/apache_beam/transforms/util_test.py
@@ -483,7 +483,8 @@
label='after reshuffle')
pipeline.run()
- @attr('ValidatesRunner')
+ # TODO(BEAM-9003): Does not work in streaming mode on Dataflow.
+ @attr('ValidatesRunner', 'sickbay-streaming')
def test_reshuffle_preserves_timestamps(self):
with TestPipeline() as pipeline:
@@ -606,7 +607,7 @@
.advance_watermark_to(start_time +
GroupIntoBatchesTest.NUM_ELEMENTS)
.advance_watermark_to_infinity())
- pipeline = TestPipeline()
+ pipeline = TestPipeline(options=StandardOptions(streaming=True))
# window duration is 6 and batch size is 5, so output batch size should be
# 5 (flush because of batchSize reached)
expected_0 = 5
diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py
index 7868a8f..adc2173 100644
--- a/sdks/python/apache_beam/typehints/decorators.py
+++ b/sdks/python/apache_beam/typehints/decorators.py
@@ -101,6 +101,7 @@
from apache_beam.typehints import native_type_compatibility
from apache_beam.typehints import typehints
+from apache_beam.typehints.native_type_compatibility import convert_to_beam_type
from apache_beam.typehints.typehints import CompositeTypeHintError
from apache_beam.typehints.typehints import SimpleTypeHintError
from apache_beam.typehints.typehints import check_constraint
@@ -266,16 +267,16 @@
input_args.append(typehints.Any)
else:
if param.kind in [param.KEYWORD_ONLY, param.VAR_KEYWORD]:
- input_kwargs[param.name] = param.annotation
+ input_kwargs[param.name] = convert_to_beam_type(param.annotation)
else:
assert param.kind in [param.POSITIONAL_ONLY,
param.POSITIONAL_OR_KEYWORD,
param.VAR_POSITIONAL], \
'Unsupported Parameter kind: %s' % param.kind
- input_args.append(param.annotation)
+ input_args.append(convert_to_beam_type(param.annotation))
output_args = []
if signature.return_annotation != signature.empty:
- output_args.append(signature.return_annotation)
+ output_args.append(convert_to_beam_type(signature.return_annotation))
else:
output_args.append(typehints.Any)
diff --git a/sdks/python/apache_beam/typehints/decorators_test.py b/sdks/python/apache_beam/typehints/decorators_test.py
index 21f62f3..20df674 100644
--- a/sdks/python/apache_beam/typehints/decorators_test.py
+++ b/sdks/python/apache_beam/typehints/decorators_test.py
@@ -26,6 +26,7 @@
from apache_beam.typehints import List
from apache_beam.typehints import WithTypeHints
from apache_beam.typehints import decorators
+from apache_beam.typehints import typehints
decorators._enable_from_callable = True
@@ -72,6 +73,41 @@
{'__unknown__keywords': decorators._ANY_VAR_KEYWORD}))
self.assertEqual(th.output_types, ((Any,), {}))
+ def test_strip_iterable_not_simple_output_noop(self):
+ th = decorators.IOTypeHints(output_types=((int, str), {}))
+ th.strip_iterable()
+ self.assertEqual(((int, str), {}), th.output_types)
+
+ def _test_strip_iterable(self, before, expected_after):
+ after = decorators.IOTypeHints(
+ output_types=((before,), {})).strip_iterable()
+ self.assertEqual(((expected_after, ), {}), after.output_types)
+
+ def _test_strip_iterable_fail(self, before):
+ with self.assertRaisesRegex(ValueError, r'not iterable'):
+ self._test_strip_iterable(before, None)
+
+ def test_strip_iterable(self):
+ # TODO(BEAM-8492): Uncomment once #9895 is merged.
+ # self._test_strip_iterable(None, None)
+ self._test_strip_iterable(typehints.Any, typehints.Any)
+ self._test_strip_iterable(typehints.Iterable[str], str)
+ self._test_strip_iterable(typehints.List[str], str)
+ self._test_strip_iterable(typehints.Iterator[str], str)
+ self._test_strip_iterable(typehints.Generator[str], str)
+ self._test_strip_iterable(typehints.Tuple[str], str)
+ self._test_strip_iterable(typehints.Tuple[str, int],
+ typehints.Union[str, int])
+ self._test_strip_iterable(typehints.Tuple[str, ...], str)
+ self._test_strip_iterable(typehints.KV[str, int],
+ typehints.Union[str, int])
+ self._test_strip_iterable(typehints.Set[str], str)
+
+ self._test_strip_iterable_fail(typehints.Union[str, int])
+ self._test_strip_iterable_fail(typehints.Optional[str])
+ self._test_strip_iterable_fail(typehints.WindowedValue[str])
+ self._test_strip_iterable_fail(typehints.Dict[str, int])
+
class WithTypeHintsTest(unittest.TestCase):
def test_get_type_hints_no_settings(self):
@@ -133,8 +169,8 @@
pass # intentionally avoiding super call
# These should be equal, but not the same object lest mutating the instance
# mutates the class.
- self.assertFalse(
- Subclass()._get_or_create_type_hints() is Subclass._type_hints)
+ self.assertIsNot(
+ Subclass()._get_or_create_type_hints(), Subclass._type_hints)
self.assertEqual(
Subclass().get_type_hints(), Subclass._type_hints)
self.assertNotEqual(
diff --git a/sdks/python/apache_beam/typehints/decorators_test_py3.py b/sdks/python/apache_beam/typehints/decorators_test_py3.py
index d48f845..e66b140 100644
--- a/sdks/python/apache_beam/typehints/decorators_test_py3.py
+++ b/sdks/python/apache_beam/typehints/decorators_test_py3.py
@@ -19,6 +19,7 @@
from __future__ import absolute_import
+import typing
import unittest
# patches unittest.TestCase to be python3 compatible
@@ -33,18 +34,20 @@
decorators._enable_from_callable = True
T = TypeVariable('T')
+# Name is 'T' so it converts to a beam type with the same name.
+T_typing = typing.TypeVar('T')
class IOTypeHintsTest(unittest.TestCase):
def test_from_callable(self):
def fn(a: int, b: str = None, *args: Tuple[T], foo: List[int],
- **kwargs: Dict[str, str]) -> Tuple:
+ **kwargs: Dict[str, str]) -> Tuple[Any, ...]:
return a, b, args, foo, kwargs
th = decorators.IOTypeHints.from_callable(fn)
self.assertEqual(th.input_types, (
(int, str, Tuple[T]), {'foo': List[int], 'kwargs': Dict[str, str]}))
- self.assertEqual(th.output_types, ((Tuple,), {}))
+ self.assertEqual(th.output_types, ((Tuple[Any, ...],), {}))
def test_from_callable_partial_annotations(self):
def fn(a: int, b=None, *args, foo: List[int], **kwargs):
@@ -77,9 +80,22 @@
self.assertEqual(th.input_types, ((T,), {}))
self.assertEqual(th.output_types, ((None,), {}))
+ def test_from_callable_convert_to_beam_types(self):
+ def fn(a: typing.List[int],
+ b: str = None,
+ *args: typing.Tuple[T_typing],
+ foo: typing.List[int],
+ **kwargs: typing.Dict[str, str]) -> typing.Tuple[typing.Any, ...]:
+ return a, b, args, foo, kwargs
+ th = decorators.IOTypeHints.from_callable(fn)
+ self.assertEqual(th.input_types, (
+ (List[int], str, Tuple[T]),
+ {'foo': List[int], 'kwargs': Dict[str, str]}))
+ self.assertEqual(th.output_types, ((Tuple[Any, ...],), {}))
+
def test_getcallargs_forhints(self):
def fn(a: int, b: str = None, *args: Tuple[T], foo: List[int],
- **kwargs: Dict[str, str]) -> Tuple:
+ **kwargs: Dict[str, str]) -> Tuple[Any, ...]:
return a, b, args, foo, kwargs
callargs = decorators.getcallargs_forhints(fn, float, foo=List[str])
self.assertDictEqual(callargs,
@@ -91,7 +107,7 @@
def test_getcallargs_forhints_default_arg(self):
# Default args are not necessarily types, so they should be ignored.
- def fn(a=List[int], b=None, *args, foo=(), **kwargs) -> Tuple:
+ def fn(a=List[int], b=None, *args, foo=(), **kwargs) -> Tuple[Any, ...]:
return a, b, args, foo, kwargs
callargs = decorators.getcallargs_forhints(fn)
self.assertDictEqual(callargs,
diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py
index 5d55fbc..f2802e0 100644
--- a/sdks/python/apache_beam/typehints/native_type_compatibility.py
+++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py
@@ -20,6 +20,7 @@
from __future__ import absolute_import
import collections
+import logging
import sys
import typing
from builtins import next
@@ -27,6 +28,8 @@
from apache_beam.typehints import typehints
+_LOGGER = logging.getLogger(__name__)
+
# Describes an entry in the type map in convert_to_beam_type.
# match is a function that takes a user type and returns whether the conversion
# should trigger.
@@ -44,7 +47,10 @@
# __union_params__ argument respectively.
if (3, 0, 0) <= sys.version_info[0:3] < (3, 5, 3):
if getattr(typ, '__tuple_params__', None) is not None:
- return typ.__tuple_params__
+ if typ.__tuple_use_ellipsis__:
+ return typ.__tuple_params__ + (Ellipsis,)
+ else:
+ return typ.__tuple_params__
elif getattr(typ, '__union_params__', None) is not None:
return typ.__union_params__
return None
@@ -183,7 +189,8 @@
"""Convert a given typing type to a Beam type.
Args:
- typ (type): typing type.
+ typ (`typing.Union[type, str]`): typing type or string literal representing
+ a type.
Returns:
type: The given type converted to a Beam type as far as we can do the
@@ -203,6 +210,11 @@
_type_var_cache[id(typ)] = new_type_variable
_type_var_cache[id(new_type_variable)] = typ
return _type_var_cache[id(typ)]
+ elif isinstance(typ, str):
+ # Special case for forward references.
+ # TODO(BEAM-8487): Currently unhandled.
+ _LOGGER.info('Converting string literal type hint to Any: "%s"', typ)
+ return typehints.Any
elif getattr(typ, '__module__', None) != 'typing':
# Only translate types from the typing module.
return typ
@@ -251,8 +263,9 @@
# Find the first matching entry.
matched_entry = next((entry for entry in type_map if entry.match(typ)), None)
if not matched_entry:
- # No match: return original type.
- return typ
+ # Please add missing type support if you see this message.
+ _LOGGER.info('Using Any for unsupported type: %s', typ)
+ return typehints.Any
if matched_entry.arity == -1:
arity = _len_arg(typ)
diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py
index 38e75dc..4bfb60b 100644
--- a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py
+++ b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py
@@ -102,6 +102,11 @@
typehints.Iterator[int],
convert_to_beam_type(typing.Generator[int, None, None]))
+ def test_string_literal_converted_to_any(self):
+ self.assertEqual(
+ typehints.Any,
+ convert_to_beam_type('typing.List[int]'))
+
def test_convert_nested_to_beam_type(self):
self.assertEqual(
typehints.List[typing.Any],
diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
index e49b11c..a718c8b 100644
--- a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
+++ b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
@@ -147,7 +147,7 @@
r'requires.*str.*got.*int.*side_input'):
_ = [1, 2, 3] | beam.ParDo(my_do_fn, side_input=1)
- def test_type_dofn_var_kwargs(self):
+ def test_typed_dofn_var_kwargs(self):
class MyDoFn(beam.DoFn):
def process(self, element: int, **side_inputs: typehints.Dict[str, str]) \
-> typehints.Generator[typehints.Optional[int]]:
@@ -161,6 +161,21 @@
r'requires.*str.*got.*int.*side_inputs'):
_ = [1, 2, 3] | beam.ParDo(my_do_fn, a=1)
+ def test_typed_callable_string_literals(self):
+ def do_fn(element: 'int') -> 'typehints.List[str]':
+ return [[str(element)] * 2]
+
+ result = [1, 2] | beam.ParDo(do_fn)
+ self.assertEqual([['1', '1'], ['2', '2']], sorted(result))
+
+ def test_typed_dofn_string_literals(self):
+ class MyDoFn(beam.DoFn):
+ def process(self, element: 'int') -> 'typehints.List[str]':
+ return [[str(element)] * 2]
+
+ result = [1, 2] | beam.ParDo(MyDoFn())
+ self.assertEqual([['1', '1'], ['2', '2']], sorted(result))
+
class AnnotationsTest(unittest.TestCase):
diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py
index e085a0e..052beb3 100644
--- a/sdks/python/apache_beam/typehints/typehints.py
+++ b/sdks/python/apache_beam/typehints/typehints.py
@@ -1175,7 +1175,7 @@
"""Obtains the type of elements yielded by an iterable.
Note that "iterable" here means: can be iterated over in a for loop, excluding
- strings.
+ strings and dicts.
Args:
type_hint: (TypeConstraint) The iterable in question. Must be normalize()-d.
@@ -1191,7 +1191,10 @@
if is_consistent_with(type_hint, Iterator[Any]):
return type_hint.yielded_type
if is_consistent_with(type_hint, Tuple[Any, ...]):
- return Union[type_hint.tuple_types]
+ if isinstance(type_hint, TupleConstraint):
+ return Union[type_hint.tuple_types]
+ else: # TupleSequenceConstraint
+ return type_hint.inner_type
if is_consistent_with(type_hint, Iterable[Any]):
return type_hint.inner_type
raise ValueError('%s is not iterable' % type_hint)
diff --git a/sdks/python/apache_beam/utils/retry.py b/sdks/python/apache_beam/utils/retry.py
index e34e364..c291393 100644
--- a/sdks/python/apache_beam/utils/retry.py
+++ b/sdks/python/apache_beam/utils/retry.py
@@ -46,8 +46,15 @@
# TODO(sourabhbajaj): Remove the GCP specific error code to a submodule
try:
from apitools.base.py.exceptions import HttpError
-except ImportError:
+except ImportError as e:
HttpError = None
+
+# Protect against environments where aws tools are not available.
+# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
+try:
+ from apache_beam.io.aws.clients.s3.messages import S3ClientError
+except ImportError:
+ S3ClientError = None
# pylint: enable=wrong-import-order, wrong-import-position
@@ -104,6 +111,8 @@
"""Filter allowing retries on server errors and non-HttpErrors."""
if (HttpError is not None) and isinstance(exception, HttpError):
return exception.status_code >= 500
+ if (S3ClientError is not None) and isinstance(exception, S3ClientError):
+ return exception.code >= 500
return not isinstance(exception, PermanentException)
@@ -120,6 +129,9 @@
if HttpError is not None and isinstance(exception, HttpError):
if exception.status_code == 408: # 408 Request Timeout
return True
+ if S3ClientError is not None and isinstance(exception, S3ClientError):
+ if exception.code == 408: # 408 Request Timeout
+ return True
return retry_on_server_errors_filter(exception)
@@ -131,6 +143,9 @@
if HttpError is not None and isinstance(exception, HttpError):
if exception.status_code == 403:
return True
+ if S3ClientError is not None and isinstance(exception, S3ClientError):
+ if exception.code == 403:
+ return True
return retry_on_server_errors_and_timeout_filter(exception)
diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py
index 4402ea4..47b8584 100644
--- a/sdks/python/apache_beam/utils/urns.py
+++ b/sdks/python/apache_beam/utils/urns.py
@@ -55,7 +55,7 @@
A class that inherits from this class will get a registration-based
from_runner_api and to_runner_api method that convert to and from
- beam_runner_api_pb2.SdkFunctionSpec.
+ beam_runner_api_pb2.FunctionSpec.
Additionally, register_pickle_urn can be called from the body of a class
to register serialization via pickling.
@@ -149,29 +149,26 @@
lambda proto, unused_context: pickler.loads(proto.value))
def to_runner_api(self, context):
- # type: (PipelineContext) -> beam_runner_api_pb2.SdkFunctionSpec
- """Returns an SdkFunctionSpec encoding this Fn.
+ # type: (PipelineContext) -> beam_runner_api_pb2.FunctionSpec
+ """Returns an FunctionSpec encoding this Fn.
Prefer overriding self.to_runner_api_parameter.
"""
from apache_beam.portability.api import beam_runner_api_pb2
urn, typed_param = self.to_runner_api_parameter(context)
- return beam_runner_api_pb2.SdkFunctionSpec(
- environment_id=context.default_environment_id(),
- spec=beam_runner_api_pb2.FunctionSpec(
- urn=urn,
- payload=typed_param.SerializeToString()
- if isinstance(typed_param, message.Message)
- else typed_param))
+ return beam_runner_api_pb2.FunctionSpec(
+ urn=urn,
+ payload=typed_param.SerializeToString()
+ if isinstance(typed_param, message.Message) else typed_param)
@classmethod
def from_runner_api(cls, fn_proto, context):
- # type: (beam_runner_api_pb2.SdkFunctionSpec, PipelineContext) -> Any
- """Converts from an SdkFunctionSpec to a Fn object.
+ # type: (beam_runner_api_pb2.FunctionSpec, PipelineContext) -> Any
+ """Converts from an FunctionSpec to a Fn object.
Prefer registering a urn with its parameter type and constructor.
"""
- parameter_type, constructor = cls._known_urns[fn_proto.spec.urn]
+ parameter_type, constructor = cls._known_urns[fn_proto.urn]
return constructor(
- proto_utils.parse_Bytes(fn_proto.spec.payload, parameter_type),
+ proto_utils.parse_Bytes(fn_proto.payload, parameter_type),
context)
diff --git a/sdks/python/apache_beam/utils/windowed_value.py b/sdks/python/apache_beam/utils/windowed_value.py
index 3883b7d..786484f 100644
--- a/sdks/python/apache_beam/utils/windowed_value.py
+++ b/sdks/python/apache_beam/utils/windowed_value.py
@@ -61,6 +61,15 @@
cls.UNKNOWN: 'UNKNOWN',
}[value]
+ @classmethod
+ def from_string(cls, value):
+ return {
+ 'EARLY': cls.EARLY,
+ 'ON_TIME': cls.ON_TIME,
+ 'LATE': cls.LATE,
+ 'UNKNOWN': cls.UNKNOWN
+ }[value]
+
class PaneInfo(object):
"""Describes the trigger firing information for a given WindowedValue.
diff --git a/sdks/python/build-requirements.txt b/sdks/python/build-requirements.txt
index 06d7c2d..b4fcd87 100644
--- a/sdks/python/build-requirements.txt
+++ b/sdks/python/build-requirements.txt
@@ -16,5 +16,4 @@
#
# TODO(BEAM-5414): latest grpcio-tools incompatible with latest protobuf 3.6.1.
grpcio-tools>=1.3.5,<=1.14.2
-mypy-protobuf==1.12
future==0.16.0
diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go
index 71da7b9..71775c4 100644
--- a/sdks/python/container/boot.go
+++ b/sdks/python/container/boot.go
@@ -143,6 +143,10 @@
os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pbpipeline.ApiServiceDescriptor{Url: *loggingEndpoint}))
os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pbpipeline.ApiServiceDescriptor{Url: *controlEndpoint}))
+ if info.GetStatusEndpoint() != nil {
+ os.Setenv("STATUS_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(info.GetStatusEndpoint()))
+ }
+
args := []string{
"-m",
sdkHarnessEntrypoint,
diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py
index b4e9297..5105ad5 100644
--- a/sdks/python/gen_protos.py
+++ b/sdks/python/gen_protos.py
@@ -121,19 +121,12 @@
raise ValueError("Proto generation failed (see log for details).")
else:
log.info('Regenerating Python proto definitions (%s).' % regenerate)
-
- ret_code = subprocess.call(["pip", "install", "mypy-protobuf==1.12"])
- if ret_code:
- raise RuntimeError(
- 'Error installing mypy-protobuf during proto generation')
-
builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto')
args = (
[sys.executable] + # expecting to be called from command line
['--proto_path=%s' % builtin_protos] +
['--proto_path=%s' % d for d in proto_dirs] +
['--python_out=%s' % out_dir] +
- ['--mypy_out=%s' % out_dir] +
# TODO(robertwb): Remove the prefix once it's the default.
['--grpc_python_out=grpc_2_0:%s' % out_dir] +
proto_files)
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 5d433bf..fcb4fd4 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -173,10 +173,11 @@
]
REQUIRED_TEST_PACKAGES = [
+ 'freezegun>=0.3.12',
'nose>=1.3.7',
'nose_xunitmp>=0.4.1',
'pandas>=0.23.4,<0.25',
- 'parameterized>=0.6.0,<0.7.0',
+ 'parameterized>=0.6.0,<0.8.0',
'pyhamcrest>=1.9,<2.0',
'pyyaml>=3.12,<6.0.0',
'requests_mock>=1.7,<2.0',
@@ -207,6 +208,10 @@
'jsons>=1.0.0,<2; python_version >= "3.5.3"',
'timeloop>=1.0.2,<2',
]
+AWS_REQUIREMENTS = [
+ 'boto3 >=1.9'
+]
+
# We must generate protos after setup_requires are installed.
def generate_protos_first(original_cmd):
@@ -267,6 +272,7 @@
'test': REQUIRED_TEST_PACKAGES,
'gcp': GCP_REQUIREMENTS,
'interactive': INTERACTIVE_BEAM,
+ 'aws': AWS_REQUIREMENTS
},
zip_safe=False,
# PyPI package information.
diff --git a/sdks/python/test-suites/direct/py2/build.gradle b/sdks/python/test-suites/direct/py2/build.gradle
index e422b74..436cd7e 100644
--- a/sdks/python/test-suites/direct/py2/build.gradle
+++ b/sdks/python/test-suites/direct/py2/build.gradle
@@ -38,6 +38,8 @@
"apache_beam.io.gcp.pubsub_integration_test:PubSubIntegrationTest",
"apache_beam.io.gcp.big_query_query_to_table_it_test:BigQueryQueryToTableIT",
"apache_beam.io.gcp.bigquery_io_read_it_test",
+ "apache_beam.io.gcp.bigquery_read_it_test",
+ "apache_beam.io.gcp.bigquery_write_it_test",
"apache_beam.io.gcp.datastore.v1new.datastore_write_it_test",
]
def batchTestOpts = basicTestOpts + ["--tests=${tests.join(',')}"]
diff --git a/sdks/python/test-suites/portable/py2/build.gradle b/sdks/python/test-suites/portable/py2/build.gradle
index b55173c..f9fb5f4 100644
--- a/sdks/python/test-suites/portable/py2/build.gradle
+++ b/sdks/python/test-suites/portable/py2/build.gradle
@@ -22,6 +22,7 @@
applyPythonNature()
def pythonRootDir = "${rootDir}/sdks/python"
+def runScriptsDir = "${rootDir}/sdks/python/scripts"
/*************************************************************************************************/
@@ -39,6 +40,7 @@
dependsOn ':runners:flink:1.9:job-server:shadowJar'
dependsOn portableWordCountFlinkRunnerBatch
dependsOn portableWordCountFlinkRunnerStreaming
+ dependsOn 'postCommitPy2IT'
dependsOn ':runners:spark:job-server:shadowJar'
dependsOn portableWordCountSparkRunnerBatch
}
@@ -146,6 +148,28 @@
}
}
+task postCommitPy2IT {
+ dependsOn 'installGcpTest'
+ dependsOn 'setupVirtualenv'
+ dependsOn ':runners:flink:1.9:job-server:shadowJar'
+
+ doLast {
+ def tests = [
+ "apache_beam.io.gcp.bigquery_read_it_test",
+ ]
+ def testOpts = ["--tests=${tests.join(',')}"]
+ def cmdArgs = mapToArgString([
+ "test_opts": testOpts,
+ "suite": "postCommitIT-flink-py2",
+ "pipeline_opts": "--runner=FlinkRunner --project=apache-beam-testing --environment_type=LOOPBACK --temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
+ ])
+ exec {
+ executable 'sh'
+ args '-c', ". ${envdir}/bin/activate && ${runScriptsDir}/run_integration_test.sh $cmdArgs"
+ }
+ }
+}
+
task crossLanguageTests {
dependsOn "crossLanguagePythonJavaFlink"
dependsOn "crossLanguagePortableWordCount"
diff --git a/sdks/python/test-suites/portable/py35/build.gradle b/sdks/python/test-suites/portable/py35/build.gradle
index 88b4e2f..479556e 100644
--- a/sdks/python/test-suites/portable/py35/build.gradle
+++ b/sdks/python/test-suites/portable/py35/build.gradle
@@ -18,6 +18,9 @@
apply plugin: org.apache.beam.gradle.BeamModulePlugin
applyPythonNature()
+
+def runScriptsDir = "${rootDir}/sdks/python/scripts"
+
// Required to setup a Python 3.5 virtualenv.
pythonVersion = '3.5'
apply from: "../common.gradle"
@@ -36,6 +39,29 @@
dependsOn ':runners:flink:1.9:job-server:shadowJar'
dependsOn portableWordCountFlinkRunnerBatch
dependsOn portableWordCountFlinkRunnerStreaming
+ dependsOn 'postCommitPy35IT'
dependsOn ':runners:spark:job-server:shadowJar'
dependsOn portableWordCountSparkRunnerBatch
}
+
+task postCommitPy35IT {
+ dependsOn 'setupVirtualenv'
+ dependsOn 'installGcpTest'
+ dependsOn ':runners:flink:1.9:job-server:shadowJar'
+
+ doLast {
+ def tests = [
+ "apache_beam.io.gcp.bigquery_read_it_test",
+ ]
+ def testOpts = ["--tests=${tests.join(',')}"]
+ def cmdArgs = mapToArgString([
+ "test_opts": testOpts,
+ "suite": "postCommitIT-flink-py35",
+ "pipeline_opts": "--runner=FlinkRunner --project=apache-beam-testing --environment_type=LOOPBACK --temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
+ ])
+ exec {
+ executable 'sh'
+ args '-c', ". ${envdir}/bin/activate && ${runScriptsDir}/run_integration_test.sh $cmdArgs"
+ }
+ }
+}
diff --git a/sdks/python/test-suites/portable/py36/build.gradle b/sdks/python/test-suites/portable/py36/build.gradle
index 496777d..6cefad6 100644
--- a/sdks/python/test-suites/portable/py36/build.gradle
+++ b/sdks/python/test-suites/portable/py36/build.gradle
@@ -18,6 +18,9 @@
apply plugin: org.apache.beam.gradle.BeamModulePlugin
applyPythonNature()
+
+def runScriptsDir = "${rootDir}/sdks/python/scripts"
+
// Required to setup a Python 3.6 virtualenv.
pythonVersion = '3.6'
apply from: "../common.gradle"
@@ -36,6 +39,29 @@
dependsOn ':runners:flink:1.9:job-server:shadowJar'
dependsOn portableWordCountFlinkRunnerBatch
dependsOn portableWordCountFlinkRunnerStreaming
+ dependsOn 'postCommitPy36IT'
dependsOn ':runners:spark:job-server:shadowJar'
dependsOn portableWordCountSparkRunnerBatch
}
+
+task postCommitPy36IT {
+ dependsOn 'setupVirtualenv'
+ dependsOn 'installGcpTest'
+ dependsOn ':runners:flink:1.9:job-server:shadowJar'
+
+ doLast {
+ def tests = [
+ "apache_beam.io.gcp.bigquery_read_it_test",
+ ]
+ def testOpts = ["--tests=${tests.join(',')}"]
+ def cmdArgs = mapToArgString([
+ "test_opts": testOpts,
+ "suite": "postCommitIT-flink-py36",
+ "pipeline_opts": "--runner=FlinkRunner --project=apache-beam-testing --environment_type=LOOPBACK --temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
+ ])
+ exec {
+ executable 'sh'
+ args '-c', ". ${envdir}/bin/activate && ${runScriptsDir}/run_integration_test.sh $cmdArgs"
+ }
+ }
+}
diff --git a/sdks/python/test-suites/portable/py37/build.gradle b/sdks/python/test-suites/portable/py37/build.gradle
index 924de81..c7acb57 100644
--- a/sdks/python/test-suites/portable/py37/build.gradle
+++ b/sdks/python/test-suites/portable/py37/build.gradle
@@ -18,6 +18,9 @@
apply plugin: org.apache.beam.gradle.BeamModulePlugin
applyPythonNature()
+
+def runScriptsDir = "${rootDir}/sdks/python/scripts"
+
// Required to setup a Python 3.7 virtualenv.
pythonVersion = '3.7'
apply from: "../common.gradle"
@@ -36,6 +39,29 @@
dependsOn ':runners:flink:1.9:job-server:shadowJar'
dependsOn portableWordCountFlinkRunnerBatch
dependsOn portableWordCountFlinkRunnerStreaming
+ dependsOn 'postCommitPy37IT'
dependsOn ':runners:spark:job-server:shadowJar'
dependsOn portableWordCountSparkRunnerBatch
}
+
+task postCommitPy37IT {
+ dependsOn 'setupVirtualenv'
+ dependsOn 'installGcpTest'
+ dependsOn ':runners:flink:1.9:job-server:shadowJar'
+
+ doLast {
+ def tests = [
+ "apache_beam.io.gcp.bigquery_read_it_test",
+ ]
+ def testOpts = ["--tests=${tests.join(',')}"]
+ def cmdArgs = mapToArgString([
+ "test_opts": testOpts,
+ "suite": "postCommitIT-flink-py37",
+ "pipeline_opts": "--runner=FlinkRunner --project=apache-beam-testing --environment_type=LOOPBACK --temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
+ ])
+ exec {
+ executable 'sh'
+ args '-c', ". ${envdir}/bin/activate && ${runScriptsDir}/run_integration_test.sh $cmdArgs"
+ }
+ }
+}
diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini
index 0c79d41..c78bddf 100644
--- a/sdks/python/tox.ini
+++ b/sdks/python/tox.ini
@@ -36,6 +36,10 @@
deps =
cython: cython==0.28.1
-r build-requirements.txt
+setenv =
+ RUN_SKIPPED_PY3_TESTS=0
+ # Use an isolated tmp dir for tests that get slowed down by scanning /tmp.
+ TMPDIR={envtmpdir}
# These 2 magic command overrides are required for Jenkins builds.
# Otherwise we get "OSError: [Errno 2] No such file or directory" errors.
@@ -63,43 +67,31 @@
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
[testenv:py35]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
python setup.py nosetests --ignore-files '.*py3[6-9]\.py$' {posargs}
[testenv:py35-pytest]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
[testenv:py36]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
python setup.py nosetests --ignore-files '.*py3[7-9]\.py$' {posargs}
[testenv:py36-pytest]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
[testenv:py37]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
python setup.py nosetests --ignore-files '.*py3[8-9]\.py$' {posargs}
[testenv:py37-pytest]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
@@ -133,8 +125,6 @@
# `platform = linux2|darwin|...`
# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes
platform = linux
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
python setup.py nosetests --ignore-files '.*py3[5-9]\.py$' {posargs}
@@ -145,8 +135,6 @@
# `platform = linux2|darwin|...`
# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes
platform = linux
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
# TODO(BEAM-8954): Remove this build_ext invocation once local source no longer
# shadows the installed apache_beam.
@@ -160,8 +148,6 @@
# `platform = linux2|darwin|...`
# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes
platform = linux
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
python setup.py nosetests --ignore-files '.*py3[7-9]\.py$' {posargs}
@@ -172,8 +158,6 @@
# `platform = linux2|darwin|...`
# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes
platform = linux
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
# TODO(BEAM-8954): Remove this build_ext invocation once local source no longer
# shadows the installed apache_beam.
@@ -187,8 +171,6 @@
# `platform = linux2|darwin|...`
# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes
platform = linux
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
python apache_beam/examples/complete/autocomplete_test.py
python setup.py nosetests --ignore-files '.*py3[8-9]\.py$' {posargs}
@@ -199,8 +181,6 @@
# `platform = linux2|darwin|...`
# See https://docs.python.org/2/library/sys.html#sys.platform for platform codes
platform = linux
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
commands =
# TODO(BEAM-8954): Remove this build_ext invocation once local source no longer
# shadows the installed apache_beam.
@@ -233,47 +213,40 @@
pytest -o junit_suite_name={envname}_v1new --junitxml=pytest_{envname}_v1new.xml apache_beam/io/gcp/datastore/v1new
[testenv:py35-gcp]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
extras = test,gcp
commands =
python setup.py nosetests --ignore-files '.*py3[6-9]\.py$' {posargs}
[testenv:py35-gcp-pytest]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
extras = test,gcp
commands =
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
[testenv:py36-gcp]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
extras = test,gcp,interactive
commands =
python setup.py nosetests --ignore-files '.*py3[7-9]\.py$' {posargs}
[testenv:py36-gcp-pytest]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
extras = test,gcp,interactive
commands =
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
[testenv:py37-gcp]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
extras = test,gcp,interactive
commands =
python setup.py nosetests --ignore-files '.*py3[8-9]\.py$' {posargs}
[testenv:py37-gcp-pytest]
-setenv =
- RUN_SKIPPED_PY3_TESTS=0
extras = test,gcp,interactive
commands =
{toxinidir}/scripts/run_pytest.sh {envname} "{posargs}"
+[testenv:py37-aws]
+extras = test,aws
+commands =
+ python setup.py nosetests --ignore-files '.*py3[8-9]\.py$' {posargs}
+
[testenv:py27-lint]
# Checks for py2 syntax errors
deps =
@@ -284,6 +257,8 @@
[testenv:py27-lint3]
# Checks for py2/3 compatibility issues
+# Don't set TMPDIR to avoid "AF_UNIX path too long" errors in pylint.
+setenv =
deps =
-r build-requirements.txt
pycodestyle==2.3.1
@@ -295,6 +270,8 @@
time {toxinidir}/scripts/run_pylint_2to3.sh
[testenv:py37-lint]
+# Don't set TMPDIR to avoid "AF_UNIX path too long" errors in pylint.
+setenv =
deps =
-r build-requirements.txt
astroid<2.4,>=2.3.0
diff --git a/settings.gradle b/settings.gradle
index 55784ca..2952e09 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -54,6 +54,7 @@
include ":runners:portability:java"
include ":runners:spark"
include ":runners:spark:job-server"
+include ":runners:spark:job-server:container"
include ":runners:samza"
include ":runners:samza:job-server"
include ":sdks:go"
@@ -91,6 +92,7 @@
include ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-2"
include ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-5"
include ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-6"
+include ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-7"
include ":sdks:java:io:elasticsearch-tests:elasticsearch-tests-common"
include ":sdks:java:io:file-based-io-tests"
include ':sdks:java:io:bigquery-io-perf-tests'
diff --git a/website/src/_includes/flink_java_pipeline_options.html b/website/src/_includes/flink_java_pipeline_options.html
new file mode 100644
index 0000000..4495a33
--- /dev/null
+++ b/website/src/_includes/flink_java_pipeline_options.html
@@ -0,0 +1,145 @@
+<!--
+Licensed 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.
+-->
+<!--
+This is an auto-generated file.
+Use generatePipelineOptionsTableJava and generatePipelineOptionsTablePython respectively
+which should be called before running the tests.
+-->
+<table class="table table-bordered">
+<tr>
+ <td><code>allowNonRestoredState</code></td>
+ <td>Flag indicating whether non restored state is allowed if the savepoint contains state for an operator that is no longer part of the pipeline.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>autoBalanceWriteFilesShardingEnabled</code></td>
+ <td>Flag indicating whether auto-balance sharding for WriteFiles transform should be enabled. This might prove useful in streaming use-case, where pipeline needs to write quite many events into files, typically divided into N shards. Default behavior on Flink would be, that some workers will receive more shards to take care of than others. This cause workers to go out of balance in terms of processing backlog and memory usage. Enabling this feature will make shards to be spread evenly among available workers in improve throughput and memory usage stability.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>autoWatermarkInterval</code></td>
+ <td>The interval in milliseconds for automatic watermark emission.</td>
+ <td></td>
+</tr>
+<tr>
+ <td><code>checkpointTimeoutMillis</code></td>
+ <td>The maximum time in milliseconds that a checkpoint may take before being discarded.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>checkpointingInterval</code></td>
+ <td>The interval in milliseconds at which to trigger checkpoints of the running pipeline. Default: No checkpointing.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>checkpointingMode</code></td>
+ <td>The checkpointing mode that defines consistency guarantee.</td>
+ <td>Default: <code>EXACTLY_ONCE</code></td>
+</tr>
+<tr>
+ <td><code>disableMetrics</code></td>
+ <td>Disable Beam metrics in Flink Runner</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>executionModeForBatch</code></td>
+ <td>Flink mode for data exchange of batch pipelines. Reference {@link org.apache.flink.api.common.ExecutionMode}. Set this to BATCH_FORCED if pipelines get blocked, see https://issues.apache.org/jira/browse/FLINK-10672</td>
+ <td>Default: <code>PIPELINED</code></td>
+</tr>
+<tr>
+ <td><code>executionRetryDelay</code></td>
+ <td>Sets the delay in milliseconds between executions. A value of {@code -1} indicates that the default value should be used.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>externalizedCheckpointsEnabled</code></td>
+ <td>Enables or disables externalized checkpoints. Works in conjunction with CheckpointingInterval</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>failOnCheckpointingErrors</code></td>
+ <td>Sets the expected behaviour for tasks in case that they encounter an error in their checkpointing procedure. If this is set to true, the task will fail on checkpointing error. If this is set to false, the task will only decline a the checkpoint and continue running. </td>
+ <td>Default: <code>true</code></td>
+</tr>
+<tr>
+ <td><code>filesToStage</code></td>
+ <td>Jar-Files to send to all workers and put on the classpath. The default value is all files from the classpath.</td>
+ <td></td>
+</tr>
+<tr>
+ <td><code>flinkMaster</code></td>
+ <td>Address of the Flink Master where the Pipeline should be executed. Can either be of the form "host:port" or one of the special values [local], [collection] or [auto].</td>
+ <td>Default: <code>[auto]</code></td>
+</tr>
+<tr>
+ <td><code>latencyTrackingInterval</code></td>
+ <td>Interval in milliseconds for sending latency tracking marks from the sources to the sinks. Interval value <= 0 disables the feature.</td>
+ <td>Default: <code>0</code></td>
+</tr>
+<tr>
+ <td><code>maxBundleSize</code></td>
+ <td>The maximum number of elements in a bundle.</td>
+ <td>Default: <code>1000</code></td>
+</tr>
+<tr>
+ <td><code>maxBundleTimeMills</code></td>
+ <td>The maximum time to wait before finalising a bundle (in milliseconds).</td>
+ <td>Default: <code>1000</code></td>
+</tr>
+<tr>
+ <td><code>maxParallelism</code></td>
+ <td>The pipeline wide maximum degree of parallelism to be used. The maximum parallelism specifies the upper limit for dynamic scaling and the number of key groups used for partitioned state.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>minPauseBetweenCheckpoints</code></td>
+ <td>The minimal pause in milliseconds before the next checkpoint is triggered.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>numberOfExecutionRetries</code></td>
+ <td>Sets the number of times that failed tasks are re-executed. A value of zero effectively disables fault tolerance. A value of -1 indicates that the system default value (as defined in the configuration) should be used.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>objectReuse</code></td>
+ <td>Sets the behavior of reusing objects.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>parallelism</code></td>
+ <td>The degree of parallelism to be used when distributing operations onto workers. If the parallelism is not set, the configured Flink default is used, or 1 if none can be found.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>retainExternalizedCheckpointsOnCancellation</code></td>
+ <td>Sets the behavior of externalized checkpoints on cancellation.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>savepointPath</code></td>
+ <td>Savepoint restore path. If specified, restores the streaming pipeline from the provided path.</td>
+ <td></td>
+</tr>
+<tr>
+ <td><code>shutdownSourcesOnFinalWatermark</code></td>
+ <td>If set, shutdown sources when their watermark reaches +Inf.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>stateBackendFactory</code></td>
+ <td>Sets the state backend factory to use in streaming mode. Defaults to the flink cluster's state.backend configuration.</td>
+ <td></td>
+</tr>
+</table>
diff --git a/website/src/_includes/flink_python_pipeline_options.html b/website/src/_includes/flink_python_pipeline_options.html
new file mode 100644
index 0000000..b57c433
--- /dev/null
+++ b/website/src/_includes/flink_python_pipeline_options.html
@@ -0,0 +1,145 @@
+<!--
+Licensed 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.
+-->
+<!--
+This is an auto-generated file.
+Use generatePipelineOptionsTableJava and generatePipelineOptionsTablePython respectively
+which should be called before running the tests.
+-->
+<table class="table table-bordered">
+<tr>
+ <td><code>allow_non_restored_state</code></td>
+ <td>Flag indicating whether non restored state is allowed if the savepoint contains state for an operator that is no longer part of the pipeline.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>auto_balance_write_files_sharding_enabled</code></td>
+ <td>Flag indicating whether auto-balance sharding for WriteFiles transform should be enabled. This might prove useful in streaming use-case, where pipeline needs to write quite many events into files, typically divided into N shards. Default behavior on Flink would be, that some workers will receive more shards to take care of than others. This cause workers to go out of balance in terms of processing backlog and memory usage. Enabling this feature will make shards to be spread evenly among available workers in improve throughput and memory usage stability.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>auto_watermark_interval</code></td>
+ <td>The interval in milliseconds for automatic watermark emission.</td>
+ <td></td>
+</tr>
+<tr>
+ <td><code>checkpoint_timeout_millis</code></td>
+ <td>The maximum time in milliseconds that a checkpoint may take before being discarded.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>checkpointing_interval</code></td>
+ <td>The interval in milliseconds at which to trigger checkpoints of the running pipeline. Default: No checkpointing.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>checkpointing_mode</code></td>
+ <td>The checkpointing mode that defines consistency guarantee.</td>
+ <td>Default: <code>EXACTLY_ONCE</code></td>
+</tr>
+<tr>
+ <td><code>disable_metrics</code></td>
+ <td>Disable Beam metrics in Flink Runner</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>execution_mode_for_batch</code></td>
+ <td>Flink mode for data exchange of batch pipelines. Reference {@link org.apache.flink.api.common.ExecutionMode}. Set this to BATCH_FORCED if pipelines get blocked, see https://issues.apache.org/jira/browse/FLINK-10672</td>
+ <td>Default: <code>PIPELINED</code></td>
+</tr>
+<tr>
+ <td><code>execution_retry_delay</code></td>
+ <td>Sets the delay in milliseconds between executions. A value of {@code -1} indicates that the default value should be used.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>externalized_checkpoints_enabled</code></td>
+ <td>Enables or disables externalized checkpoints. Works in conjunction with CheckpointingInterval</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>fail_on_checkpointing_errors</code></td>
+ <td>Sets the expected behaviour for tasks in case that they encounter an error in their checkpointing procedure. If this is set to true, the task will fail on checkpointing error. If this is set to false, the task will only decline a the checkpoint and continue running. </td>
+ <td>Default: <code>true</code></td>
+</tr>
+<tr>
+ <td><code>files_to_stage</code></td>
+ <td>Jar-Files to send to all workers and put on the classpath. The default value is all files from the classpath.</td>
+ <td></td>
+</tr>
+<tr>
+ <td><code>flink_master</code></td>
+ <td>Address of the Flink Master where the Pipeline should be executed. Can either be of the form "host:port" or one of the special values [local], [collection] or [auto].</td>
+ <td>Default: <code>[auto]</code></td>
+</tr>
+<tr>
+ <td><code>latency_tracking_interval</code></td>
+ <td>Interval in milliseconds for sending latency tracking marks from the sources to the sinks. Interval value <= 0 disables the feature.</td>
+ <td>Default: <code>0</code></td>
+</tr>
+<tr>
+ <td><code>max_bundle_size</code></td>
+ <td>The maximum number of elements in a bundle.</td>
+ <td>Default: <code>1000</code></td>
+</tr>
+<tr>
+ <td><code>max_bundle_time_mills</code></td>
+ <td>The maximum time to wait before finalising a bundle (in milliseconds).</td>
+ <td>Default: <code>1000</code></td>
+</tr>
+<tr>
+ <td><code>max_parallelism</code></td>
+ <td>The pipeline wide maximum degree of parallelism to be used. The maximum parallelism specifies the upper limit for dynamic scaling and the number of key groups used for partitioned state.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>min_pause_between_checkpoints</code></td>
+ <td>The minimal pause in milliseconds before the next checkpoint is triggered.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>number_of_execution_retries</code></td>
+ <td>Sets the number of times that failed tasks are re-executed. A value of zero effectively disables fault tolerance. A value of -1 indicates that the system default value (as defined in the configuration) should be used.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>object_reuse</code></td>
+ <td>Sets the behavior of reusing objects.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>parallelism</code></td>
+ <td>The degree of parallelism to be used when distributing operations onto workers. If the parallelism is not set, the configured Flink default is used, or 1 if none can be found.</td>
+ <td>Default: <code>-1</code></td>
+</tr>
+<tr>
+ <td><code>retain_externalized_checkpoints_on_cancellation</code></td>
+ <td>Sets the behavior of externalized checkpoints on cancellation.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>savepoint_path</code></td>
+ <td>Savepoint restore path. If specified, restores the streaming pipeline from the provided path.</td>
+ <td></td>
+</tr>
+<tr>
+ <td><code>shutdown_sources_on_final_watermark</code></td>
+ <td>If set, shutdown sources when their watermark reaches +Inf.</td>
+ <td>Default: <code>false</code></td>
+</tr>
+<tr>
+ <td><code>state_backend_factory</code></td>
+ <td>Sets the state backend factory to use in streaming mode. Defaults to the flink cluster's state.backend configuration.</td>
+ <td></td>
+</tr>
+</table>
diff --git a/website/src/contribute/release-guide.md b/website/src/contribute/release-guide.md
index 04039df..2a27bde 100644
--- a/website/src/contribute/release-guide.md
+++ b/website/src/contribute/release-guide.md
@@ -529,6 +529,18 @@
Check if there are outstanding cherry-picks into the release branch, [e.g. for `2.14.0`](https://github.com/apache/beam/pulls?utf8=%E2%9C%93&q=is%3Apr+base%3Arelease-2.14.0).
Make sure they have blocker JIRAs attached and are OK to get into the release by checking with community if needed.
+As the Release Manager you are empowered to accept or reject cherry-picks to the release branch. You are encouraged to ask the following questions to be answered on each cherry-pick PR and you can choose to reject cherry-pick requests if these questions are not satisfactorily answered:
+
+* Is this a regression from a previous release? (If no, fix could go to a newer version.)
+* Is this a new feature or related to a new feature? (If yes, fix could go to a new version.)
+* Would this impact production workloads for users? (E.g. if this is a direct runner only fix it may not need to be a cherry pick.)
+* What percentage of users would be impacted by this issue if it is not fixed? (E.g. If this is predicted to be a small number it may not need to be a cherry pick.)
+* Would it be possible for the impacted users to skip this version? (If users could skip this version, fix could go to a newer version.)
+
+It is important to accept major/blocking fixes to isolated issues to make a higher quality release. However, beyond that each cherry pick will increase the time required for the release and add more last minute code to the release branch. Neither late releases nor not fully tested code will provide positive user value.
+
+_Tip_: Another tool in your toolbox is the known issues section of the release blog. Consider adding known issues there for minor issues instead of accepting cherry picks to the release branch.
+
**********
@@ -855,6 +867,7 @@
* {$KNOWN_ISSUE_1}
* {$KNOWN_ISSUE_2}
+ * See a full list of open [issues that affects](https://issues.apache.org/jira/browse/BEAM-8989?jql=project = BEAM AND affectedVersion = 2.16.0 ORDER BY priority DESC, updated DESC) this version.
## List of Contributors
diff --git a/website/src/documentation/runners/flink.md b/website/src/documentation/runners/flink.md
index a7ab744..6e7307f 100644
--- a/website/src/documentation/runners/flink.md
+++ b/website/src/documentation/runners/flink.md
@@ -336,7 +336,7 @@
"--flink_master=localhost:8081",
"--environment_type=LOOPBACK"
])
-with beam.Pipeline(options) as p:
+with beam.Pipeline(options=options) as p:
...
```
@@ -350,7 +350,7 @@
### Streaming Execution
-If your pipeline uses an unbounded data source or sink, the Flink Runner will automatically switch to streaming mode. You can enforce streaming mode by using the `streaming` setting mentioned below.
+If your pipeline uses an unbounded data source or sink, the Flink Runner will automatically switch to streaming mode. You can enforce streaming mode by using the `--streaming` flag.
Note: The Runner will print a warning message when unbounded sources are used and checkpointing is not enabled.
Many sources like `PubSubIO` rely on their checkpoints to be acknowledged which can only be done when checkpointing is enabled for the `FlinkRunner`. To enable checkpointing, please set <span class="language-java">`checkpointingInterval`</span><span class="language-py">`checkpointing_interval`</span> to the desired checkpointing interval in milliseconds.
@@ -359,284 +359,23 @@
When executing your pipeline with the Flink Runner, you can set these pipeline options.
-See the reference documentation for the<span class="language-java">
+The following list of Flink-specific pipeline options is generated automatically from the
[FlinkPipelineOptions](https://beam.apache.org/releases/javadoc/{{ site.release_latest }}/index.html?org/apache/beam/runners/flink/FlinkPipelineOptions.html)
-</span><span class="language-py">
-[PipelineOptions](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/options/pipeline_options.py)
-</span>interface (and its subinterfaces) for the complete list of pipeline configuration options.
-
+reference class:
<!-- Java Options -->
<div class="language-java">
-<table class="table table-bordered">
-<tr>
- <th>Field</th>
- <th>Description</th>
- <th>Default Value</th>
-</tr>
-<tr>
- <td><code>runner</code></td>
- <td>The pipeline runner to use. This option allows you to determine the pipeline runner at runtime.</td>
- <td>Set to <code>FlinkRunner</code> to run using Flink.</td>
-</tr>
-<tr>
- <td><code>streaming</code></td>
- <td>Whether streaming mode is enabled or disabled; <code>true</code> if enabled. Set to <code>true</code> if running pipelines with unbounded <code>PCollection</code>s.</td>
- <td><code>false</code></td>
-</tr>
-<tr>
- <td><code>flinkMaster</code></td>
- <td>The url of the Flink JobManager on which to execute pipelines. This can either be the address of a cluster JobManager, in the form <code>"host:port"</code> or one of the special Strings <code>"[local]"</code> or <code>"[auto]"</code>. <code>"[local]"</code> will start a local Flink Cluster in the JVM while <code>"[auto]"</code> will let the system decide where to execute the pipeline based on the environment.</td>
- <td><code>[auto]</code></td>
-</tr>
-<tr>
- <td><code>filesToStage</code></td>
- <td>Jar Files to send to all workers and put on the classpath. Here you have to put the fat jar that contains your program along with all dependencies.</td>
- <td>empty</td>
-</tr>
-<tr>
- <td><code>parallelism</code></td>
- <td>The degree of parallelism to be used when distributing operations onto workers.</td>
- <td>For local execution: <code>Number of available CPU cores</code>
- For remote execution: <code>Default parallelism configuerd at remote cluster</code>
- Otherwise: <code>1</code>
- </td>
-</tr>
-<tr>
- <td><code>maxParallelism</code></td>
- <td>The pipeline wide maximum degree of parallelism to be used. The maximum parallelism specifies the upper limit for dynamic scaling and the number of key groups used for partitioned state.</td>
- <td><code>-1L</code>, meaning same as the parallelism</td>
-</tr>
-<tr>
- <td><code>checkpointingInterval</code></td>
- <td>The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for fault tolerance).</td>
- <td><code>-1L</code>, i.e. disabled</td>
-</tr>
-<tr>
- <td><code>checkpointMode</code></td>
- <td>The checkpointing mode that defines consistency guarantee.</td>
- <td><code>EXACTLY_ONCE</code></td>
-</tr>
-<tr>
- <td><code>checkpointTimeoutMillis</code></td>
- <td>The maximum time in milliseconds that a checkpoint may take before being discarded</td>
- <td><code>-1</code>, the cluster default</td>
-</tr>
-<tr>
- <td><code>minPauseBetweenCheckpoints</code></td>
- <td>The minimal pause in milliseconds before the next checkpoint is triggered.</td>
- <td><code>-1</code>, the cluster default</td>
-</tr>
-<tr>
- <td><code>failOnCheckpointingErrors</code></td>
- <td>
- Sets the expected behaviour for tasks in case that they encounter an error in their
- checkpointing procedure. If this is set to true, the task will fail on checkpointing error.
- If this is set to false, the task will only decline a the checkpoint and continue running.
- </td>
- <td><code>-1</code>, the cluster default</td>
-</tr>
-<tr>
- <td><code>numberOfExecutionRetries</code></td>
- <td>Sets the number of times that failed tasks are re-executed. A value of <code>0</code> effectively disables fault tolerance. A value of <code>-1</code> indicates that the system default value (as defined in the configuration) should be used.</td>
- <td><code>-1</code></td>
-</tr>
-<tr>
- <td><code>executionRetryDelay</code></td>
- <td>Sets the delay between executions. A value of <code>-1</code> indicates that the default value should be used.</td>
- <td><code>-1</code></td>
-</tr>
-<tr>
- <td><code>objectReuse</code></td>
- <td>Sets the behavior of reusing objects.</td>
- <td><code>false</code>, no Object reuse</td>
-</tr>
-<tr>
- <td><code>stateBackend</code></td>
- <td>Sets the state backend to use in streaming mode. The default is to read this setting from the Flink config.</td>
- <td><code>empty</code>, i.e. read from Flink config</td>
-</tr>
-<tr>
- <td><code>enableMetrics</code></td>
- <td>Enable/disable Beam metrics in Flink Runner</td>
- <td>Default: <code>true</code></td>
-</tr>
-<tr>
- <td><code>externalizedCheckpointsEnabled</code></td>
- <td>Enables or disables externalized checkpoints. Works in conjunction with CheckpointingInterval</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>retainExternalizedCheckpointsOnCancellation</code></td>
- <td>Sets the behavior of externalized checkpoints on cancellation.</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>maxBundleSize</code></td>
- <td>The maximum number of elements in a bundle.</td>
- <td>Default: <code>1000</code></td>
-</tr>
-<tr>
- <td><code>maxBundleTimeMills</code></td>
- <td>The maximum time to wait before finalising a bundle (in milliseconds).</td>
- <td>Default: <code>1000</code></td>
-</tr>
-<tr>
- <td><code>shutdownSourcesOnFinalWatermark</code></td>
- <td>If set, shutdown sources when their watermark reaches +Inf.</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>latencyTrackingInterval</code></td>
- <td>Interval in milliseconds for sending latency tracking marks from the sources to the sinks. Interval value <= 0 disables the feature.</td>
- <td>Default: <code>0</code></td>
-</tr>
-<tr>
- <td><code>autoWatermarkInterval</code></td>
- <td>The interval in milliseconds for automatic watermark emission.</td>
-</tr>
-<tr>
- <td><code>executionModeForBatch</code></td>
- <td>Flink mode for data exchange of batch pipelines. Reference {@link org.apache.flink.api.common.ExecutionMode}. Set this to BATCH_FORCED if pipelines get blocked, see https://issues.apache.org/jira/browse/FLINK-10672</td>
- <td>Default: <code>PIPELINED</code></td>
-</tr>
-<tr>
- <td><code>savepointPath</code></td>
- <td>Savepoint restore path. If specified, restores the streaming pipeline from the provided path.</td>
- <td>Default: None</td>
-</tr>
-<tr>
- <td><code>allowNonRestoredState</code></td>
- <td>Flag indicating whether non restored state is allowed if the savepoint contains state for an operator that is no longer part of the pipeline.</td>
- <td>Default: <code>false</code></td>
-</tr>
-</table>
+{% include flink_java_pipeline_options.html %}
</div>
-
<!-- Python Options -->
<div class="language-py">
-<table class="table table-bordered">
-
-<tr>
- <td><code>files_to_stage</code></td>
- <td>Jar-Files to send to all workers and put on the classpath. The default value is all files from the classpath.</td>
-</tr>
-<tr>
- <td><code>flink_master</code></td>
- <td>Address of the Flink Master where the Pipeline should be executed. Can either be of the form "host:port" or one of the special values [local], [collection] or [auto].</td>
- <td>Default: <code>[auto]</code></td>
-</tr>
-<tr>
- <td><code>parallelism</code></td>
- <td>The degree of parallelism to be used when distributing operations onto workers. If the parallelism is not set, the configured Flink default is used, or 1 if none can be found.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>max_parallelism</code></td>
- <td>The pipeline wide maximum degree of parallelism to be used. The maximum parallelism specifies the upper limit for dynamic scaling and the number of key groups used for partitioned state.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>checkpointing_interval</code></td>
- <td>The interval in milliseconds at which to trigger checkpoints of the running pipeline. Default: No checkpointing.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>checkpointing_mode</code></td>
- <td>The checkpointing mode that defines consistency guarantee.</td>
- <td>Default: <code>EXACTLY_ONCE</code></td>
-</tr>
-<tr>
- <td><code>checkpoint_timeout_millis</code></td>
- <td>The maximum time in milliseconds that a checkpoint may take before being discarded.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>min_pause_between_checkpoints</code></td>
- <td>The minimal pause in milliseconds before the next checkpoint is triggered.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>fail_on_checkpointing_errors</code></td>
- <td>Sets the expected behaviour for tasks in case that they encounter an error in their checkpointing procedure. If this is set to true, the task will fail on checkpointing error. If this is set to false, the task will only decline a the checkpoint and continue running. </td>
- <td>Default: <code>true</code></td>
-</tr>
-<tr>
- <td><code>number_of_execution_retries</code></td>
- <td>Sets the number of times that failed tasks are re-executed. A value of zero effectively disables fault tolerance. A value of -1 indicates that the system default value (as defined in the configuration) should be used.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>execution_retry_delay</code></td>
- <td>Sets the delay in milliseconds between executions. A value of {@code -1} indicates that the default value should be used.</td>
- <td>Default: <code>-1</code></td>
-</tr>
-<tr>
- <td><code>object_reuse</code></td>
- <td>Sets the behavior of reusing objects.</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>state_backend</code></td>
- <td>Sets the state backend to use in streaming mode. Otherwise the default is read from the Flink config.</td>
-</tr>
-<tr>
- <td><code>enable_metrics</code></td>
- <td>Enable/disable Beam metrics in Flink Runner</td>
- <td>Default: <code>true</code></td>
-</tr>
-<tr>
- <td><code>externalized_checkpoints_enabled</code></td>
- <td>Enables or disables externalized checkpoints. Works in conjunction with CheckpointingInterval</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>retain_externalized_checkpoints_on_cancellation</code></td>
- <td>Sets the behavior of externalized checkpoints on cancellation.</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>max_bundle_size</code></td>
- <td>The maximum number of elements in a bundle.</td>
- <td>Default: <code>1000</code></td>
-</tr>
-<tr>
- <td><code>max_bundle_time_mills</code></td>
- <td>The maximum time to wait before finalising a bundle (in milliseconds).</td>
- <td>Default: <code>1000</code></td>
-</tr>
-<tr>
- <td><code>shutdown_sources_on_final_watermark</code></td>
- <td>If set, shutdown sources when their watermark reaches +Inf.</td>
- <td>Default: <code>false</code></td>
-</tr>
-<tr>
- <td><code>latency_tracking_interval</code></td>
- <td>Interval in milliseconds for sending latency tracking marks from the sources to the sinks. Interval value <= 0 disables the feature.</td>
- <td>Default: <code>0</code></td>
-</tr>
-<tr>
- <td><code>auto_watermark_interval</code></td>
- <td>The interval in milliseconds for automatic watermark emission.</td>
-</tr>
-<tr>
- <td><code>execution_mode_for_batch</code></td>
- <td>Flink mode for data exchange of batch pipelines. Reference {@link org.apache.flink.api.common.ExecutionMode}. Set this to BATCH_FORCED if pipelines get blocked, see https://issues.apache.org/jira/browse/FLINK-10672</td>
- <td>Default: <code>PIPELINED</code></td>
-</tr>
-<tr>
- <td><code>savepoint_path</code></td>
- <td>Savepoint restore path. If specified, restores the streaming pipeline from the provided path.</td>
-</tr>
-<tr>
- <td><code>allow_non_restored_state</code></td>
- <td>Flag indicating whether non restored state is allowed if the savepoint contains state for an operator that is no longer part of the pipeline.</td>
- <td>Default: <code>false</code></td>
-</tr>
-
-</table>
+{% include flink_python_pipeline_options.html %}
</div>
+For general Beam pipeline options see the
+[PipelineOptions](https://beam.apache.org/releases/javadoc/{{ site.release_latest }}/index.html?org/apache/beam/sdk/options/PipelineOptions.html)
+reference.
+
## Capability
The [Beam Capability Matrix]({{ site.baseurl