From 51ec5fcf4054eaae329f631ff61839e7b2049961 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Wed, 20 Nov 2019 12:32:38 -0800 Subject: [PATCH 01/67] Modify AggregateProjectMergeRule to have a condition --- .../sql/impl/planner/BeamRuleSets.java | 4 +- .../rule/BeamAggregateProjectMergeRule.java | 72 ++++++++++++ .../BeamAggregateProjectMergeRuleTest.java | 109 ++++++++++++++++++ 3 files changed, 183 insertions(+), 2 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java 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 f30f9f344052..14774b9d3f12 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 @@ public class BeamRuleSets { 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/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 000000000000..46a865113e56 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java @@ -0,0 +1,72 @@ +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); + private Set visitedNodes = new HashSet<>(); + + public BeamAggregateProjectMergeRule( + Class aggregateClass, + Class projectClass, + RelBuilderFactory relBuilderFactory) { + super(aggregateClass, projectClass, relBuilderFactory); + } + + @Override + public void onMatch(RelOptRuleCall call) { + final Project project = call.rel(1); + BeamIOSourceRel io = getUnderlyingIO(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: + * 1) Aggregate <- Project <- IO. + * 2) Aggregate <- Project <- Chain of Project/Filter <- IO. + * 3) Aggregate <- Project <- Something else. + * 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(SingleRel parent) { + // No need to look at the same node more than once. + if (visitedNodes.contains(parent)) { + return null; + } + visitedNodes.add(parent); + List nodes = ((RelSubset) parent.getInput()).getRelList(); + + for (RelNode node : nodes) { + if (node instanceof Filter || node instanceof Project) { + return getUnderlyingIO((SingleRel) node); + } else if (node instanceof BeamIOSourceRel) { + return (BeamIOSourceRel) node; + } + } + + return 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 000000000000..4fdb3be29d8f --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java @@ -0,0 +1,109 @@ +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.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_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(); + } +} From f0fa11aee96c756c37905904a6cbc96a42d00382 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Wed, 20 Nov 2019 13:07:40 -0800 Subject: [PATCH 02/67] SpotlesApply --- .../rule/BeamAggregateProjectMergeRule.java | 40 ++++++++++++++----- .../BeamAggregateProjectMergeRuleTest.java | 26 ++++++++++-- 2 files changed, 54 insertions(+), 12 deletions(-) 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 index 46a865113e56..39c1da88b7f6 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; @@ -17,11 +34,13 @@ /** * 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}. + * 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 static final AggregateProjectMergeRule INSTANCE = + new BeamAggregateProjectMergeRule( + Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER); private Set visitedNodes = new HashSet<>(); public BeamAggregateProjectMergeRule( @@ -36,20 +55,23 @@ public void onMatch(RelOptRuleCall call) { final Project project = call.rel(1); BeamIOSourceRel io = getUnderlyingIO(project); - // Only perform AggregateProjectMergeRule when IO is not present or project push-down is not supported. + // 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: - * 1) Aggregate <- Project <- IO. - * 2) Aggregate <- Project <- Chain of Project/Filter <- IO. - * 3) Aggregate <- Project <- Something else. + * Following scenarios are possible:
+ * 1) Aggregate <- Project <- IO.
+ * 2) Aggregate <- Project <- Chain of Project/Filter <- IO.
+ * 3) Aggregate <- Project <- Something else.
* 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. + * @return {@code BeamIOSourceRel} when it is present or null when some other {@code RelNode} is + * present. */ private BeamIOSourceRel getUnderlyingIO(SingleRel parent) { // No need to look at the same node more than once. 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 index 4fdb3be29d8f..0a115b663c08 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; @@ -47,7 +64,6 @@ public void buildUp() { sqlEnv = BeamSqlEnv.inMemory(tableProvider); } - @Test public void testBeamAggregateProjectMergeRule_withProjectTable() { // When an IO supports project push-down, Projects should be merged with an IO. @@ -76,7 +92,9 @@ public void testBeamAggregateProjectMergeRule_withFilterTable() { // IO projects al fields. assertThat(ioSourceRel, instanceOf(BeamIOSourceRel.class)); - assertThat(ioSourceRel.getRowType().getFieldNames(), containsInAnyOrder("unused1", "name", "id", "unused2")); + assertThat( + ioSourceRel.getRowType().getFieldNames(), + containsInAnyOrder("unused1", "name", "id", "unused2")); } @Test @@ -93,7 +111,9 @@ public void testBeamAggregateProjectMergeRule_withNoneTable() { // IO projects al fields. assertThat(ioSourceRel, instanceOf(BeamIOSourceRel.class)); - assertThat(ioSourceRel.getRowType().getFieldNames(), containsInAnyOrder("unused1", "name", "id", "unused2")); + assertThat( + ioSourceRel.getRowType().getFieldNames(), + containsInAnyOrder("unused1", "name", "id", "unused2")); } private static Table getTable(String name, PushDownOptions options) { From fe96a249a326479a580be9ea1e5715cc4de62b08 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Wed, 20 Nov 2019 14:53:03 -0800 Subject: [PATCH 03/67] Test for a query with a predicate --- .../rule/BeamAggregateProjectMergeRuleTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 index 0a115b663c08..2bce48cb89de 100644 --- 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 @@ -25,6 +25,7 @@ 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; @@ -78,6 +79,22 @@ public void testBeamAggregateProjectMergeRule_withProjectTable() { 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. From a25f02cbe0f32ce8a4372d9c3e62138d74d98d76 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Wed, 20 Nov 2019 16:25:03 -0800 Subject: [PATCH 04/67] A list of visited nodes should be unique per onMatch invocation --- .../sql/impl/rule/BeamAggregateProjectMergeRule.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 index 39c1da88b7f6..38c5df2f6b14 100644 --- 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 @@ -41,7 +41,6 @@ public class BeamAggregateProjectMergeRule extends AggregateProjectMergeRule { public static final AggregateProjectMergeRule INSTANCE = new BeamAggregateProjectMergeRule( Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER); - private Set visitedNodes = new HashSet<>(); public BeamAggregateProjectMergeRule( Class aggregateClass, @@ -53,7 +52,7 @@ public BeamAggregateProjectMergeRule( @Override public void onMatch(RelOptRuleCall call) { final Project project = call.rel(1); - BeamIOSourceRel io = getUnderlyingIO(project); + BeamIOSourceRel io = getUnderlyingIO(new HashSet<>(), project); // Only perform AggregateProjectMergeRule when IO is not present or project push-down is not // supported. @@ -73,7 +72,7 @@ public void onMatch(RelOptRuleCall call) { * @return {@code BeamIOSourceRel} when it is present or null when some other {@code RelNode} is * present. */ - private BeamIOSourceRel getUnderlyingIO(SingleRel parent) { + private BeamIOSourceRel getUnderlyingIO(Set visitedNodes, SingleRel parent) { // No need to look at the same node more than once. if (visitedNodes.contains(parent)) { return null; @@ -83,7 +82,7 @@ private BeamIOSourceRel getUnderlyingIO(SingleRel parent) { for (RelNode node : nodes) { if (node instanceof Filter || node instanceof Project) { - return getUnderlyingIO((SingleRel) node); + return getUnderlyingIO(visitedNodes, (SingleRel) node); } else if (node instanceof BeamIOSourceRel) { return (BeamIOSourceRel) node; } From 9d36670cf598fdddffc80b692805919cd78e66b5 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Thu, 21 Nov 2019 09:55:18 -0800 Subject: [PATCH 05/67] Make sure all nodes are explored --- .../sql/impl/rule/BeamAggregateProjectMergeRule.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 index 38c5df2f6b14..b146b0476065 100644 --- 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 @@ -82,7 +82,11 @@ private BeamIOSourceRel getUnderlyingIO(Set visitedNodes, SingleRel par for (RelNode node : nodes) { if (node instanceof Filter || node instanceof Project) { - return getUnderlyingIO(visitedNodes, (SingleRel) node); + // 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; } From ecf1a99a48119bf9882d9ce8fabca21b73dddbb6 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Tue, 17 Dec 2019 10:00:55 -0800 Subject: [PATCH 06/67] Update release docs --- website/_config.yml | 2 +- website/src/.htaccess | 2 +- website/src/get-started/downloads.md | 7 +++++++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/website/_config.yml b/website/_config.yml index 19eb6d162d7b..ff65ef8c548f 100644 --- a/website/_config.yml +++ b/website/_config.yml @@ -62,7 +62,7 @@ kramdown: toc_levels: 2..6 # The most recent release of Beam. -release_latest: 2.16.0 +release_latest: 2.17.0 # Plugins are configured in the Gemfile. diff --git a/website/src/.htaccess b/website/src/.htaccess index 20d45861efdf..3f3eb2b3dce6 100644 --- a/website/src/.htaccess +++ b/website/src/.htaccess @@ -21,4 +21,4 @@ RewriteRule ^(.*)$ https://beam.apache.org/$1 [L,R=301] # The following redirect maintains the previously supported URLs. RedirectMatch permanent "/documentation/sdks/(javadoc|pydoc)(.*)" "https://beam.apache.org/releases/$1$2" # Keep this updated to point to the current release. -RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.16.0$2" +RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.17.0$2" diff --git a/website/src/get-started/downloads.md b/website/src/get-started/downloads.md index ab19e5b811c4..75842edc954c 100644 --- a/website/src/get-started/downloads.md +++ b/website/src/get-started/downloads.md @@ -90,6 +90,13 @@ versions denoted `0.x.y`. ## Releases +## 2.17.0 (2019-12-16) +Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.17.0/apache-beam-2.17.0-source-release.zip). +[SHA-512](https://www.apache.org/dist/beam/2.17.0/apache-beam-2.17.0-source-release.zip.sha512). +[signature](https://www.apache.org/dist/beam/2.17.0/apache-beam-2.17.0-source-release.zip.asc). + +[Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12345494). + ## 2.16.0 (2019-10-07) Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.16.0/apache-beam-2.16.0-source-release.zip). [SHA-512](https://www.apache.org/dist/beam/2.16.0/apache-beam-2.16.0-source-release.zip.sha512). From bf6577e09add3748edba4b1126a1edf02798ab88 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Mon, 16 Dec 2019 18:06:36 -0800 Subject: [PATCH 07/67] Import freezegun for Python time testing. --- sdks/python/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 5d433bff69b8..0a4f41c912a2 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -173,6 +173,7 @@ def get_version(): ] REQUIRED_TEST_PACKAGES = [ + 'freezegun>=0.3.12', 'nose>=1.3.7', 'nose_xunitmp>=0.4.1', 'pandas>=0.23.4,<0.25', From b329de6cbdd4aac0aa353a4e235a95f997db95b3 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Mon, 16 Dec 2019 16:08:31 -0800 Subject: [PATCH 08/67] Allow message stream to yield duplicates. This is useful so that even if a certain state has already been read by the state stream, the corresponding message can be returned as _last_error_message. --- .../runners/portability/portable_runner.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py b/sdks/python/apache_beam/runners/portability/portable_runner.py index e29a3451f182..dac60dcd06e9 100644 --- a/sdks/python/apache_beam/runners/portability/portable_runner.py +++ b/sdks/python/apache_beam/runners/portability/portable_runner.py @@ -417,6 +417,7 @@ def _last_error_message(self): 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 +425,12 @@ def read_messages(): "%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') From b5ae54dc221d21e3dd1220ec7d88b5b7fba830dd Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Tue, 17 Dec 2019 11:23:44 -0800 Subject: [PATCH 09/67] Blogpost stub --- website/src/_data/authors.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/website/src/_data/authors.yml b/website/src/_data/authors.yml index 6830fe1a08f9..a8e874db12c0 100644 --- a/website/src/_data/authors.yml +++ b/website/src/_data/authors.yml @@ -81,6 +81,10 @@ markliu: name: Mark Liu email: markliu@apache.org twitter: +ardagan: + name: Mikhail Gryzykhin + email: mikhail@apache.org + twitter: robertwb: name: Robert Bradshaw email: robertwb@apache.org From 447986501090508064b945a220926cd901923bdf Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Tue, 10 Dec 2019 16:53:36 -0800 Subject: [PATCH 10/67] [BEAM-8891] Create and submit Spark portable jar in Python. - The state and message streams got a little messy, as they're treated as if they're independent, but at least in this implementation the message stream is just a superset of the state stream. To untangle things a bit, I decided to a) allow duplicate messages and b) store message history, in addition to state history. - For unit testing, I added library `freezegun` to freeze time at 0, which was way easier than doing a bunch of elaborate mocking just to get one insignificant value lined up. Follow-ups: - The Spark REST API will return errors if there's a problem with the job submission, but it doesn't return errors from the cluster itself once the job is running. I couldn't figure out a good solution to this, so I left it as a TODO(BEAM-8983). - There should be an e2e test for this. BEAM-8984 --- .../runners/spark/SparkPipelineRunner.java | 4 + .../apache_beam/options/pipeline_options.py | 20 ++ .../runners/portability/spark_runner.py | 30 +- .../portability/spark_uber_jar_job_server.py | 318 ++++++++++++++++++ .../spark_uber_jar_job_server_test.py | 209 ++++++++++++ 5 files changed, 565 insertions(+), 16 deletions(-) create mode 100644 sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py create mode 100644 sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server_test.py 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 6df3bf321567..d0c1c0ffa496 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 @@ public static void main(String[] args) throws Exception { 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/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 33218fbd3adb..1b4e822b1826 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -991,6 +991,26 @@ def _add_argparse_args(cls, parser): ' 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 master' + ' directly, rather than starting up a job server.' + ' Only applies when Spark master is set to a' + ' cluster address. Requires Python 3.6+.') + + class TestOptions(PipelineOptions): @classmethod diff --git a/sdks/python/apache_beam/runners/portability/spark_runner.py b/sdks/python/apache_beam/runners/portability/spark_runner.py index 8c3939e7c4c4..c292ea9b4ecd 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 @@ def run_pipeline(self, pipeline, options): 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 + and not re.match(LOCAL_MASTER_PATTERN, spark_options.spark_master_url)): + if sys.version_info < (3, 6): + raise ValueError( + 'spark_submit_uber_jar requires Python 3.6+, current version %s' + % sys.version) + return spark_uber_jar_job_server.SparkUberJarJobServer( + re.sub('^spark://', 'http://', spark_options.spark_master_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 000000000000..7ddf31e142c8 --- /dev/null +++ b/sdks/python/apache_beam/runners/portability/spark_uber_jar_job_server.py @@ -0,0 +1,318 @@ +# +# 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 json +import logging +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 +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, master_url, options): + super(SparkUberJarJobServer, self).__init__() + self._master_url = master_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._master_url, + self.executable_jar(), + job_id, + job_name, + pipeline, + options, + artifact_port=self._artifact_port) + + +class SparkBeamJob(abstract_job_service.AbstractBeamJob): + """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.""" + + # 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(SparkBeamJob, self).__init__(job_id, job_name, pipeline, options) + self._master_url = master_url + self._executable_jar = executable_jar + self._jar_uploaded = False + self._artifact_port = artifact_port + # Message history is a superset of state history. + self._message_history = self._state_history[:] + + 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): + url = '%s/%s' % (self._master_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 000000000000..7106403f04df --- /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() From f5d2abd77225ccc2a1a7cd49c1e54658bee4077b Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Wed, 18 Dec 2019 13:52:08 -0800 Subject: [PATCH 11/67] Add blogpost file --- website/src/_posts_2019-12-16-beam-2.17.0.md | 73 ++++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 website/src/_posts_2019-12-16-beam-2.17.0.md diff --git a/website/src/_posts_2019-12-16-beam-2.17.0.md b/website/src/_posts_2019-12-16-beam-2.17.0.md new file mode 100644 index 000000000000..2c62845e171e --- /dev/null +++ b/website/src/_posts_2019-12-16-beam-2.17.0.md @@ -0,0 +1,73 @@ +--- +layout: post +title: "Apache Beam 2.17.0" +date: 2019-12-16 00:00:01 -0800 +# Date above corrected but keep the old URL: +permalink: /blog/2019/12/16/beam-2.17.0.html +excerpt_separator: +categories: blog +authors: + - ardagan + +--- + + +We are happy to present the new 2.17.0 release of Beam. This release includes both improvements and new functionality. +See the [download page]({{ site.baseurl }}/get-started/downloads/#2170-2019-12-16) for this release. +For more information on changes in 2.17.0, check out the +[detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12345970&projectId=12319527). + +## Highlights +WIP + +### New Features / Improvements +WIP + +### Dependency Changes +WIP + +### Bugfixes + +* Various bug fixes and performance improvements. + + +### Known Issues + +* [BEAM-8989](https://issues.apache.org/jira/browse/BEAM-8989) Apache Nemo + runner broken due to backwards incompatible change since 2.16.0. + +## List of Contributors + + According to git shortlog, the following people contributed to the 2.17.0 release. Thank you to all contributors! + +Ahmet Altay, Alan Myrvold, Alexey Romanenko, Andre-Philippe Paquet, Andrew +Pilloud, angulartist, Ankit Jhalaria, Ankur Goenka, Anton Kedin, Aryan Naraghi, +Aurélien Geron, B M VISHWAS, Bartok Jozsef, Boyuan Zhang, Brian Hulette, Cerny +Ondrej, Chad Dombrova, Chamikara Jayalath, ChethanU, cmach, Colm O hEigeartaigh, +Cyrus Maden, Daniel Oliveira, Daniel Robert, Dante, David Cavazos, David +Moravek, David Yan, Enrico Canzonieri, Etienne Chauchot, gxercavins, Hai Lu, +Hannah Jiang, Ian Lance Taylor, Ismaël Mejía, Israel Herraiz, James Wen, Jan +Lukavský, Jean-Baptiste Onofré, Jeff Klukas, jesusrv1103, Jofre, Kai Jiang, +Kamil Wasilewski, Kasia Kucharczyk, Kenneth Knowles, Kirill Kozlov, +kirillkozlov, Kohki YAMAGIWA, Kyle Weaver, Leonardo Alves Miguel, lloigor, +lostluck, Luis Enrique Ortíz Ramirez, Luke Cwik, Mark Liu, Maximilian Michels, +Michal Walenia, Mikhail Gryzykhin, mrociorg, Nicolas Delsaux, Ning Kang, NING +KANG, Pablo Estrada, pabloem, Piotr Szczepanik, rahul8383, Rakesh Kumar, Renat +Nasyrov, Reuven Lax, Robert Bradshaw, Robert Burke, Rui Wang, Ruslan Altynnikov, +Ryan Skraba, Salman Raza, Saul Chavez, Sebastian Jambor, sunjincheng121, Tatu +Saloranta, tchiarato, Thomas Weise, Tomo Suzuki, Tudor Marian, tvalentyn, Udi +Meiri, Valentyn Tymofieiev, Viola Lyu, Vishwas, Yichi Zhang, Yifan Zou, Yueyang +Qiu, Łukasz Gajowy + From 8724133538e8fb90f5851552779e3a01d0fcc752 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin Date: Wed, 18 Dec 2019 15:38:08 -0800 Subject: [PATCH 12/67] Add blogpost highlights --- website/src/_posts_2019-12-16-beam-2.17.0.md | 36 +++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/website/src/_posts_2019-12-16-beam-2.17.0.md b/website/src/_posts_2019-12-16-beam-2.17.0.md index 2c62845e171e..1a10f136c44d 100644 --- a/website/src/_posts_2019-12-16-beam-2.17.0.md +++ b/website/src/_posts_2019-12-16-beam-2.17.0.md @@ -30,18 +30,44 @@ For more information on changes in 2.17.0, check out the [detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12345970&projectId=12319527). ## Highlights -WIP +* [BEAM-7962](https://issues.apache.org/jira/browse/BEAM-7962) - Drop support for Flink 1.5 and 1.6 +* [BEAM-7635](https://issues.apache.org/jira/browse/BEAM-7635) - Migrate SnsIO to AWS SDK for Java 2 +* [BEAM-8483](https://issues.apache.org/jira/browse/BEAM-8483) - Make beam_fn_api flag opt-out rather than opt-in for runners. ### New Features / Improvements -WIP +* [BEAM-7730](https://issues.apache.org/jira/browse/BEAM-7730) - Add Flink 1.9 build target and Make FlinkRunner compatible with Flink 1.9 +* [BEAM-7990](https://issues.apache.org/jira/browse/BEAM-7990) - Add ability to read parquet files into PCollection +* [BEAM-8183](https://issues.apache.org/jira/browse/BEAM-8183) - Optionally bundle multiple pipelines into a single Flink jar +* [BEAM-8355](https://issues.apache.org/jira/browse/BEAM-8355) - Make BooleanCoder a standard coder +* [BEAM-8372](https://issues.apache.org/jira/browse/BEAM-8372) - Allow submission of Flink UberJar directly to flink cluster. +* [BEAM-8394](https://issues.apache.org/jira/browse/BEAM-8394) - Add withDataSourceConfiguration() method in JdbcIO.ReadRows class +* [BEAM-5428](https://issues.apache.org/jira/browse/BEAM-5428) - Implement cross-bundle state caching. +* [BEAM-5967](https://issues.apache.org/jira/browse/BEAM-5967) - Add handling of DynamicMessage in ProtoCoder +* [BEAM-7473](https://issues.apache.org/jira/browse/BEAM-7473) - Update RestrictionTracker within Python to not be required to be thread safe +* [BEAM-7920](https://issues.apache.org/jira/browse/BEAM-7920) - AvroTableProvider +* [BEAM-8098](https://issues.apache.org/jira/browse/BEAM-8098) - Improve documentation on BigQueryIO +* [BEAM-8100](https://issues.apache.org/jira/browse/BEAM-8100) - Add exception handling to Json transforms in Java SDK +* [BEAM-8138](https://issues.apache.org/jira/browse/BEAM-8138) - Fix code snippets in FileIO Java docs +* [BEAM-8306](https://issues.apache.org/jira/browse/BEAM-8306) - improve estimation of data byte size reading from source in ElasticsearchIO +* [BEAM-8312](https://issues.apache.org/jira/browse/BEAM-8312) - Flink portable pipeline jars do not need to stage artifacts remotely +* [BEAM-8351](https://issues.apache.org/jira/browse/BEAM-8351) - Support passing in arbitrary KV pairs to sdk worker via external environment config +* [BEAM-8396](https://issues.apache.org/jira/browse/BEAM-8396) - Default to LOOPBACK mode for local flink (spark, ...) runner. +* [BEAM-8410](https://issues.apache.org/jira/browse/BEAM-8410) - JdbcIO should support setConnectionInitSqls in its DataSource +* [BEAM-8417](https://issues.apache.org/jira/browse/BEAM-8417) - Expose ExternalWorkerHandler hostname +* [BEAM-8471](https://issues.apache.org/jira/browse/BEAM-8471) - Flink native job submission for portable pipelines +* [BEAM-8609](https://issues.apache.org/jira/browse/BEAM-8609) - Add HllCount to Java transform catalog +* [BEAM-8861](https://issues.apache.org/jira/browse/BEAM-8861) - Disallow self-signed certificates by default in ElasticsearchIO ### Dependency Changes -WIP +* [BEAM-8285](https://issues.apache.org/jira/browse/BEAM-8285) - Upgrade ZetaSQL to 2019.09.1 +* [BEAM-8392](https://issues.apache.org/jira/browse/BEAM-8392) - Upgrade pyarrow version bounds: 0.15.1<= to <0.16.0 +* [BEAM-5895](https://issues.apache.org/jira/browse/BEAM-5895) - Upgrade com.rabbitmq:amqp-client to 5.7.3 +* [BEAM-6896](https://issues.apache.org/jira/browse/BEAM-6896) - Upgrade PyYAML version bounds: 3.12<= to <6.0.0 -### Bugfixes +### Bugfixes * Various bug fixes and performance improvements. - +* [BEAM-8819] - AvroCoder for SpecificRecords is not serialized correctly since 2.13.0 ### Known Issues From 4f1a2bb25e8f5851c8893a5066db456668ce8c29 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin <12602502+Ardagan@users.noreply.github.com> Date: Thu, 19 Dec 2019 13:05:55 -0800 Subject: [PATCH 13/67] Update release notes version to correct one. --- website/src/get-started/downloads.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/src/get-started/downloads.md b/website/src/get-started/downloads.md index 75842edc954c..9f4e2465e0c3 100644 --- a/website/src/get-started/downloads.md +++ b/website/src/get-started/downloads.md @@ -95,7 +95,7 @@ Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.17.0 [SHA-512](https://www.apache.org/dist/beam/2.17.0/apache-beam-2.17.0-source-release.zip.sha512). [signature](https://www.apache.org/dist/beam/2.17.0/apache-beam-2.17.0-source-release.zip.asc). -[Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12345494). +[Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12345970). ## 2.16.0 (2019-10-07) Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.16.0/apache-beam-2.16.0-source-release.zip). From 2375217840aed6ee9d496c51aea32a4efa0b8a40 Mon Sep 17 00:00:00 2001 From: Ning Kang Date: Tue, 17 Dec 2019 11:47:25 -0800 Subject: [PATCH 14/67] [BEAM-8977] Resolve test flakiness 1. Removed test logic depending on execution of asynchronous tasks since there is no control of them in a testing environment. 2. Replaced the dynamic plotting tests with tests directly/indirectly invoking underlying logic of the asynchronous task. --- .../display/pcoll_visualization_test.py | 70 ++++++++----------- 1 file changed, 31 insertions(+), 39 deletions(-) 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 624688e0d94a..b2d330e20434 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,53 +87,46 @@ def test_dynamic_plotting_return_handle(self): h.stop() @patch('apache_beam.runners.interactive.display.pcoll_visualization' - '.PCollectionVisualization.display_facets') - def test_dynamic_plotting_update_same_display(self, - mocked_display_facets): + '.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) + + 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, is_main_job=True) - # 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() + # When job is running, the dynamic plotting will not be stopped. + self.assertFalse(ie.current_env().is_terminated(self._p)) - @patch('timeloop.Timeloop.stop') - def test_auto_stop_dynamic_plotting_when_job_is_terminated( - self, - mocked_timeloop): - fake_pipeline_result = runner.PipelineResult(runner.PipelineState.RUNNING) - ie.current_env().set_pipeline_result( - self._p, - fake_pipeline_result, - is_main_job=True) - # 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() fake_pipeline_result = runner.PipelineResult(runner.PipelineState.DONE) ie.current_env().set_pipeline_result( self._p, fake_pipeline_result, is_main_job=True) - # 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() + # 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, From 6f7b2b29131de0b3b8bc1a55a15a6b42b1bb56d9 Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Mon, 23 Dec 2019 16:18:01 -0800 Subject: [PATCH 15/67] Refactor shared uber jar generation code into common subclass. --- .../portability/abstract_job_service.py | 78 ++++++++++++++++++- .../portability/flink_uber_jar_job_server.py | 71 +---------------- .../portability/spark_uber_jar_job_server.py | 71 +---------------- 3 files changed, 85 insertions(+), 135 deletions(-) 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 e2852578d93b..df7f9d22684b 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 @@ def to_runner_api(self): 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 b3189716ffbe..9552df8bc4af 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,72 +81,17 @@ def create_beam_job(self, job_id, job_name, pipeline, options): 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) 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 index 7ddf31e142c8..7bad8db1a443 100644 --- 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 @@ -21,24 +21,16 @@ from __future__ import print_function import itertools -import json import logging -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 from apache_beam.utils.timestamp import Timestamp @@ -84,78 +76,23 @@ def create_beam_job(self, job_id, job_name, pipeline, options): artifact_port=self._artifact_port) -class SparkBeamJob(abstract_job_service.AbstractBeamJob): +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.""" - # 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(SparkBeamJob, self).__init__(job_id, job_name, pipeline, options) + super(SparkBeamJob, 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 # Message history is a superset of state history. self._message_history = self._state_history[:] - 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): url = '%s/%s' % (self._master_url, path) response = method(url, **kwargs) From fc935255a733146c76caf971187e69692fdb2acb Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Tue, 24 Dec 2019 11:21:56 +0800 Subject: [PATCH 16/67] [BEAM-8935] Fail fast if sdk harness startup failed. --- .../environment/DockerEnvironmentFactory.java | 40 +++++++++++-------- .../EmbeddedEnvironmentFactory.java | 20 +++++++++- .../ProcessEnvironmentFactory.java | 2 +- .../DockerEnvironmentFactoryTest.java | 15 +++---- 4 files changed, 47 insertions(+), 30 deletions(-) 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 154aaecc5bb9..5ec4a3989b11 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 @@ public RemoteEnvironment createEnvironment(Environment environment) throws Excep 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 a6967701d4a0..a06a5d7bd444 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 @@ public RemoteEnvironment createEnvironment(Environment environment) throws Excep } }); - // 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 a90a24571d52..539d76ad2e03 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 @@ public RemoteEnvironment createEnvironment(Environment environment) throws Excep 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/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 04dc1486cfcd..f1636400ef19 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.ArgumentMatchers.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 @@ public void createsCorrectEnvironment() throws Exception { } @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 @@ public void logsDockerOutputOnClose() throws Exception { @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"); From 961b8ab7752b6079b0dd76c8b8f25e2da9292f0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C3=96zt=C3=BCrk?= Date: Thu, 26 Dec 2019 14:56:55 +0300 Subject: [PATCH 17/67] Python example parameters fix Current example throws: `TypeError: Runner PipelineOptions() is not a PipelineRunner object or the name of a registered runner` --- website/src/documentation/runners/flink.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/src/documentation/runners/flink.md b/website/src/documentation/runners/flink.md index 67ba23bf0cdc..6e7307fe54e0 100644 --- a/website/src/documentation/runners/flink.md +++ b/website/src/documentation/runners/flink.md @@ -336,7 +336,7 @@ options = PipelineOptions([ "--flink_master=localhost:8081", "--environment_type=LOOPBACK" ]) -with beam.Pipeline(options) as p: +with beam.Pipeline(options=options) as p: ... ``` From 93e9f31da8d7efa98e8b7be066627642845b8bf2 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Fri, 20 Dec 2019 16:13:45 -0800 Subject: [PATCH 18/67] Fix BytesValue unparsing --- .../bigquery/BeamSqlUnparseContext.java | 63 +++++++++++++++++++ .../sql/zetasql/BeamZetaSqlCalcRel.java | 3 +- .../sql/zetasql/ZetaSQLQueryPlanner.java | 4 +- .../sql/zetasql/ZetaSQLDialectSpecTest.java | 23 +++++++ 4 files changed, 90 insertions(+), 3 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java 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 000000000000..99b61fe59218 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java @@ -0,0 +1,63 @@ +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.SqlDialect; +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; + +public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext { + + public BeamSqlUnparseContext(SqlDialect dialect, IntFunction field) { + super(dialect, field); + } + + @Override + public SqlNode toSql(RexProgram program, RexNode rex) { + if (rex.getKind().equals(SqlKind.LITERAL)) { + final RexLiteral literal = (RexLiteral) rex; + if (literal.getTypeName().getFamily().equals(SqlTypeFamily.BINARY)) { + + return new SqlByteStringLiteral(BitString.createFromBytes(literal.getValueAs(byte[].class)), POS); + } + } + + return super.toSql(program, rex); + } + + private 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/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 index 67b2c328a61c..8cc8d3ed2daa 100644 --- 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 @@ -32,6 +32,7 @@ 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; @@ -73,7 +74,7 @@ public BeamZetaSqlCalcRel( i -> new SqlIdentifier( getProgram().getInputRowType().getFieldList().get(i).getName(), SqlParserPos.ZERO); - context = new SqlImplementor.SimpleContext(DIALECT, fn); + context = new BeamSqlUnparseContext(DIALECT, fn); } @Override 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 0f5ec160caa8..8e6597f7bc26 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 @@ -73,8 +73,8 @@ public ZetaSQLQueryPlanner(JdbcConnection jdbcConnection, RuleSet[] 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(); + return replaceBeamCalcRule(BeamRuleSets.getRuleSets()); + //return BeamRuleSets.getRuleSets(); } private static RuleSet[] replaceBeamCalcRule(RuleSet[] ruleSets) { 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 2bccfe8f23a7..80bb94e638ae 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 @@ -140,6 +140,29 @@ public void testSimpleSelect() { pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); } + @Test + public void testCeil() { + String sql = "SELECT @p0 IS NULL AS ColA"; + + ByteString byteString = ByteString.copyFrom(new byte[] { 0x62 }); + + ImmutableMap params = + ImmutableMap.builder() + .put("p0", Value.createBytesValue(byteString)) + .build(); + + ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); + BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); + PCollection 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 testEQ1() { String sql = "SELECT @p0 = @p1 AS ColA"; From 2996adc7578aed0c5a78b6d4eed2bbc685b84e84 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Mon, 23 Dec 2019 11:24:47 -0800 Subject: [PATCH 19/67] Fix floating point literals --- .../sdk/extensions/sql/zetasql/ZetaSqlUtils.java | 3 +++ .../sql/zetasql/ZetaSQLDialectSpecTest.java | 16 ++++++++++++++++ 2 files changed, 19 insertions(+) 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 index a548a0ce7b31..391041a7edb8 100644 --- 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 @@ -158,6 +158,9 @@ public static Object zetaSqlValueToJavaObject(Value value, FieldType fieldType) case DECIMAL: return value.getNumericValue(); case DOUBLE: + if (value.getType().getKind().equals(TypeKind.TYPE_INT64)) { + return (double) value.getInt64Value(); + } return value.getDoubleValue(); case STRING: return value.getStringValue(); 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 80bb94e638ae..ef7c68d7421d 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 @@ -163,6 +163,22 @@ public void testCeil() { 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 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 testEQ1() { String sql = "SELECT @p0 = @p1 AS ColA"; From 4484f6317c4bff0c77848884883bf83252ea5ac0 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Mon, 23 Dec 2019 14:04:29 -0800 Subject: [PATCH 20/67] Fix string literals --- .../bigquery/BeamSqlUnparseContext.java | 12 ++++++++---- .../sql/zetasql/ZetaSQLDialectSpecTest.java | 17 +++++++++++++++++ 2 files changed, 25 insertions(+), 4 deletions(-) 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 index 99b61fe59218..bfc517bf331b 100644 --- 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 @@ -16,6 +16,7 @@ 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 { @@ -28,15 +29,18 @@ public SqlNode toSql(RexProgram program, RexNode rex) { if (rex.getKind().equals(SqlKind.LITERAL)) { final RexLiteral literal = (RexLiteral) rex; if (literal.getTypeName().getFamily().equals(SqlTypeFamily.BINARY)) { - - return new SqlByteStringLiteral(BitString.createFromBytes(literal.getValueAs(byte[].class)), POS); + BitString bitString = BitString.createFromBytes(literal.getValueAs(byte[].class)); + return new SqlByteStringLiteral(bitString, POS); + } else if (literal.getTypeName().getFamily().equals(SqlTypeFamily.CHARACTER)) { + String escaped = StringEscapeUtils.escapeJava(literal.getValueAs(String.class)); + return SqlLiteral.createCharString(escaped, POS); } } return super.toSql(program, rex); } - private class SqlByteStringLiteral extends SqlLiteral { + private static class SqlByteStringLiteral extends SqlLiteral { SqlByteStringLiteral(BitString bytes, SqlParserPos pos) { super(bytes, SqlTypeName.BINARY, pos); @@ -44,7 +48,7 @@ private class SqlByteStringLiteral extends SqlLiteral { @Override public SqlByteStringLiteral clone(SqlParserPos pos) { - return new SqlByteStringLiteral((BitString)this.value, pos); + return new SqlByteStringLiteral((BitString) this.value, pos); } @Override 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 ef7c68d7421d..2f4f63792ef5 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 @@ -141,6 +141,7 @@ public void testSimpleSelect() { } @Test + // TODO: update test name to something more accurate. public void testCeil() { String sql = "SELECT @p0 IS NULL AS ColA"; @@ -179,6 +180,22 @@ public void testFloat() { 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 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"; From 2994da212edc907b78ab8ee8a6c046f46802c90c Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Mon, 23 Dec 2019 14:46:51 -0800 Subject: [PATCH 21/67] Add null check for SqlTypeFamily --- .../provider/bigquery/BeamSqlUnparseContext.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) 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 index bfc517bf331b..8b06c48a73e1 100644 --- 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 @@ -28,12 +28,15 @@ public BeamSqlUnparseContext(SqlDialect dialect, IntFunction field) { public SqlNode toSql(RexProgram program, RexNode rex) { if (rex.getKind().equals(SqlKind.LITERAL)) { final RexLiteral literal = (RexLiteral) rex; - if (literal.getTypeName().getFamily().equals(SqlTypeFamily.BINARY)) { - BitString bitString = BitString.createFromBytes(literal.getValueAs(byte[].class)); - return new SqlByteStringLiteral(bitString, POS); - } else if (literal.getTypeName().getFamily().equals(SqlTypeFamily.CHARACTER)) { - String escaped = StringEscapeUtils.escapeJava(literal.getValueAs(String.class)); - return SqlLiteral.createCharString(escaped, POS); + SqlTypeFamily family = literal.getTypeName().getFamily(); + if (family != null) { + if (family.equals(SqlTypeFamily.BINARY)) { + BitString bitString = BitString.createFromBytes(literal.getValueAs(byte[].class)); + return new SqlByteStringLiteral(bitString, POS); + } else if (family.equals(SqlTypeFamily.CHARACTER)) { + String escaped = StringEscapeUtils.escapeJava(literal.getValueAs(String.class)); + return SqlLiteral.createCharString(escaped, POS); + } } } From a831aef1579767c2b5a7d619b2e6b50b1a30c708 Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Mon, 23 Dec 2019 15:47:38 -0800 Subject: [PATCH 22/67] ZetaSqlCalcRule should be disaled by defualt --- .../beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 8e6597f7bc26..0f5ec160caa8 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 @@ -73,8 +73,8 @@ public ZetaSQLQueryPlanner(JdbcConnection jdbcConnection, RuleSet[] 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(); + // return replaceBeamCalcRule(BeamRuleSets.getRuleSets()); + return BeamRuleSets.getRuleSets(); } private static RuleSet[] replaceBeamCalcRule(RuleSet[] ruleSets) { From af6f6b81f42c57fb4213b3796dbb83796349ec9b Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Mon, 23 Dec 2019 15:49:15 -0800 Subject: [PATCH 23/67] spotles --- .../bigquery/BeamSqlUnparseContext.java | 17 +++++++++++++++++ .../sql/zetasql/ZetaSQLDialectSpecTest.java | 18 ++++++------------ 2 files changed, 23 insertions(+), 12 deletions(-) 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 index 8b06c48a73e1..ebdacf78ff23 100644 --- 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 @@ -1,3 +1,20 @@ +/* + * 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; 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 2f4f63792ef5..e140c02da04f 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 @@ -141,16 +141,13 @@ public void testSimpleSelect() { } @Test - // TODO: update test name to something more accurate. - public void testCeil() { + public void testByteString() { String sql = "SELECT @p0 IS NULL AS ColA"; - ByteString byteString = ByteString.copyFrom(new byte[] { 0x62 }); + ByteString byteString = ByteString.copyFrom(new byte[] {0x62}); ImmutableMap params = - ImmutableMap.builder() - .put("p0", Value.createBytesValue(byteString)) - .build(); + ImmutableMap.builder().put("p0", Value.createBytesValue(byteString)).build(); ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); @@ -158,8 +155,7 @@ public void testCeil() { final Schema schema = Schema.builder().addNullableField("ColA", FieldType.BOOLEAN).build(); - PAssert.that(stream) - .containsInAnyOrder(Row.withSchema(schema).addValues(false).build()); + PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(false).build()); pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); } @@ -174,8 +170,7 @@ public void testFloat() { final Schema schema = Schema.builder().addNullableField("ColA", FieldType.DOUBLE).build(); - PAssert.that(stream) - .containsInAnyOrder(Row.withSchema(schema).addValues(3.0).build()); + PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(3.0).build()); pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); } @@ -190,8 +185,7 @@ public void testStringLiterals() { final Schema schema = Schema.builder().addNullableField("ColA", FieldType.STRING).build(); - PAssert.that(stream) - .containsInAnyOrder(Row.withSchema(schema).addValues("abc\n").build()); + PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues("abc\n").build()); pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); } From ea6ef3cfdc2a7df8b7b48e7a9555ce28076c4e56 Mon Sep 17 00:00:00 2001 From: Heejong Lee Date: Thu, 26 Dec 2019 17:27:04 -0800 Subject: [PATCH 24/67] [BEAM-9034] Update environment_id for ExternalTransform in Python SDK --- sdks/python/apache_beam/transforms/external.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py index 3fb90f2843a9..bf458021b6a8 100644 --- a/sdks/python/apache_beam/transforms/external.py +++ b/sdks/python/apache_beam/transforms/external.py @@ -419,7 +419,8 @@ def _normalize(coder_proto): 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): From 0ac760ae5b7ae874d00c0c39d973563be02775c5 Mon Sep 17 00:00:00 2001 From: Yu Feng Date: Fri, 27 Dec 2019 15:18:43 -0800 Subject: [PATCH 25/67] Catch __module__ is None. The current code crashes when __module__ is None. According to https://docs.python.org/3/reference/datamodel.html#the-standard-type-hierarchy __module__ is "The name of the module the function was defined in, or None if unavailable." As a very particular example, Protobuf sets generated type's __module__ to None: https://github.com/protocolbuffers/protobuf/blob/master/python/google/protobuf/message_factory.py#L85 --- sdks/python/apache_beam/internal/pickler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py index 3cca9d398165..02968406f261 100644 --- a/sdks/python/apache_beam/internal/pickler.py +++ b/sdks/python/apache_beam/internal/pickler.py @@ -75,6 +75,7 @@ def __exit__(self, *unused_exc_info): 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__) From b3bb77d7e341d1eb2c958486e38f6f2ce887fd55 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Mon, 30 Dec 2019 11:43:56 -0800 Subject: [PATCH 26/67] [BEAM-9012] Change __init__ hints so they work with pytype (#10466) * Change __init__ hints so they work with pytype * fixup stray "type:" and whitespace * fixup whitespace Co-authored-by: Udi Meiri --- sdks/python/apache_beam/options/pipeline_options.py | 5 ++--- sdks/python/apache_beam/pipeline.py | 7 ++----- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 778e06107f46..d3cc23f2734c 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -169,9 +169,8 @@ def _add_argparse_args(cls, parser): 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 diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 1e89772f7381..f42f44a03a7f 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -119,11 +119,8 @@ def sdk_transforms_with_environment(cls): frozenset([common_urns.primitives.ASSIGN_WINDOWS.urn])] return frozenset().union(*sets) - def __init__(self, - runner=None, # type: Optional[Union[str, PipelineRunner]] - options=None, # type: Optional[PipelineOptions] - argv=None # type: Optional[List[str]] - ): + 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: From bb2c9f19b8bf87f2cd62e0e19bd2e18e0ac41526 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Mon, 30 Dec 2019 13:49:55 -0800 Subject: [PATCH 27/67] [BEAM-9039] Fix race on reading channel readErr. (#10456) Close off a race condition where a closing DataChannel might have new readers created for it while it is failing, causing stuckness in the bundles. In particular, the c.readErr must be interacted while c.mu is held. Otherwise something like the following happens. Given a channel C, and goroutines G1,G2. 1. G1 A request for a new reader on C arrives, checks C.readErr finds it null. 2. G2 An error occurs on reading. The lock is acquired, and C.readErr is set. Readers are closed. The channel is officially closed with A.forceRecreate, removing it from the DataManager cache. 3. G1 calls A.makeReader, and creates a new reader there. There could be an arbitrary number of G1s. --- sdks/go/pkg/beam/core/runtime/harness/datamgr.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 7e9f7b146e98..76153565a4ae 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 @@ type DataChannel struct { 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 @@ func (c *DataChannel) terminateStreamOnError(err error) { // 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 @@ func (c *DataChannel) read(ctx context.Context) { 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 @@ func (r *errReader) Close() error { 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 } From eadf91b0a0991b2b2d11fd7e236b62373a50b094 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Mon, 30 Dec 2019 14:39:20 -0800 Subject: [PATCH 28/67] [BEAM-5605] Increase precision of fraction used during splitting. --- .../src/main/proto/beam_fn_api.proto | 2 +- .../pkg/beam/core/runtime/exec/datasource.go | 2 +- sdks/go/pkg/beam/core/runtime/exec/plan.go | 2 +- .../model/fnexecution_v1/beam_fn_api.pb.go | 408 ++++----- .../model/pipeline_v1/beam_runner_api.pb.go | 783 ++++++++++-------- 5 files changed, 642 insertions(+), 555 deletions(-) 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 3710d960c443..f8bcf8bef2c9 100644 --- a/model/fn-execution/src/main/proto/beam_fn_api.proto +++ b/model/fn-execution/src/main/proto/beam_fn_api.proto @@ -426,7 +426,7 @@ message ProcessBundleSplitRequest { // // 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. diff --git a/sdks/go/pkg/beam/core/runtime/exec/datasource.go b/sdks/go/pkg/beam/core/runtime/exec/datasource.go index 85f8bcad6c20..06131b7c6107 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 @@ func (n *DataSource) Progress() ProgressReportSnapshot { // 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") } diff --git a/sdks/go/pkg/beam/core/runtime/exec/plan.go b/sdks/go/pkg/beam/core/runtime/exec/plan.go index d221c7e87efb..d87ff54ea733 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 @@ func (p *Plan) Metrics() *fnpb.Metrics { 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/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go index 902133cd662f..c4cb943dad97 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 @@ -1821,7 +1821,7 @@ type ProcessBundleSplitRequest_DesiredSplit struct { // // 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"` @@ -1861,7 +1861,7 @@ func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_DiscardUnknown() { 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 } @@ -3660,208 +3660,208 @@ func init() { func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_6d954c03a4758710) } var fileDescriptor_6d954c03a4758710 = []byte{ - // 3214 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xcd, 0x6f, 0x1b, 0xc7, - 0xd9, 0xf7, 0xf2, 0x43, 0x22, 0x1f, 0x52, 0x12, 0x39, 0x92, 0x6c, 0x7a, 0x5f, 0xe7, 0x7d, 0x1d, - 0xbe, 0x0d, 0x20, 0xa4, 0x08, 0xfd, 0x89, 0xc4, 0x4e, 0x13, 0x27, 0x12, 0x45, 0xdb, 0x8c, 0x65, - 0x9b, 0x5d, 0xd9, 0x75, 0x9b, 0x34, 0x59, 0xac, 0xb8, 0x43, 0x7a, 0x61, 0x72, 0x77, 0x33, 0xb3, - 0x94, 0x25, 0x37, 0x68, 0xfa, 0x85, 0x16, 0x2d, 0xda, 0xe6, 0xd2, 0x43, 0xda, 0x5b, 0x5b, 0xa0, - 0x40, 0x2f, 0xfd, 0x03, 0x72, 0xed, 0xa1, 0xe8, 0xa9, 0xff, 0x40, 0x2e, 0x05, 0x5a, 0xa0, 0x6d, - 0x7a, 0x2e, 0xd0, 0x5b, 0x31, 0x1f, 0xfb, 0xc1, 0xe5, 0xd2, 0x59, 0x52, 0x72, 0x6f, 0x3b, 0x33, - 0xfb, 0xfc, 0x7e, 0x33, 0xcf, 0xfe, 0xe6, 0x99, 0x67, 0x66, 0x16, 0xaa, 0x7b, 0xd8, 0x18, 0xea, - 0x3d, 0x5b, 0x37, 0x5c, 0xab, 0xe1, 0x12, 0xc7, 0x73, 0xd0, 0x0b, 0x0e, 0xe9, 0x37, 0x0c, 0xd7, - 0xe8, 0x3e, 0xc4, 0x0d, 0xd6, 0xda, 0x18, 0x3a, 0x26, 0x1e, 0x34, 0x7a, 0xb6, 0x8e, 0x0f, 0x70, - 0x77, 0xe4, 0x59, 0x8e, 0xdd, 0xd8, 0xbf, 0xa0, 0xae, 0x73, 0x4b, 0x32, 0xb2, 0x6d, 0x4c, 0x42, - 0x6b, 0x75, 0x05, 0xdb, 0xa6, 0xeb, 0x58, 0xb6, 0x47, 0x65, 0xc5, 0xd9, 0xbe, 0xe3, 0xf4, 0x07, - 0xf8, 0x1c, 0x2f, 0xed, 0x8d, 0x7a, 0xe7, 0x4c, 0x4c, 0xbb, 0xc4, 0x72, 0x3d, 0x87, 0xc8, 0x37, - 0xfe, 0x2f, 0xfe, 0x86, 0x67, 0x0d, 0x31, 0xf5, 0x8c, 0xa1, 0x2b, 0x5f, 0xf8, 0xdf, 0x09, 0x88, - 0x11, 0x31, 0x78, 0x3f, 0xa6, 0xb4, 0x3f, 0x26, 0x86, 0xeb, 0x62, 0xe2, 0x77, 0x61, 0x69, 0x88, - 0x3d, 0x62, 0x75, 0x65, 0xb1, 0xfe, 0x0b, 0x05, 0x96, 0x35, 0x3c, 0x74, 0x3c, 0x7c, 0x83, 0xb8, - 0xdd, 0x8e, 0x43, 0x3c, 0x34, 0x84, 0x93, 0x86, 0x6b, 0xe9, 0x14, 0x93, 0x7d, 0xab, 0x8b, 0xf5, - 0xb0, 0x8b, 0x35, 0xe5, 0xac, 0xb2, 0x51, 0xba, 0xf8, 0x4a, 0x23, 0xd9, 0x29, 0xae, 0xe5, 0xe2, - 0x81, 0x65, 0xe3, 0xc6, 0xfe, 0x85, 0xc6, 0xa6, 0x6b, 0xed, 0x0a, 0xfb, 0xed, 0xc0, 0x5c, 0x5b, - 0x33, 0x12, 0x6a, 0xd1, 0x69, 0x28, 0x74, 0x1d, 0x13, 0x13, 0xdd, 0x32, 0x6b, 0x99, 0xb3, 0xca, - 0x46, 0x51, 0x5b, 0xe4, 0xe5, 0xb6, 0x59, 0xff, 0x6b, 0x0e, 0x50, 0xdb, 0xa6, 0x1e, 0x19, 0x75, - 0xd9, 0x08, 0x35, 0xfc, 0xfe, 0x08, 0x53, 0x0f, 0xbd, 0x00, 0xcb, 0x56, 0x58, 0xcb, 0xec, 0x14, - 0x6e, 0xb7, 0x14, 0xa9, 0x6d, 0x9b, 0xe8, 0x3e, 0x14, 0x08, 0xee, 0x5b, 0xd4, 0xc3, 0xa4, 0xf6, - 0xb7, 0x45, 0xde, 0xf5, 0x97, 0x1b, 0xa9, 0xbe, 0x67, 0x43, 0x93, 0x76, 0x92, 0xf1, 0xe6, 0x09, - 0x2d, 0x80, 0x42, 0x18, 0x96, 0x5d, 0xe2, 0x74, 0x31, 0xa5, 0xfa, 0xde, 0xc8, 0x36, 0x07, 0xb8, - 0xf6, 0x77, 0x01, 0xfe, 0xa5, 0x94, 0xe0, 0x1d, 0x61, 0xbd, 0xc5, 0x8d, 0x43, 0x86, 0x25, 0x37, - 0x5a, 0x8f, 0xbe, 0x09, 0xa7, 0xc6, 0x69, 0x74, 0x97, 0x38, 0x7d, 0x82, 0x29, 0xad, 0xfd, 0x43, - 0xf0, 0x35, 0xe7, 0xe1, 0xeb, 0x48, 0x90, 0x90, 0x77, 0xdd, 0x4d, 0x6a, 0x47, 0x23, 0x58, 0x8b, - 0xf1, 0x53, 0x77, 0x60, 0x79, 0xb5, 0xcf, 0x04, 0xf9, 0x9b, 0xf3, 0x90, 0xef, 0x32, 0x84, 0x90, - 0x19, 0xb9, 0x13, 0x8d, 0xe8, 0x21, 0xac, 0xf4, 0x2c, 0xdb, 0x18, 0x58, 0x4f, 0xb0, 0xef, 0xde, - 0x7f, 0x0a, 0xc6, 0xd7, 0x52, 0x32, 0x5e, 0x97, 0xe6, 0x71, 0xff, 0x2e, 0xf7, 0xc6, 0x1a, 0xb6, - 0x8a, 0xb0, 0x48, 0x44, 0x63, 0xfd, 0x3b, 0x79, 0x58, 0x1d, 0xd3, 0x19, 0x75, 0x1d, 0x9b, 0xe2, - 0xb4, 0x42, 0x5b, 0x83, 0x3c, 0x26, 0xc4, 0x21, 0x52, 0xbe, 0xa2, 0x80, 0xbe, 0x32, 0x29, 0xbf, - 0x57, 0x66, 0x96, 0x9f, 0xe8, 0xc8, 0x98, 0xfe, 0x7a, 0xd3, 0xf4, 0xf7, 0xda, 0x7c, 0xfa, 0x0b, - 0x28, 0x62, 0x02, 0xfc, 0xf0, 0x73, 0x05, 0xb8, 0x7d, 0x34, 0x01, 0x06, 0xc4, 0x53, 0x14, 0xb8, - 0xff, 0x74, 0x05, 0x6e, 0x1e, 0x41, 0x81, 0x01, 0x75, 0x92, 0x04, 0xad, 0xa9, 0x12, 0x7c, 0x7d, - 0x4e, 0x09, 0x06, 0x74, 0x71, 0x0d, 0x02, 0xd3, 0x88, 0x68, 0xad, 0xff, 0x44, 0x81, 0x95, 0x58, - 0xdc, 0x41, 0x4f, 0xe0, 0x74, 0xcc, 0x05, 0x63, 0xd1, 0x38, 0xbb, 0x51, 0xba, 0x78, 0x6d, 0x1e, - 0x37, 0x44, 0x82, 0xf2, 0x29, 0x37, 0xb9, 0xa1, 0x8e, 0xa0, 0x12, 0xd7, 0x61, 0xfd, 0xd7, 0x00, - 0xa7, 0xa6, 0x00, 0xa1, 0x65, 0xc8, 0x04, 0x13, 0x24, 0x63, 0x99, 0xc8, 0x06, 0xf0, 0x88, 0x61, - 0xd3, 0x9e, 0x43, 0x86, 0xb4, 0x96, 0xe1, 0x9d, 0xbd, 0x73, 0xb4, 0xce, 0x36, 0xee, 0x05, 0x80, - 0x2d, 0xdb, 0x23, 0x87, 0x5a, 0x84, 0x01, 0x79, 0x50, 0x76, 0xbb, 0xce, 0x60, 0x80, 0xf9, 0xb4, - 0xa4, 0xb5, 0x2c, 0x67, 0xec, 0x1c, 0x91, 0xb1, 0x13, 0x81, 0x14, 0x9c, 0x63, 0x2c, 0xe8, 0x47, - 0x0a, 0xac, 0x3d, 0xb6, 0x6c, 0xd3, 0x79, 0x6c, 0xd9, 0x7d, 0x9d, 0x7a, 0xc4, 0xf0, 0x70, 0xdf, - 0xc2, 0xb4, 0x96, 0xe3, 0xf4, 0x0f, 0x8e, 0x48, 0xff, 0xc0, 0x87, 0xde, 0x0d, 0x90, 0x45, 0x2f, - 0x56, 0x1f, 0x4f, 0xb6, 0xa0, 0x3d, 0x58, 0xe0, 0x4b, 0x27, 0xad, 0xe5, 0x39, 0xfb, 0x5b, 0x47, - 0x64, 0x6f, 0x72, 0x30, 0x41, 0x28, 0x91, 0x99, 0x9b, 0xb1, 0xbd, 0x6f, 0x11, 0xc7, 0x1e, 0x62, - 0xdb, 0xa3, 0xb5, 0x85, 0x63, 0x71, 0x73, 0x2b, 0x02, 0x29, 0xdd, 0x1c, 0x65, 0x41, 0x07, 0x70, - 0x86, 0x7a, 0x86, 0x87, 0xf5, 0x29, 0x99, 0xc9, 0xe2, 0xd1, 0x32, 0x93, 0xd3, 0x1c, 0x3c, 0xa9, - 0x49, 0x1d, 0xc0, 0x4a, 0x4c, 0x75, 0xa8, 0x02, 0xd9, 0x47, 0xf8, 0x50, 0x4a, 0x9d, 0x3d, 0xa2, - 0x26, 0xe4, 0xf7, 0x8d, 0xc1, 0x08, 0xf3, 0x15, 0xa0, 0x74, 0xf1, 0xa5, 0x14, 0xfd, 0xe8, 0x04, - 0xa8, 0x9a, 0xb0, 0x7d, 0x35, 0x73, 0x45, 0x51, 0x1d, 0xa8, 0x4e, 0x28, 0x2e, 0x81, 0x6f, 0x7b, - 0x9c, 0xaf, 0x91, 0x86, 0xaf, 0x19, 0xc0, 0x46, 0x09, 0x3f, 0x80, 0xda, 0x34, 0x8d, 0x25, 0xf0, - 0xbe, 0x35, 0xce, 0x7b, 0x39, 0x05, 0x6f, 0x1c, 0xfd, 0x30, 0xca, 0xde, 0x85, 0x52, 0x44, 0x63, - 0x09, 0x84, 0xd7, 0xc6, 0x09, 0x37, 0x52, 0x10, 0x72, 0xc0, 0x98, 0x4f, 0x27, 0xe4, 0x75, 0x3c, - 0x3e, 0x8d, 0xc0, 0x46, 0x08, 0xeb, 0xff, 0xce, 0x42, 0x55, 0x28, 0x7c, 0xd3, 0x75, 0x07, 0x56, - 0x97, 0xa7, 0xe7, 0xe8, 0x79, 0x28, 0x07, 0xd1, 0x2a, 0x4c, 0x25, 0x4a, 0x41, 0x5d, 0xdb, 0x64, - 0xa9, 0xb0, 0x65, 0xbb, 0x23, 0x2f, 0x92, 0x0a, 0xf3, 0x72, 0xdb, 0x44, 0x35, 0x58, 0xc4, 0x03, - 0xcc, 0x98, 0x6a, 0xd9, 0xb3, 0xca, 0x46, 0x59, 0xf3, 0x8b, 0xe8, 0x1b, 0x50, 0x75, 0x46, 0x1e, - 0xb3, 0x7a, 0x6c, 0x78, 0x98, 0x0c, 0x0d, 0xf2, 0xc8, 0x8f, 0x3e, 0x69, 0xc3, 0xed, 0x44, 0x67, - 0x1b, 0x77, 0x39, 0xe2, 0x83, 0x00, 0x50, 0xcc, 0xc9, 0x8a, 0x13, 0xab, 0x46, 0x1d, 0x00, 0x8b, - 0xea, 0x7b, 0xce, 0xc8, 0x36, 0xb1, 0x59, 0xcb, 0x9f, 0x55, 0x36, 0x96, 0x2f, 0x5e, 0x48, 0xe1, - 0xb9, 0x36, 0xdd, 0x12, 0x36, 0x8d, 0x96, 0x3d, 0x1a, 0x6a, 0x45, 0xcb, 0x2f, 0xa3, 0xaf, 0x43, - 0x65, 0xe8, 0xd8, 0x96, 0xe7, 0x10, 0x16, 0x50, 0x2d, 0xbb, 0xe7, 0xf8, 0x31, 0x26, 0x0d, 0xee, - 0xed, 0xc0, 0xb4, 0x6d, 0xf7, 0x1c, 0x6d, 0x65, 0x38, 0x56, 0xa6, 0xaa, 0x0e, 0xeb, 0x89, 0x43, - 0x4b, 0xd0, 0xc3, 0xf9, 0x71, 0x3d, 0xa8, 0x0d, 0xb1, 0xb1, 0x6a, 0xf8, 0x1b, 0xab, 0xc6, 0x3d, - 0x7f, 0x67, 0x16, 0xfd, 0xf6, 0xbf, 0x57, 0xa0, 0xb6, 0x8d, 0x07, 0xc6, 0x21, 0x36, 0x27, 0x25, - 0xf0, 0x36, 0x94, 0x8c, 0xb0, 0x28, 0xb7, 0x53, 0x57, 0xe6, 0xfd, 0x48, 0x5a, 0x14, 0x0c, 0xdd, - 0x82, 0x35, 0x99, 0xce, 0x62, 0x53, 0x67, 0x9b, 0x46, 0xdd, 0x64, 0xdd, 0x90, 0xbd, 0x3f, 0x3d, - 0xd1, 0xfb, 0x6d, 0xb9, 0x6d, 0xd4, 0x50, 0x60, 0xc6, 0x06, 0xc4, 0xfb, 0x5e, 0xff, 0x69, 0x0e, - 0xd6, 0x92, 0xb6, 0x29, 0xe8, 0x0d, 0x38, 0x33, 0x35, 0x21, 0x09, 0x45, 0x7d, 0x7a, 0x4a, 0x4e, - 0xd1, 0x36, 0x91, 0x05, 0xe5, 0x2e, 0x1b, 0xa9, 0xee, 0x39, 0x8f, 0xb0, 0xed, 0xe7, 0x05, 0xd7, - 0x8f, 0xb0, 0x75, 0x6a, 0x34, 0x99, 0xd5, 0x3d, 0x06, 0xa7, 0x95, 0xba, 0xc1, 0x33, 0x55, 0xff, - 0x98, 0x01, 0x08, 0xdb, 0xd0, 0xfb, 0x00, 0x23, 0x8a, 0x89, 0xce, 0x43, 0xbd, 0xf4, 0x7d, 0xe7, - 0x78, 0x78, 0x1b, 0xf7, 0x29, 0x26, 0xbb, 0x0c, 0xf7, 0xe6, 0x09, 0xad, 0x38, 0xf2, 0x0b, 0x8c, - 0x92, 0x5a, 0x26, 0xd6, 0xf9, 0x24, 0x96, 0x5f, 0xe2, 0xb8, 0x28, 0x77, 0x2d, 0x13, 0xb7, 0x19, - 0x2e, 0xa3, 0xa4, 0x7e, 0x81, 0xed, 0x45, 0xb8, 0x67, 0x6b, 0xc0, 0xa3, 0x84, 0x28, 0xa8, 0x25, - 0x28, 0x06, 0x5d, 0x54, 0x5f, 0x84, 0x62, 0x60, 0x8c, 0x9e, 0x1b, 0xeb, 0xa2, 0xf8, 0x7c, 0x21, - 0xdc, 0xd6, 0x02, 0xe4, 0xbc, 0x43, 0x17, 0xd7, 0x3f, 0xcd, 0xc0, 0x7a, 0xe2, 0xbe, 0x01, 0xdd, - 0x84, 0x45, 0x79, 0xa2, 0x20, 0x7d, 0xda, 0x48, 0x39, 0xc0, 0xdb, 0xc2, 0x4a, 0xf3, 0xcd, 0xd9, - 0xc6, 0x86, 0x60, 0x6a, 0x99, 0x23, 0x63, 0xa0, 0x13, 0xc7, 0xf1, 0x7c, 0x71, 0xbc, 0x91, 0x12, - 0x70, 0xda, 0xb4, 0xd3, 0x96, 0x7c, 0x58, 0x8d, 0xa1, 0x26, 0x46, 0x98, 0xec, 0x71, 0x45, 0x18, - 0x74, 0x09, 0xd6, 0xd9, 0x84, 0xb2, 0x08, 0xa6, 0xba, 0xcc, 0xf6, 0xc5, 0x6c, 0xcf, 0x9d, 0x55, - 0x36, 0x0a, 0xda, 0x9a, 0xdf, 0x78, 0x3d, 0xd2, 0x56, 0x6f, 0xc1, 0x99, 0xa7, 0xed, 0xd2, 0x53, - 0x6e, 0x44, 0xeb, 0x1f, 0xaf, 0xc2, 0xa2, 0x74, 0x2b, 0x32, 0xa0, 0xe4, 0x46, 0xf2, 0x6f, 0x65, - 0x26, 0x57, 0x4a, 0x90, 0x46, 0xc7, 0x8b, 0x25, 0xdc, 0x51, 0x4c, 0xf5, 0xd3, 0x12, 0x40, 0x98, - 0xc6, 0xa0, 0x27, 0xe0, 0xef, 0xa6, 0xb0, 0xa9, 0xcb, 0xd5, 0xc9, 0x17, 0xc5, 0xad, 0x59, 0x89, - 0x03, 0x58, 0x7f, 0x22, 0x60, 0xb3, 0x25, 0x21, 0xb5, 0xaa, 0x1b, 0xaf, 0x42, 0xef, 0xc3, 0x8a, - 0xd1, 0xf5, 0xac, 0x7d, 0x1c, 0x12, 0x8b, 0xe9, 0x76, 0x73, 0x7e, 0xe2, 0x4d, 0x0e, 0x18, 0xb0, - 0x2e, 0x1b, 0x63, 0x65, 0x64, 0x01, 0x44, 0x16, 0x5c, 0x21, 0xa0, 0xf6, 0xfc, 0x6c, 0xf1, 0xb5, - 0x36, 0x02, 0x8e, 0x6e, 0x40, 0x8e, 0x05, 0x15, 0xb9, 0xaa, 0x5f, 0x9a, 0x91, 0x84, 0xcd, 0x7c, - 0x8d, 0x03, 0xa8, 0x7f, 0xc9, 0x42, 0xe1, 0x36, 0x36, 0xe8, 0x88, 0x60, 0x13, 0xfd, 0x58, 0x81, - 0x35, 0x91, 0x6e, 0x48, 0x9f, 0xe9, 0x5d, 0x67, 0x24, 0x3e, 0x19, 0xa3, 0x79, 0x7b, 0xfe, 0xb1, - 0xf8, 0x14, 0x0d, 0x1e, 0x44, 0xa4, 0xc7, 0x9a, 0x1c, 0x5c, 0x0c, 0x0e, 0x59, 0x13, 0x0d, 0xe8, - 0x23, 0x05, 0xd6, 0x65, 0x22, 0x13, 0xeb, 0x8f, 0x08, 0x03, 0xef, 0x1c, 0x43, 0x7f, 0xc4, 0xda, - 0x9f, 0xd0, 0xa1, 0x55, 0x67, 0xb2, 0x05, 0x6d, 0x40, 0xc5, 0x73, 0x3c, 0x63, 0x20, 0x96, 0x53, - 0xea, 0xfa, 0xc9, 0x97, 0xa2, 0x2d, 0xf3, 0x7a, 0xb6, 0x5e, 0xee, 0xb2, 0x5a, 0xb5, 0x05, 0xa7, - 0xa6, 0x0c, 0x35, 0x21, 0xb1, 0x58, 0x8b, 0x26, 0x16, 0xd9, 0x68, 0xa6, 0x7a, 0x1d, 0x6a, 0xd3, - 0x7a, 0x38, 0x13, 0x0e, 0x85, 0xea, 0xc4, 0xac, 0x41, 0xef, 0x41, 0x61, 0x28, 0xfd, 0x20, 0x27, - 0xe5, 0xd6, 0xd1, 0x3d, 0xaa, 0x05, 0x98, 0xea, 0x47, 0x59, 0x58, 0x1e, 0x9f, 0x32, 0xcf, 0x9a, - 0x12, 0xbd, 0x04, 0xa8, 0x47, 0x0c, 0x11, 0x13, 0x09, 0x1e, 0x1a, 0x96, 0x6d, 0xd9, 0x7d, 0xee, - 0x0e, 0x45, 0xab, 0xfa, 0x2d, 0x9a, 0xdf, 0x80, 0x7e, 0xa9, 0xc0, 0xe9, 0x71, 0x85, 0xd1, 0x88, - 0x99, 0x98, 0xc1, 0xf8, 0xb8, 0xe2, 0xc5, 0xb8, 0xd6, 0x68, 0xd0, 0x0b, 0xa1, 0xb7, 0x53, 0x4e, - 0x72, 0xab, 0xfa, 0x16, 0x9c, 0x79, 0x9a, 0xe1, 0x4c, 0x32, 0x78, 0x1d, 0x56, 0x3e, 0x3f, 0xcd, - 0x9d, 0x6e, 0xfe, 0xa7, 0x3c, 0xe4, 0x58, 0xec, 0x40, 0x3a, 0x94, 0xc4, 0x1a, 0xad, 0xdb, 0xc6, - 0xd0, 0x4f, 0x9d, 0xae, 0xcd, 0x11, 0x85, 0x64, 0xe1, 0x8e, 0x31, 0xc4, 0x1a, 0x0c, 0x83, 0x67, - 0x84, 0xa1, 0xcc, 0xa7, 0x3a, 0x26, 0xba, 0x69, 0x78, 0x86, 0x7f, 0xa0, 0xf9, 0xc6, 0x3c, 0x14, - 0x4d, 0x01, 0xb4, 0x6d, 0x78, 0xc6, 0xcd, 0x13, 0x5a, 0xa9, 0x1b, 0x16, 0x91, 0x07, 0x55, 0xd3, - 0xa2, 0x1e, 0xb1, 0xf6, 0xb8, 0xa9, 0xe0, 0x9a, 0xf1, 0x2c, 0x73, 0x8c, 0x6b, 0x3b, 0x82, 0x26, - 0x09, 0x2b, 0x66, 0xac, 0x0e, 0xe9, 0x00, 0x7d, 0x63, 0xd4, 0xc7, 0x82, 0xee, 0xb3, 0xd9, 0x4e, - 0x12, 0xc7, 0xe8, 0x6e, 0x30, 0x18, 0xc9, 0x53, 0xec, 0xfb, 0x05, 0xf5, 0x1a, 0x40, 0xe8, 0x57, - 0x74, 0x06, 0x8a, 0xec, 0x2b, 0x51, 0xd7, 0xe8, 0x62, 0xb9, 0x89, 0x0c, 0x2b, 0x10, 0x82, 0x1c, - 0xff, 0x86, 0x59, 0xde, 0xc0, 0x9f, 0xd5, 0xff, 0x67, 0x9b, 0xf0, 0xd0, 0x4b, 0x81, 0x20, 0x94, - 0x88, 0x20, 0xd4, 0xf7, 0xa0, 0x12, 0x1f, 0x2d, 0x7b, 0x93, 0xbb, 0xd7, 0x7f, 0x93, 0x17, 0x98, - 0xc4, 0xe8, 0x68, 0x28, 0xe5, 0xc4, 0x1e, 0x59, 0xcd, 0xd0, 0xb2, 0x39, 0x67, 0x56, 0x63, 0x8f, - 0xbc, 0xc6, 0x38, 0xe0, 0x29, 0x11, 0xab, 0x31, 0x0e, 0xd4, 0x77, 0xa0, 0x18, 0x0c, 0x2f, 0xb9, - 0x0b, 0xe8, 0x0a, 0x14, 0x83, 0xcb, 0xb0, 0x14, 0x9b, 0xb2, 0xf0, 0x65, 0x96, 0xc5, 0x32, 0xe7, - 0xab, 0x87, 0x50, 0x89, 0x67, 0x34, 0x09, 0x33, 0xe2, 0xee, 0xf8, 0xc6, 0xef, 0xea, 0xdc, 0x11, - 0x21, 0xba, 0x2f, 0xfc, 0x4d, 0x06, 0x9e, 0x7b, 0xea, 0x39, 0xf8, 0x31, 0x26, 0xd2, 0xcf, 0x36, - 0xc1, 0x7d, 0x17, 0x96, 0x5c, 0x62, 0x0d, 0x0d, 0x72, 0x28, 0xb3, 0x74, 0x91, 0x95, 0xcc, 0xbf, - 0x8d, 0x2d, 0x4b, 0x38, 0x9e, 0x9d, 0xd7, 0xbf, 0x9d, 0x83, 0xd3, 0x53, 0x2f, 0x8d, 0xd2, 0xde, - 0xc8, 0x3c, 0x81, 0x65, 0x13, 0x53, 0x8b, 0x60, 0x53, 0xdc, 0x19, 0xf8, 0xe3, 0xdf, 0x3d, 0xea, - 0xad, 0x55, 0x63, 0x5b, 0xc0, 0xf2, 0x3a, 0x99, 0x3b, 0x2c, 0x99, 0xd1, 0x3a, 0xf5, 0x77, 0x0a, - 0x94, 0xa3, 0x6f, 0xa1, 0x8b, 0xb0, 0x1e, 0xac, 0x52, 0x4e, 0x4f, 0xae, 0x38, 0x26, 0x16, 0xd7, - 0xa9, 0x19, 0x6d, 0xd5, 0x6f, 0xbc, 0xdb, 0xd3, 0xfc, 0x26, 0x74, 0x1e, 0xd6, 0x8c, 0xc1, 0xc0, - 0x79, 0xec, 0x0f, 0x40, 0x17, 0xd7, 0xc8, 0x7c, 0x18, 0x59, 0x0d, 0xc9, 0x36, 0x8e, 0xdf, 0xe1, - 0x2d, 0xe8, 0x0a, 0xd4, 0x30, 0xf5, 0xac, 0xa1, 0xc1, 0xf6, 0xff, 0x63, 0x69, 0x1d, 0x95, 0x73, - 0xf1, 0x64, 0xd0, 0x1e, 0xcd, 0x55, 0xa8, 0xfa, 0x91, 0x02, 0x68, 0x72, 0x58, 0x09, 0x13, 0xa3, - 0x3b, 0x3e, 0x31, 0x6e, 0x1f, 0xab, 0x33, 0xa3, 0x93, 0xe5, 0x5f, 0x59, 0x50, 0xa7, 0x5f, 0xdb, - 0x4c, 0x2a, 0x50, 0x39, 0x4e, 0x05, 0xfe, 0xd7, 0xf6, 0xa1, 0x23, 0x58, 0xee, 0x3e, 0x34, 0x6c, - 0x1b, 0x0f, 0xc6, 0x45, 0x7a, 0xe7, 0xc8, 0x17, 0x5b, 0x8d, 0xa6, 0xc0, 0x15, 0x95, 0x4b, 0xdd, - 0x48, 0x89, 0xaa, 0x3f, 0x57, 0xa0, 0x1c, 0x6d, 0x4f, 0x73, 0x30, 0x79, 0x1e, 0xd6, 0x06, 0x06, - 0xf5, 0x74, 0xdf, 0xed, 0xfe, 0x51, 0x24, 0x13, 0x42, 0x5e, 0x43, 0xac, 0xad, 0x23, 0x9a, 0xa4, - 0xaa, 0xd0, 0x65, 0x38, 0xd9, 0xb3, 0x08, 0xf5, 0xf4, 0xc0, 0x95, 0xd1, 0xe3, 0xcb, 0xbc, 0xb6, - 0xc6, 0x5b, 0x35, 0xd9, 0x28, 0xad, 0xea, 0xd7, 0x60, 0x3d, 0xf1, 0xfa, 0x36, 0xed, 0x06, 0xb8, - 0x06, 0x27, 0x93, 0xef, 0xde, 0xea, 0x9f, 0x28, 0x50, 0x08, 0xf2, 0xd2, 0x9b, 0x62, 0x3d, 0x90, - 0xba, 0xb9, 0x9c, 0xd2, 0xdf, 0x41, 0x66, 0xc7, 0xd6, 0x28, 0x4d, 0xac, 0x28, 0x26, 0xe4, 0xf8, - 0x8a, 0x95, 0x32, 0x2e, 0xc5, 0x5d, 0x9d, 0x99, 0x74, 0x35, 0x92, 0x7d, 0x13, 0xa7, 0xbc, 0xfc, - 0xb9, 0xfe, 0xb3, 0x2c, 0x94, 0xf9, 0xd9, 0x8d, 0xef, 0x8e, 0xf8, 0x5d, 0xdb, 0x24, 0x7d, 0x26, - 0x89, 0x7e, 0x07, 0x8a, 0xe2, 0x16, 0x85, 0x4d, 0xec, 0x2c, 0x9f, 0xc4, 0xe7, 0x52, 0x0e, 0x9e, - 0xd3, 0xdf, 0xc2, 0x87, 0x5a, 0x81, 0xca, 0x27, 0x74, 0x0b, 0xb2, 0x7d, 0xec, 0xcd, 0xfa, 0x6b, - 0x05, 0x07, 0xba, 0x81, 0x23, 0xbf, 0x01, 0x30, 0x14, 0x74, 0x0f, 0x16, 0x0c, 0xd7, 0xc5, 0xb6, - 0xe9, 0x27, 0x7f, 0x57, 0x67, 0xc1, 0xdb, 0xe4, 0xa6, 0x21, 0xa4, 0xc4, 0x42, 0x5f, 0x86, 0x7c, - 0x77, 0x80, 0x0d, 0xe2, 0x67, 0x79, 0x57, 0x66, 0x01, 0x6d, 0x32, 0xcb, 0x10, 0x53, 0x20, 0x45, - 0x7f, 0x1b, 0xf8, 0x24, 0x03, 0x4b, 0xf2, 0xb3, 0xc8, 0xc8, 0x14, 0xff, 0x2e, 0xc9, 0x7f, 0x06, - 0xec, 0x8c, 0x39, 0xee, 0x95, 0x99, 0x1d, 0x17, 0x5c, 0x27, 0x73, 0xcf, 0xdd, 0x8f, 0x7b, 0xee, - 0xd5, 0x79, 0x3c, 0x17, 0x60, 0xfa, 0xae, 0xd3, 0x62, 0xae, 0xbb, 0x3a, 0x87, 0xeb, 0x02, 0x50, - 0xe9, 0xbb, 0xe8, 0x75, 0xf7, 0x1f, 0x0a, 0x50, 0xf0, 0x45, 0x85, 0x3a, 0xb0, 0x20, 0x7e, 0x9e, - 0x92, 0xa9, 0xcf, 0xcb, 0x33, 0xaa, 0xb2, 0xa1, 0x71, 0x6b, 0xd6, 0x7d, 0x81, 0x83, 0x28, 0xac, - 0x0e, 0x47, 0x03, 0xb6, 0xde, 0xb9, 0xfa, 0xc4, 0x19, 0xec, 0xe6, 0xac, 0xf0, 0xb7, 0x25, 0x54, - 0xf4, 0xd0, 0xb5, 0x3a, 0x8c, 0x57, 0x22, 0x13, 0x96, 0xf7, 0x8c, 0xbe, 0x1e, 0x39, 0x66, 0xce, - 0xce, 0xf4, 0x67, 0x46, 0xc0, 0xb7, 0x65, 0xf4, 0xa3, 0x47, 0xca, 0xe5, 0xbd, 0x48, 0x99, 0x0d, - 0xcd, 0xf2, 0x30, 0x31, 0xf6, 0x06, 0x38, 0x3a, 0xb4, 0xdc, 0x7c, 0x43, 0x6b, 0x4b, 0xa8, 0xb1, - 0xa1, 0x59, 0xf1, 0x4a, 0xf4, 0x2d, 0x05, 0x6a, 0x81, 0x43, 0x1f, 0xe1, 0x43, 0x1a, 0xa5, 0xce, - 0x73, 0xea, 0xd6, 0xbc, 0x5e, 0xbd, 0x85, 0x0f, 0x69, 0x94, 0x7e, 0x7d, 0x98, 0xd4, 0xa0, 0xaa, - 0xb0, 0x20, 0x3e, 0x73, 0x34, 0x35, 0x29, 0xf3, 0xd4, 0x44, 0x25, 0x50, 0x9d, 0x18, 0x48, 0x9a, - 0x85, 0xad, 0x0e, 0x4b, 0xe1, 0x38, 0x22, 0x11, 0x39, 0x38, 0x01, 0x6f, 0x9b, 0xe8, 0x24, 0x2c, - 0x88, 0xcb, 0x76, 0x19, 0x93, 0x65, 0x49, 0xfd, 0x9e, 0x02, 0xd5, 0x09, 0x61, 0x3c, 0x63, 0x52, - 0x7f, 0xe8, 0xb9, 0x70, 0xe8, 0xfb, 0xb0, 0x9e, 0xe8, 0xc8, 0x67, 0x3d, 0xfc, 0x0f, 0xa1, 0x1c, - 0x95, 0x69, 0x4a, 0xba, 0x70, 0x6e, 0x44, 0xe8, 0x82, 0x1b, 0x93, 0x59, 0x06, 0x1e, 0xdc, 0x4d, - 0xbc, 0x09, 0x2b, 0xb1, 0x45, 0x05, 0xbd, 0x04, 0xa8, 0xeb, 0xd8, 0x9e, 0x65, 0x8f, 0x78, 0xe6, - 0x25, 0x2e, 0x9b, 0xa4, 0x5e, 0xaa, 0xd1, 0x16, 0x7e, 0x97, 0x52, 0xbf, 0x0f, 0x95, 0x78, 0x74, - 0x9d, 0x11, 0x22, 0x58, 0xb6, 0x33, 0x91, 0x65, 0x7b, 0x03, 0xd0, 0xe4, 0xea, 0x14, 0xbc, 0xa9, - 0x44, 0xde, 0x5c, 0x87, 0xd5, 0x84, 0x68, 0x5c, 0x5f, 0x85, 0xea, 0xc4, 0x4a, 0x54, 0x5f, 0x93, - 0xa8, 0x63, 0x31, 0xb6, 0xfe, 0xab, 0x1c, 0x14, 0x76, 0x1c, 0x79, 0x48, 0xf4, 0x35, 0x28, 0x50, - 0xbc, 0x8f, 0x89, 0xe5, 0x89, 0x49, 0xb2, 0x9c, 0xfa, 0xbc, 0xc1, 0x87, 0x68, 0xec, 0x4a, 0x7b, - 0x71, 0x3f, 0x1b, 0xc0, 0xcd, 0xbf, 0x09, 0x47, 0x35, 0xb6, 0xbf, 0xa5, 0xd4, 0xe8, 0xfb, 0xa7, - 0x0f, 0x7e, 0x91, 0xdf, 0x59, 0x11, 0xa3, 0x8b, 0xf9, 0xc7, 0x2d, 0x6a, 0xa2, 0x90, 0x90, 0xd3, - 0xe4, 0xd3, 0xa4, 0x54, 0x0b, 0x93, 0xb2, 0x7b, 0x1e, 0xca, 0x03, 0xa7, 0xaf, 0x0f, 0x1c, 0x79, - 0xef, 0xba, 0x28, 0x5e, 0x19, 0x38, 0xfd, 0x1d, 0x59, 0xc5, 0x54, 0xe7, 0x3d, 0x24, 0xd8, 0x30, - 0x6b, 0x05, 0xde, 0x28, 0x4b, 0xea, 0x57, 0x21, 0xb7, 0x63, 0x51, 0x0f, 0x75, 0x80, 0xbd, 0xae, - 0x63, 0xdb, 0x23, 0x16, 0xf6, 0x37, 0x1c, 0xe7, 0x66, 0x74, 0xaa, 0x06, 0x03, 0xf1, 0x64, 0x61, - 0xaa, 0x12, 0x28, 0xf8, 0x3e, 0xae, 0xf7, 0x20, 0xc7, 0xdc, 0x8c, 0x56, 0xa0, 0x74, 0xff, 0xce, - 0x6e, 0xa7, 0xd5, 0x6c, 0x5f, 0x6f, 0xb7, 0xb6, 0x2b, 0x27, 0x50, 0x11, 0xf2, 0xf7, 0xb4, 0xcd, - 0x66, 0xab, 0xa2, 0xb0, 0xc7, 0xed, 0xd6, 0xd6, 0xfd, 0x1b, 0x95, 0x0c, 0x2a, 0x40, 0xae, 0x7d, - 0xe7, 0xfa, 0xdd, 0x4a, 0x16, 0x01, 0x2c, 0xdc, 0xb9, 0x7b, 0xaf, 0xdd, 0x6c, 0x55, 0x72, 0xac, - 0xf6, 0xc1, 0xa6, 0x76, 0xa7, 0x92, 0x67, 0xaf, 0xb6, 0x34, 0xed, 0xae, 0x56, 0x59, 0x40, 0x65, - 0x28, 0x34, 0xb5, 0xf6, 0xbd, 0x76, 0x73, 0x73, 0xa7, 0xb2, 0x58, 0x2f, 0x03, 0xec, 0x38, 0xfd, - 0xa6, 0x63, 0x7b, 0xc4, 0x19, 0xd4, 0xff, 0x9c, 0xe3, 0x4a, 0x22, 0xde, 0x03, 0x87, 0x3c, 0x0a, - 0xff, 0x39, 0xfb, 0x1f, 0x28, 0x3e, 0xe6, 0x15, 0xe1, 0x24, 0x2e, 0x88, 0x8a, 0xb6, 0x89, 0xf6, - 0xa0, 0xd2, 0x15, 0xe6, 0xba, 0xff, 0x6f, 0xb3, 0x54, 0xc1, 0xdc, 0xff, 0xde, 0xac, 0x48, 0xc0, - 0x96, 0xc4, 0x63, 0x1c, 0x03, 0xa7, 0xdf, 0xb7, 0xec, 0x7e, 0xc8, 0x91, 0x3d, 0x22, 0x87, 0x04, - 0x0c, 0x38, 0x4c, 0xa8, 0x1a, 0xc4, 0xb3, 0x7a, 0x46, 0xd7, 0x0b, 0x49, 0x72, 0x47, 0x23, 0xa9, - 0xf8, 0x88, 0x01, 0x4b, 0x8f, 0xdf, 0x88, 0xed, 0x5b, 0x94, 0x09, 0x38, 0xa0, 0xc9, 0x1f, 0x8d, - 0xa6, 0x1a, 0x40, 0x06, 0x3c, 0xef, 0xc2, 0x82, 0x6b, 0x10, 0x63, 0x48, 0x6b, 0xc0, 0x85, 0x39, - 0xc3, 0x4a, 0x1c, 0xfb, 0xfa, 0x8d, 0x0e, 0xc7, 0x91, 0xbf, 0x7c, 0x09, 0x50, 0xf5, 0x2a, 0x94, - 0x22, 0xd5, 0x9f, 0x77, 0x86, 0x5c, 0x8c, 0xee, 0xe4, 0xbf, 0xc8, 0x03, 0x5b, 0x48, 0x22, 0x83, - 0x6b, 0x90, 0x17, 0x2b, 0x91, 0xbc, 0xb8, 0x7e, 0x9e, 0x85, 0x3b, 0xc7, 0x4d, 0x2f, 0xc7, 0xfa, - 0x8b, 0x4c, 0xc1, 0xa1, 0xc5, 0xd3, 0xd0, 0x2f, 0x7e, 0xac, 0xc0, 0xd2, 0x16, 0x36, 0x86, 0xd7, - 0x6d, 0x39, 0x01, 0xd0, 0xf7, 0x15, 0x58, 0xf4, 0x9f, 0xd3, 0x26, 0xcd, 0x09, 0xbf, 0x09, 0xab, - 0x57, 0xe7, 0xb1, 0x15, 0xc1, 0xfc, 0xc4, 0x86, 0x72, 0x5e, 0xb9, 0xf8, 0x01, 0x80, 0xe8, 0x19, - 0xdf, 0x4b, 0xda, 0x72, 0x4f, 0x79, 0x6e, 0xc6, 0x7d, 0xa9, 0x3a, 0xab, 0x81, 0x64, 0xff, 0x81, - 0x02, 0x25, 0x41, 0x2f, 0x16, 0xf2, 0x03, 0xc8, 0x8b, 0x87, 0x4b, 0xb3, 0x24, 0x74, 0x72, 0x44, - 0xea, 0xe5, 0xd9, 0x8c, 0xe4, 0xf2, 0x25, 0x7a, 0xf2, 0xc3, 0xe0, 0x13, 0xed, 0x88, 0xf9, 0x8a, - 0x0e, 0x60, 0xd1, 0x7f, 0xbc, 0x3c, 0xeb, 0x12, 0xc6, 0x02, 0xb7, 0x7a, 0x21, 0xbd, 0x95, 0x1f, - 0x17, 0x45, 0x5f, 0x7e, 0x9b, 0x81, 0x9a, 0xe8, 0x4b, 0xeb, 0xc0, 0xc3, 0xc4, 0x36, 0x06, 0x42, - 0x65, 0x1d, 0x47, 0x28, 0xa7, 0x14, 0xd1, 0x35, 0xba, 0x3a, 0xf7, 0x84, 0x53, 0x5f, 0x9d, 0xc7, - 0xd4, 0xf7, 0x1a, 0xfa, 0xae, 0x02, 0x10, 0xce, 0x00, 0x94, 0x7e, 0x7f, 0x1b, 0x9b, 0x66, 0xea, - 0xd5, 0x39, 0x2c, 0xfd, 0x5e, 0x6c, 0x6d, 0xc2, 0x17, 0xa6, 0x59, 0x47, 0x8d, 0xb7, 0x8a, 0xc2, - 0xa1, 0x9b, 0xae, 0xf5, 0xf6, 0x72, 0xa4, 0x49, 0xdf, 0xbf, 0xb0, 0xb7, 0xc0, 0x93, 0x87, 0x4b, - 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0x05, 0x22, 0x35, 0x90, 0x84, 0x33, 0x00, 0x00, + // 3216 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xcd, 0x8f, 0x1b, 0xc7, + 0x95, 0x57, 0x0f, 0xc9, 0x19, 0xf2, 0x91, 0x33, 0x43, 0xd6, 0xcc, 0x48, 0x54, 0xaf, 0xbc, 0x2b, + 0x73, 0xd7, 0xc0, 0xc0, 0x0b, 0x53, 0x9f, 0xb0, 0x25, 0xaf, 0x2d, 0x7b, 0x86, 0x43, 0x49, 0xb4, + 0x46, 0x12, 0xb7, 0x47, 0x5a, 0xed, 0xda, 0x6b, 0x37, 0x7a, 0xd8, 0x45, 0xaa, 0xa1, 0x66, 0x77, + 0xbb, 0xaa, 0x39, 0xd2, 0x28, 0x46, 0x9c, 0x2f, 0x24, 0x48, 0x90, 0xc4, 0x97, 0x1c, 0x9c, 0xdc, + 0x92, 0x00, 0x01, 0x72, 0xc9, 0x1f, 0xe0, 0x6b, 0x0e, 0x41, 0x4e, 0xf9, 0x07, 0x7c, 0x09, 0x90, + 0x00, 0x49, 0x9c, 0x73, 0x80, 0xdc, 0x82, 0xfa, 0xe8, 0x0f, 0x36, 0x9b, 0x72, 0x93, 0x33, 0xca, + 0xad, 0xab, 0xaa, 0xdf, 0xef, 0x57, 0xf5, 0xfa, 0x57, 0xaf, 0x5e, 0x55, 0x35, 0xd4, 0xf6, 0xb1, + 0x31, 0xd4, 0xfb, 0x8e, 0x6e, 0x78, 0x56, 0xd3, 0x23, 0xae, 0xef, 0xa2, 0x97, 0x5c, 0x32, 0x68, + 0x1a, 0x9e, 0xd1, 0x7b, 0x88, 0x9b, 0xac, 0xb5, 0x39, 0x74, 0x4d, 0x6c, 0x37, 0xfb, 0x8e, 0x8e, + 0x9f, 0xe0, 0xde, 0xc8, 0xb7, 0x5c, 0xa7, 0x79, 0x70, 0x41, 0xdd, 0xe0, 0x96, 0x64, 0xe4, 0x38, + 0x98, 0x44, 0xd6, 0xea, 0x2a, 0x76, 0x4c, 0xcf, 0xb5, 0x1c, 0x9f, 0xca, 0x8a, 0xb3, 0x03, 0xd7, + 0x1d, 0xd8, 0xf8, 0x1c, 0x2f, 0xed, 0x8f, 0xfa, 0xe7, 0x4c, 0x4c, 0x7b, 0xc4, 0xf2, 0x7c, 0x97, + 0xc8, 0x37, 0xfe, 0x2d, 0xf9, 0x86, 0x6f, 0x0d, 0x31, 0xf5, 0x8d, 0xa1, 0x27, 0x5f, 0xf8, 0xd7, + 0x09, 0x88, 0x11, 0x31, 0x78, 0x3f, 0xa6, 0xb4, 0x3f, 0x26, 0x86, 0xe7, 0x61, 0x12, 0x74, 0x61, + 0x79, 0x88, 0x7d, 0x62, 0xf5, 0x64, 0xb1, 0xf1, 0x13, 0x05, 0x56, 0x34, 0x3c, 0x74, 0x7d, 0x7c, + 0x83, 0x78, 0xbd, 0xae, 0x4b, 0x7c, 0x34, 0x84, 0x93, 0x86, 0x67, 0xe9, 0x14, 0x93, 0x03, 0xab, + 0x87, 0xf5, 0xa8, 0x8b, 0x75, 0xe5, 0xac, 0xb2, 0x59, 0xbe, 0xf8, 0x5a, 0x33, 0xdd, 0x29, 0x9e, + 0xe5, 0x61, 0xdb, 0x72, 0x70, 0xf3, 0xe0, 0x42, 0x73, 0xcb, 0xb3, 0xf6, 0x84, 0xfd, 0x4e, 0x68, + 0xae, 0xad, 0x1b, 0x29, 0xb5, 0xe8, 0x34, 0x14, 0x7b, 0xae, 0x89, 0x89, 0x6e, 0x99, 0xf5, 0x85, + 0xb3, 0xca, 0x66, 0x49, 0x5b, 0xe2, 0xe5, 0x8e, 0xd9, 0xf8, 0x63, 0x1e, 0x50, 0xc7, 0xa1, 0x3e, + 0x19, 0xf5, 0xd8, 0x08, 0x35, 0xfc, 0xe1, 0x08, 0x53, 0x1f, 0xbd, 0x04, 0x2b, 0x56, 0x54, 0xcb, + 0xec, 0x14, 0x6e, 0xb7, 0x1c, 0xab, 0xed, 0x98, 0xe8, 0x3e, 0x14, 0x09, 0x1e, 0x58, 0xd4, 0xc7, + 0xa4, 0xfe, 0xa7, 0x25, 0xde, 0xf5, 0x57, 0x9b, 0x99, 0xbe, 0x67, 0x53, 0x93, 0x76, 0x92, 0xf1, + 0xe6, 0x09, 0x2d, 0x84, 0x42, 0x18, 0x56, 0x3c, 0xe2, 0xf6, 0x30, 0xa5, 0xfa, 0xfe, 0xc8, 0x31, + 0x6d, 0x5c, 0xff, 0xb3, 0x00, 0xff, 0xaf, 0x8c, 0xe0, 0x5d, 0x61, 0xbd, 0xcd, 0x8d, 0x23, 0x86, + 0x65, 0x2f, 0x5e, 0x8f, 0xbe, 0x0a, 0xa7, 0xc6, 0x69, 0x74, 0x8f, 0xb8, 0x03, 0x82, 0x29, 0xad, + 0xff, 0x45, 0xf0, 0xb5, 0xe6, 0xe1, 0xeb, 0x4a, 0x90, 0x88, 0x77, 0xc3, 0x4b, 0x6b, 0x47, 0x23, + 0x58, 0x4f, 0xf0, 0x53, 0xcf, 0xb6, 0xfc, 0xfa, 0x17, 0x82, 0xfc, 0xed, 0x79, 0xc8, 0xf7, 0x18, + 0x42, 0xc4, 0x8c, 0xbc, 0x89, 0x46, 0xf4, 0x10, 0x56, 0xfb, 0x96, 0x63, 0xd8, 0xd6, 0x53, 0x1c, + 0xb8, 0xf7, 0xaf, 0x82, 0xf1, 0x8d, 0x8c, 0x8c, 0xd7, 0xa5, 0x79, 0xd2, 0xbf, 0x2b, 0xfd, 0xb1, + 0x86, 0xed, 0x12, 0x2c, 0x11, 0xd1, 0xd8, 0xf8, 0x46, 0x01, 0xd6, 0xc6, 0x74, 0x46, 0x3d, 0xd7, + 0xa1, 0x38, 0xab, 0xd0, 0xd6, 0xa1, 0x80, 0x09, 0x71, 0x89, 0x94, 0xaf, 0x28, 0xa0, 0xff, 0x99, + 0x94, 0xdf, 0x6b, 0x33, 0xcb, 0x4f, 0x74, 0x64, 0x4c, 0x7f, 0xfd, 0x69, 0xfa, 0x7b, 0x63, 0x3e, + 0xfd, 0x85, 0x14, 0x09, 0x01, 0x7e, 0xfc, 0xa5, 0x02, 0xdc, 0x39, 0x9a, 0x00, 0x43, 0xe2, 0x29, + 0x0a, 0x3c, 0x78, 0xb6, 0x02, 0xb7, 0x8e, 0xa0, 0xc0, 0x90, 0x3a, 0x4d, 0x82, 0xd6, 0x54, 0x09, + 0xbe, 0x39, 0xa7, 0x04, 0x43, 0xba, 0xa4, 0x06, 0x81, 0x69, 0x44, 0xb4, 0x36, 0x7e, 0xa0, 0xc0, + 0x6a, 0x22, 0xee, 0xa0, 0xa7, 0x70, 0x3a, 0xe1, 0x82, 0xb1, 0x68, 0x9c, 0xdb, 0x2c, 0x5f, 0xbc, + 0x36, 0x8f, 0x1b, 0x62, 0x41, 0xf9, 0x94, 0x97, 0xde, 0xd0, 0x40, 0x50, 0x4d, 0xea, 0xb0, 0xf1, + 0x73, 0x80, 0x53, 0x53, 0x80, 0xd0, 0x0a, 0x2c, 0x84, 0x13, 0x64, 0xc1, 0x32, 0x91, 0x03, 0xe0, + 0x13, 0xc3, 0xa1, 0x7d, 0x97, 0x0c, 0x69, 0x7d, 0x81, 0x77, 0xf6, 0xce, 0xd1, 0x3a, 0xdb, 0xbc, + 0x17, 0x02, 0xb6, 0x1d, 0x9f, 0x1c, 0x6a, 0x31, 0x06, 0xe4, 0x43, 0xc5, 0xeb, 0xb9, 0xb6, 0x8d, + 0xf9, 0xb4, 0xa4, 0xf5, 0x1c, 0x67, 0xec, 0x1e, 0x91, 0xb1, 0x1b, 0x83, 0x14, 0x9c, 0x63, 0x2c, + 0xe8, 0x7b, 0x0a, 0xac, 0x3f, 0xb6, 0x1c, 0xd3, 0x7d, 0x6c, 0x39, 0x03, 0x9d, 0xfa, 0xc4, 0xf0, + 0xf1, 0xc0, 0xc2, 0xb4, 0x9e, 0xe7, 0xf4, 0x0f, 0x8e, 0x48, 0xff, 0x20, 0x80, 0xde, 0x0b, 0x91, + 0x45, 0x2f, 0xd6, 0x1e, 0x4f, 0xb6, 0xa0, 0x7d, 0x58, 0xe4, 0x4b, 0x27, 0xad, 0x17, 0x38, 0xfb, + 0x3b, 0x47, 0x64, 0x6f, 0x71, 0x30, 0x41, 0x28, 0x91, 0x99, 0x9b, 0xb1, 0x73, 0x60, 0x11, 0xd7, + 0x19, 0x62, 0xc7, 0xa7, 0xf5, 0xc5, 0x63, 0x71, 0x73, 0x3b, 0x06, 0x29, 0xdd, 0x1c, 0x67, 0x41, + 0x4f, 0xe0, 0x0c, 0xf5, 0x0d, 0x1f, 0xeb, 0x53, 0x32, 0x93, 0xa5, 0xa3, 0x65, 0x26, 0xa7, 0x39, + 0x78, 0x5a, 0x93, 0x6a, 0xc3, 0x6a, 0x42, 0x75, 0xa8, 0x0a, 0xb9, 0x47, 0xf8, 0x50, 0x4a, 0x9d, + 0x3d, 0xa2, 0x16, 0x14, 0x0e, 0x0c, 0x7b, 0x84, 0xf9, 0x0a, 0x50, 0xbe, 0xf8, 0x4a, 0x86, 0x7e, + 0x74, 0x43, 0x54, 0x4d, 0xd8, 0xbe, 0xbe, 0x70, 0x45, 0x51, 0x5d, 0xa8, 0x4d, 0x28, 0x2e, 0x85, + 0x6f, 0x67, 0x9c, 0xaf, 0x99, 0x85, 0xaf, 0x15, 0xc2, 0xc6, 0x09, 0x3f, 0x82, 0xfa, 0x34, 0x8d, + 0xa5, 0xf0, 0xbe, 0x33, 0xce, 0x7b, 0x39, 0x03, 0x6f, 0x12, 0xfd, 0x30, 0xce, 0xde, 0x83, 0x72, + 0x4c, 0x63, 0x29, 0x84, 0xd7, 0xc6, 0x09, 0x37, 0x33, 0x10, 0x72, 0xc0, 0x84, 0x4f, 0x27, 0xe4, + 0x75, 0x3c, 0x3e, 0x8d, 0xc1, 0xc6, 0x08, 0x1b, 0x7f, 0xcf, 0x41, 0x4d, 0x28, 0x7c, 0xcb, 0xf3, + 0x6c, 0xab, 0xc7, 0xd3, 0x73, 0xf4, 0x22, 0x54, 0xc2, 0x68, 0x15, 0xa5, 0x12, 0xe5, 0xb0, 0xae, + 0x63, 0xb2, 0x54, 0xd8, 0x72, 0xbc, 0x91, 0x1f, 0x4b, 0x85, 0x79, 0xb9, 0x63, 0xa2, 0x3a, 0x2c, + 0x61, 0x1b, 0x33, 0xa6, 0x7a, 0xee, 0xac, 0xb2, 0x59, 0xd1, 0x82, 0x22, 0xfa, 0x0a, 0xd4, 0xdc, + 0x91, 0xcf, 0xac, 0x1e, 0x1b, 0x3e, 0x26, 0x43, 0x83, 0x3c, 0x0a, 0xa2, 0x4f, 0xd6, 0x70, 0x3b, + 0xd1, 0xd9, 0xe6, 0x5d, 0x8e, 0xf8, 0x20, 0x04, 0x14, 0x73, 0xb2, 0xea, 0x26, 0xaa, 0x51, 0x17, + 0xc0, 0xa2, 0xfa, 0xbe, 0x3b, 0x72, 0x4c, 0x6c, 0xd6, 0x0b, 0x67, 0x95, 0xcd, 0x95, 0x8b, 0x17, + 0x32, 0x78, 0xae, 0x43, 0xb7, 0x85, 0x4d, 0xb3, 0xed, 0x8c, 0x86, 0x5a, 0xc9, 0x0a, 0xca, 0xe8, + 0xff, 0xa1, 0x3a, 0x74, 0x1d, 0xcb, 0x77, 0x09, 0x0b, 0xa8, 0x96, 0xd3, 0x77, 0x83, 0x18, 0x93, + 0x05, 0xf7, 0x76, 0x68, 0xda, 0x71, 0xfa, 0xae, 0xb6, 0x3a, 0x1c, 0x2b, 0x53, 0x55, 0x87, 0x8d, + 0xd4, 0xa1, 0xa5, 0xe8, 0xe1, 0xfc, 0xb8, 0x1e, 0xd4, 0xa6, 0xd8, 0x58, 0x35, 0x83, 0x8d, 0x55, + 0xf3, 0x5e, 0xb0, 0x33, 0x8b, 0x7f, 0xfb, 0x5f, 0x2b, 0x50, 0xdf, 0xc1, 0xb6, 0x71, 0x88, 0xcd, + 0x49, 0x09, 0xbc, 0x0b, 0x65, 0x23, 0x2a, 0xca, 0xed, 0xd4, 0x95, 0x79, 0x3f, 0x92, 0x16, 0x07, + 0x43, 0xb7, 0x60, 0x5d, 0xa6, 0xb3, 0xd8, 0xd4, 0xd9, 0xa6, 0x51, 0x37, 0x59, 0x37, 0x64, 0xef, + 0x4f, 0x4f, 0xf4, 0x7e, 0x47, 0x6e, 0x1b, 0x35, 0x14, 0x9a, 0xb1, 0x01, 0xf1, 0xbe, 0x37, 0x7e, + 0x98, 0x87, 0xf5, 0xb4, 0x6d, 0x0a, 0x7a, 0x0b, 0xce, 0x4c, 0x4d, 0x48, 0x22, 0x51, 0x9f, 0x9e, + 0x92, 0x53, 0x74, 0x4c, 0x64, 0x41, 0xa5, 0xc7, 0x46, 0xaa, 0xfb, 0xee, 0x23, 0xec, 0x04, 0x79, + 0xc1, 0xf5, 0x23, 0x6c, 0x9d, 0x9a, 0x2d, 0x66, 0x75, 0x8f, 0xc1, 0x69, 0xe5, 0x5e, 0xf8, 0x4c, + 0xd5, 0xdf, 0x2e, 0x00, 0x44, 0x6d, 0xe8, 0x43, 0x80, 0x11, 0xc5, 0x44, 0xe7, 0xa1, 0x5e, 0xfa, + 0xbe, 0x7b, 0x3c, 0xbc, 0xcd, 0xfb, 0x14, 0x93, 0x3d, 0x86, 0x7b, 0xf3, 0x84, 0x56, 0x1a, 0x05, + 0x05, 0x46, 0x49, 0x2d, 0x13, 0xeb, 0x7c, 0x12, 0xcb, 0x2f, 0x71, 0x5c, 0x94, 0x7b, 0x96, 0x89, + 0x3b, 0x0c, 0x97, 0x51, 0xd2, 0xa0, 0xc0, 0xf6, 0x22, 0xdc, 0xb3, 0x75, 0xe0, 0x51, 0x42, 0x14, + 0xd4, 0x32, 0x94, 0xc2, 0x2e, 0xaa, 0x2f, 0x43, 0x29, 0x34, 0x46, 0x2f, 0x8c, 0x75, 0x51, 0x7c, + 0xbe, 0x08, 0x6e, 0x7b, 0x11, 0xf2, 0xfe, 0xa1, 0x87, 0x1b, 0x9f, 0x2f, 0xc0, 0x46, 0xea, 0xbe, + 0x01, 0xdd, 0x84, 0x25, 0x79, 0xa2, 0x20, 0x7d, 0xda, 0xcc, 0x38, 0xc0, 0xdb, 0xc2, 0x4a, 0x0b, + 0xcc, 0xd9, 0xc6, 0x86, 0x60, 0x6a, 0x99, 0x23, 0xc3, 0xd6, 0x89, 0xeb, 0xfa, 0x81, 0x38, 0xde, + 0xca, 0x08, 0x38, 0x6d, 0xda, 0x69, 0xcb, 0x01, 0xac, 0xc6, 0x50, 0x53, 0x23, 0x4c, 0xee, 0xb8, + 0x22, 0x0c, 0xba, 0x04, 0x1b, 0x6c, 0x42, 0x59, 0x04, 0x53, 0x5d, 0x66, 0xfb, 0x62, 0xb6, 0xe7, + 0xcf, 0x2a, 0x9b, 0x45, 0x6d, 0x3d, 0x68, 0xbc, 0x1e, 0x6b, 0x6b, 0xb4, 0xe1, 0xcc, 0xb3, 0x76, + 0xe9, 0x19, 0x37, 0xa2, 0x8d, 0x4f, 0xd7, 0x60, 0x49, 0xba, 0x15, 0x19, 0x50, 0xf6, 0x62, 0xf9, + 0xb7, 0x32, 0x93, 0x2b, 0x25, 0x48, 0xb3, 0xeb, 0x27, 0x12, 0xee, 0x38, 0xa6, 0xfa, 0x79, 0x19, + 0x20, 0x4a, 0x63, 0xd0, 0x53, 0x08, 0x76, 0x53, 0xd8, 0xd4, 0xe5, 0xea, 0x14, 0x88, 0xe2, 0xd6, + 0xac, 0xc4, 0x21, 0x6c, 0x30, 0x11, 0xb0, 0xd9, 0x96, 0x90, 0x5a, 0xcd, 0x4b, 0x56, 0xa1, 0x0f, + 0x61, 0xd5, 0xe8, 0xf9, 0xd6, 0x01, 0x8e, 0x88, 0xc5, 0x74, 0xbb, 0x39, 0x3f, 0xf1, 0x16, 0x07, + 0x0c, 0x59, 0x57, 0x8c, 0xb1, 0x32, 0xb2, 0x00, 0x62, 0x0b, 0xae, 0x10, 0x50, 0x67, 0x7e, 0xb6, + 0xe4, 0x5a, 0x1b, 0x03, 0x47, 0x37, 0x20, 0xcf, 0x82, 0x8a, 0x5c, 0xd5, 0x2f, 0xcd, 0x48, 0xc2, + 0x66, 0xbe, 0xc6, 0x01, 0xd4, 0x3f, 0xe4, 0xa0, 0x78, 0x1b, 0x1b, 0x74, 0x44, 0xb0, 0x89, 0xbe, + 0xaf, 0xc0, 0xba, 0x48, 0x37, 0xa4, 0xcf, 0xf4, 0x9e, 0x3b, 0x12, 0x9f, 0x8c, 0xd1, 0xbc, 0x3b, + 0xff, 0x58, 0x02, 0x8a, 0x26, 0x0f, 0x22, 0xd2, 0x63, 0x2d, 0x0e, 0x2e, 0x06, 0x87, 0xac, 0x89, + 0x06, 0xf4, 0x89, 0x02, 0x1b, 0x32, 0x91, 0x49, 0xf4, 0x47, 0x84, 0x81, 0xf7, 0x8e, 0xa1, 0x3f, + 0x62, 0xed, 0x4f, 0xe9, 0xd0, 0x9a, 0x3b, 0xd9, 0x82, 0x36, 0xa1, 0xea, 0xbb, 0xbe, 0x61, 0x8b, + 0xe5, 0x94, 0x7a, 0x41, 0xf2, 0xa5, 0x68, 0x2b, 0xbc, 0x9e, 0xad, 0x97, 0x7b, 0xac, 0x56, 0x6d, + 0xc3, 0xa9, 0x29, 0x43, 0x4d, 0x49, 0x2c, 0xd6, 0xe3, 0x89, 0x45, 0x2e, 0x9e, 0xa9, 0x5e, 0x87, + 0xfa, 0xb4, 0x1e, 0xce, 0x84, 0x43, 0xa1, 0x36, 0x31, 0x6b, 0xd0, 0x07, 0x50, 0x1c, 0x4a, 0x3f, + 0xc8, 0x49, 0xb9, 0x7d, 0x74, 0x8f, 0x6a, 0x21, 0xa6, 0xfa, 0x49, 0x0e, 0x56, 0xc6, 0xa7, 0xcc, + 0xf3, 0xa6, 0x44, 0xaf, 0x00, 0xea, 0x13, 0x43, 0xc4, 0x44, 0x82, 0x87, 0x86, 0xe5, 0x58, 0xce, + 0x80, 0xbb, 0x43, 0xd1, 0x6a, 0x41, 0x8b, 0x16, 0x34, 0xa0, 0x9f, 0x2a, 0x70, 0x7a, 0x5c, 0x61, + 0x34, 0x66, 0x26, 0x66, 0x30, 0x3e, 0xae, 0x78, 0x31, 0xae, 0x35, 0x1a, 0xf6, 0x42, 0xe8, 0xed, + 0x94, 0x9b, 0xde, 0xaa, 0xbe, 0x03, 0x67, 0x9e, 0x65, 0x38, 0x93, 0x0c, 0xde, 0x84, 0xd5, 0x2f, + 0x4f, 0x73, 0xa7, 0x9b, 0xff, 0xae, 0x00, 0x79, 0x16, 0x3b, 0x90, 0x0e, 0x65, 0xb1, 0x46, 0xeb, + 0x8e, 0x31, 0x0c, 0x52, 0xa7, 0x6b, 0x73, 0x44, 0x21, 0x59, 0xb8, 0x63, 0x0c, 0xb1, 0x06, 0xc3, + 0xf0, 0x19, 0x61, 0xa8, 0xf0, 0xa9, 0x8e, 0x89, 0x6e, 0x1a, 0xbe, 0x11, 0x1c, 0x68, 0xbe, 0x35, + 0x0f, 0x45, 0x4b, 0x00, 0xed, 0x18, 0xbe, 0x71, 0xf3, 0x84, 0x56, 0xee, 0x45, 0x45, 0xe4, 0x43, + 0xcd, 0xb4, 0xa8, 0x4f, 0xac, 0x7d, 0x6e, 0x2a, 0xb8, 0x66, 0x3c, 0xcb, 0x1c, 0xe3, 0xda, 0x89, + 0xa1, 0x49, 0xc2, 0xaa, 0x99, 0xa8, 0x43, 0x3a, 0xc0, 0xc0, 0x18, 0x0d, 0xb0, 0xa0, 0xfb, 0x62, + 0xb6, 0x93, 0xc4, 0x31, 0xba, 0x1b, 0x0c, 0x46, 0xf2, 0x94, 0x06, 0x41, 0x41, 0xbd, 0x06, 0x10, + 0xf9, 0x15, 0x9d, 0x81, 0x12, 0xfb, 0x4a, 0xd4, 0x33, 0x7a, 0x58, 0x6e, 0x22, 0xa3, 0x0a, 0x84, + 0x20, 0xcf, 0xbf, 0x61, 0x8e, 0x37, 0xf0, 0x67, 0xf5, 0xdf, 0xd9, 0x26, 0x3c, 0xf2, 0x52, 0x28, + 0x08, 0x25, 0x26, 0x08, 0xf5, 0x03, 0xa8, 0x26, 0x47, 0xcb, 0xde, 0xe4, 0xee, 0x0d, 0xde, 0xe4, + 0x05, 0x26, 0x31, 0x3a, 0x1a, 0x4a, 0x39, 0xb1, 0x47, 0x56, 0x33, 0xb4, 0x1c, 0xce, 0x99, 0xd3, + 0xd8, 0x23, 0xaf, 0x31, 0x9e, 0xf0, 0x94, 0x88, 0xd5, 0x18, 0x4f, 0xd4, 0xf7, 0xa0, 0x14, 0x0e, + 0x2f, 0xbd, 0x0b, 0xe8, 0x0a, 0x94, 0xc2, 0xcb, 0xb0, 0x0c, 0x9b, 0xb2, 0xe8, 0x65, 0x96, 0xc5, + 0x32, 0xe7, 0xab, 0x87, 0x50, 0x4d, 0x66, 0x34, 0x29, 0x33, 0xe2, 0xee, 0xf8, 0xc6, 0xef, 0xea, + 0xdc, 0x11, 0x21, 0xbe, 0x2f, 0xfc, 0xc5, 0x02, 0xbc, 0xf0, 0xcc, 0x73, 0xf0, 0x63, 0x4c, 0xa4, + 0x9f, 0x6f, 0x82, 0xfb, 0x3e, 0x2c, 0x7b, 0xc4, 0x1a, 0x1a, 0xe4, 0x50, 0x66, 0xe9, 0x22, 0x2b, + 0x99, 0x7f, 0x1b, 0x5b, 0x91, 0x70, 0x3c, 0x3b, 0x6f, 0x7c, 0x3d, 0x0f, 0xa7, 0xa7, 0x5e, 0x1a, + 0x65, 0xbd, 0x91, 0x79, 0x0a, 0x2b, 0x26, 0xa6, 0x16, 0xc1, 0xa6, 0xb8, 0x33, 0x08, 0xc6, 0xbf, + 0x77, 0xd4, 0x5b, 0xab, 0xe6, 0x8e, 0x80, 0xe5, 0x75, 0x32, 0x77, 0x58, 0x36, 0xe3, 0x75, 0xea, + 0xaf, 0x14, 0xa8, 0xc4, 0xdf, 0x42, 0x17, 0x61, 0x23, 0x5c, 0xa5, 0xdc, 0xbe, 0x5c, 0x71, 0x4c, + 0x2c, 0xae, 0x53, 0x15, 0x6d, 0x2d, 0x68, 0xbc, 0xdb, 0xd7, 0x82, 0x26, 0x74, 0x1e, 0xd6, 0x0d, + 0xdb, 0x76, 0x1f, 0x07, 0x03, 0xd0, 0xc5, 0x35, 0x32, 0x1f, 0x46, 0x4e, 0x43, 0xb2, 0x8d, 0xe3, + 0x77, 0x79, 0x0b, 0xba, 0x02, 0x75, 0x4c, 0x7d, 0x6b, 0x68, 0xb0, 0xfd, 0xff, 0x58, 0x5a, 0x47, + 0xe5, 0x5c, 0x3c, 0x19, 0xb6, 0xc7, 0x73, 0x15, 0xaa, 0x7e, 0xa2, 0x00, 0x9a, 0x1c, 0x56, 0xca, + 0xc4, 0xe8, 0x8d, 0x4f, 0x8c, 0xdb, 0xc7, 0xea, 0xcc, 0xf8, 0x64, 0xf9, 0x5b, 0x0e, 0xd4, 0xe9, + 0xd7, 0x36, 0x93, 0x0a, 0x54, 0x8e, 0x53, 0x81, 0xff, 0xb4, 0x7d, 0xe8, 0x08, 0x56, 0x7a, 0x0f, + 0x0d, 0xc7, 0xc1, 0xf6, 0xb8, 0x48, 0xef, 0x1c, 0xf9, 0x62, 0xab, 0xd9, 0x12, 0xb8, 0xa2, 0x72, + 0xb9, 0x17, 0x2b, 0x51, 0xf5, 0xc7, 0x0a, 0x54, 0xe2, 0xed, 0x59, 0x0e, 0x26, 0xcf, 0xc3, 0xba, + 0x6d, 0x50, 0x5f, 0x0f, 0xdc, 0x1e, 0x1c, 0x45, 0x32, 0x21, 0x14, 0x34, 0xc4, 0xda, 0xba, 0xa2, + 0x49, 0xaa, 0x0a, 0x5d, 0x86, 0x93, 0x7d, 0x8b, 0x50, 0x5f, 0x0f, 0x5d, 0x19, 0x3f, 0xbe, 0x2c, + 0x68, 0xeb, 0xbc, 0x55, 0x93, 0x8d, 0xd2, 0xaa, 0x71, 0x0d, 0x36, 0x52, 0xaf, 0x6f, 0xb3, 0x6e, + 0x80, 0xeb, 0x70, 0x32, 0xfd, 0xee, 0xad, 0xf1, 0x99, 0x02, 0xc5, 0x30, 0x2f, 0xbd, 0x29, 0xd6, + 0x03, 0xa9, 0x9b, 0xcb, 0x19, 0xfd, 0x1d, 0x66, 0x76, 0x6c, 0x8d, 0xd2, 0xc4, 0x8a, 0x62, 0x42, + 0x9e, 0xaf, 0x58, 0x19, 0xe3, 0x52, 0xd2, 0xd5, 0x0b, 0x93, 0xae, 0x46, 0xb2, 0x6f, 0xe2, 0x94, + 0x97, 0x3f, 0x37, 0x7e, 0x94, 0x83, 0x0a, 0x3f, 0xbb, 0x09, 0xdc, 0x91, 0xbc, 0x6b, 0x9b, 0xa4, + 0x5f, 0x48, 0xa3, 0xdf, 0x85, 0x92, 0xb8, 0x45, 0x61, 0x13, 0x3b, 0xc7, 0x27, 0xf1, 0xb9, 0x8c, + 0x83, 0xe7, 0xf4, 0xb7, 0xf0, 0xa1, 0x56, 0xa4, 0xf2, 0x09, 0xdd, 0x82, 0xdc, 0x00, 0xfb, 0xb3, + 0xfe, 0x5a, 0xc1, 0x81, 0x6e, 0xe0, 0xd8, 0x6f, 0x00, 0x0c, 0x05, 0xdd, 0x83, 0x45, 0xc3, 0xf3, + 0xb0, 0x63, 0x06, 0xc9, 0xdf, 0xd5, 0x59, 0xf0, 0xb6, 0xb8, 0x69, 0x04, 0x29, 0xb1, 0xd0, 0x7f, + 0x43, 0xa1, 0x67, 0x63, 0x83, 0x04, 0x59, 0xde, 0x95, 0x59, 0x40, 0x5b, 0xcc, 0x32, 0xc2, 0x14, + 0x48, 0xf1, 0xdf, 0x06, 0x3e, 0x5b, 0x80, 0x65, 0xf9, 0x59, 0x64, 0x64, 0x4a, 0x7e, 0x97, 0xf4, + 0x3f, 0x03, 0x76, 0xc7, 0x1c, 0xf7, 0xda, 0xcc, 0x8e, 0x0b, 0xaf, 0x93, 0xb9, 0xe7, 0xee, 0x27, + 0x3d, 0xf7, 0xfa, 0x3c, 0x9e, 0x0b, 0x31, 0x03, 0xd7, 0x69, 0x09, 0xd7, 0x5d, 0x9d, 0xc3, 0x75, + 0x21, 0xa8, 0xf4, 0x5d, 0xfc, 0xba, 0xfb, 0x37, 0x45, 0x28, 0x06, 0xa2, 0x42, 0x5d, 0x58, 0x14, + 0x3f, 0x4f, 0xc9, 0xd4, 0xe7, 0xd5, 0x19, 0x55, 0xd9, 0xd4, 0xb8, 0x35, 0xeb, 0xbe, 0xc0, 0x41, + 0x14, 0xd6, 0x86, 0x23, 0x9b, 0xad, 0x77, 0x9e, 0x3e, 0x71, 0x06, 0xbb, 0x35, 0x2b, 0xfc, 0x6d, + 0x09, 0x15, 0x3f, 0x74, 0xad, 0x0d, 0x93, 0x95, 0xc8, 0x84, 0x95, 0x7d, 0x63, 0xa0, 0xc7, 0x8e, + 0x99, 0x73, 0x33, 0xfd, 0x99, 0x11, 0xf2, 0x6d, 0x1b, 0x83, 0xf8, 0x91, 0x72, 0x65, 0x3f, 0x56, + 0x66, 0x43, 0xb3, 0x7c, 0x4c, 0x8c, 0x7d, 0x1b, 0xc7, 0x87, 0x96, 0x9f, 0x6f, 0x68, 0x1d, 0x09, + 0x35, 0x36, 0x34, 0x2b, 0x59, 0x89, 0xbe, 0xa6, 0x40, 0x3d, 0x74, 0xe8, 0x23, 0x7c, 0x48, 0xe3, + 0xd4, 0x05, 0x4e, 0xdd, 0x9e, 0xd7, 0xab, 0xb7, 0xf0, 0x21, 0x8d, 0xd3, 0x6f, 0x0c, 0xd3, 0x1a, + 0x54, 0x15, 0x16, 0xc5, 0x67, 0x8e, 0xa7, 0x26, 0x15, 0x9e, 0x9a, 0xa8, 0x04, 0x6a, 0x13, 0x03, + 0xc9, 0xb2, 0xb0, 0x35, 0x60, 0x39, 0x1a, 0x47, 0x2c, 0x22, 0x87, 0x27, 0xe0, 0x1d, 0x13, 0x9d, + 0x84, 0x45, 0x71, 0xd9, 0x2e, 0x63, 0xb2, 0x2c, 0xa9, 0xdf, 0x52, 0xa0, 0x36, 0x21, 0x8c, 0xe7, + 0x4c, 0x1a, 0x0c, 0x3d, 0x1f, 0x0d, 0xfd, 0x00, 0x36, 0x52, 0x1d, 0xf9, 0xbc, 0x87, 0xff, 0x31, + 0x54, 0xe2, 0x32, 0xcd, 0x48, 0x17, 0xcd, 0x8d, 0x18, 0x5d, 0x78, 0x63, 0x32, 0xcb, 0xc0, 0xc3, + 0xbb, 0x89, 0xb7, 0x61, 0x35, 0xb1, 0xa8, 0xa0, 0x57, 0x00, 0xf5, 0x5c, 0xc7, 0xb7, 0x9c, 0x11, + 0xcf, 0xbc, 0xc4, 0x65, 0x93, 0xd4, 0x4b, 0x2d, 0xde, 0xc2, 0xef, 0x52, 0x1a, 0xf7, 0xa1, 0x9a, + 0x8c, 0xae, 0x33, 0x42, 0x84, 0xcb, 0xf6, 0x42, 0x6c, 0xd9, 0xde, 0x04, 0x34, 0xb9, 0x3a, 0x85, + 0x6f, 0x2a, 0xb1, 0x37, 0x37, 0x60, 0x2d, 0x25, 0x1a, 0x37, 0xd6, 0xa0, 0x36, 0xb1, 0x12, 0x35, + 0xd6, 0x25, 0xea, 0x58, 0x8c, 0x6d, 0xfc, 0x2c, 0x0f, 0xc5, 0x5d, 0x57, 0x1e, 0x12, 0xfd, 0x1f, + 0x14, 0x29, 0x3e, 0xc0, 0xc4, 0xf2, 0xc5, 0x24, 0x59, 0xc9, 0x7c, 0xde, 0x10, 0x40, 0x34, 0xf7, + 0xa4, 0xbd, 0xb8, 0x9f, 0x0d, 0xe1, 0xe6, 0xdf, 0x84, 0xa3, 0x3a, 0xdb, 0xdf, 0x52, 0x6a, 0x0c, + 0x82, 0xd3, 0x87, 0xa0, 0xc8, 0xef, 0xac, 0x88, 0xd1, 0xc3, 0xfc, 0xe3, 0x96, 0x34, 0x51, 0x48, + 0xc9, 0x69, 0x0a, 0x59, 0x52, 0xaa, 0xc5, 0x49, 0xd9, 0xbd, 0x08, 0x15, 0xdb, 0x1d, 0xe8, 0xb6, + 0x2b, 0xef, 0x5d, 0x97, 0xc4, 0x2b, 0xb6, 0x3b, 0xd8, 0x95, 0x55, 0x4c, 0x75, 0xfe, 0x43, 0x82, + 0x0d, 0xb3, 0x5e, 0xe4, 0x8d, 0xb2, 0xa4, 0xfe, 0x2f, 0xe4, 0x77, 0x2d, 0xea, 0xa3, 0x2e, 0xb0, + 0xd7, 0x75, 0xec, 0xf8, 0xc4, 0xc2, 0xc1, 0x86, 0xe3, 0xdc, 0x8c, 0x4e, 0xd5, 0xc0, 0x16, 0x4f, + 0x16, 0xa6, 0x2a, 0x81, 0x62, 0xe0, 0xe3, 0x46, 0x1f, 0xf2, 0xcc, 0xcd, 0x68, 0x15, 0xca, 0xf7, + 0xef, 0xec, 0x75, 0xdb, 0xad, 0xce, 0xf5, 0x4e, 0x7b, 0xa7, 0x7a, 0x02, 0x95, 0xa0, 0x70, 0x4f, + 0xdb, 0x6a, 0xb5, 0xab, 0x0a, 0x7b, 0xdc, 0x69, 0x6f, 0xdf, 0xbf, 0x51, 0x5d, 0x40, 0x45, 0xc8, + 0x77, 0xee, 0x5c, 0xbf, 0x5b, 0xcd, 0x21, 0x80, 0xc5, 0x3b, 0x77, 0xef, 0x75, 0x5a, 0xed, 0x6a, + 0x9e, 0xd5, 0x3e, 0xd8, 0xd2, 0xee, 0x54, 0x0b, 0xec, 0xd5, 0xb6, 0xa6, 0xdd, 0xd5, 0xaa, 0x8b, + 0xa8, 0x02, 0xc5, 0x96, 0xd6, 0xb9, 0xd7, 0x69, 0x6d, 0xed, 0x56, 0x97, 0x1a, 0x15, 0x80, 0x5d, + 0x77, 0xd0, 0x72, 0x1d, 0x9f, 0xb8, 0x76, 0xe3, 0xf7, 0x79, 0xae, 0x24, 0xe2, 0x3f, 0x70, 0xc9, + 0xa3, 0xe8, 0x9f, 0xb3, 0x7f, 0x81, 0xd2, 0x63, 0x5e, 0x11, 0x4d, 0xe2, 0xa2, 0xa8, 0xe8, 0x98, + 0x68, 0x1f, 0xaa, 0x3d, 0x61, 0xae, 0x07, 0xff, 0x36, 0x4b, 0x15, 0xcc, 0xfd, 0xef, 0xcd, 0xaa, + 0x04, 0x6c, 0x4b, 0x3c, 0xc6, 0x61, 0xbb, 0x83, 0x81, 0xe5, 0x0c, 0x22, 0x8e, 0xdc, 0x11, 0x39, + 0x24, 0x60, 0xc8, 0x61, 0x42, 0xcd, 0x20, 0xbe, 0xd5, 0x37, 0x7a, 0x7e, 0x44, 0x92, 0x3f, 0x1a, + 0x49, 0x35, 0x40, 0x0c, 0x59, 0xfa, 0xfc, 0x46, 0xec, 0xc0, 0xa2, 0x4c, 0xc0, 0x21, 0x4d, 0xe1, + 0x68, 0x34, 0xb5, 0x10, 0x32, 0xe4, 0x79, 0x1f, 0x16, 0x3d, 0x83, 0x18, 0x43, 0x5a, 0x07, 0x2e, + 0xcc, 0x19, 0x56, 0xe2, 0xc4, 0xd7, 0x6f, 0x76, 0x39, 0x8e, 0xfc, 0xe5, 0x4b, 0x80, 0xaa, 0x57, + 0xa1, 0x1c, 0xab, 0xfe, 0xb2, 0x33, 0xe4, 0x52, 0x7c, 0x27, 0xff, 0x9f, 0x3c, 0xb0, 0x45, 0x24, + 0x32, 0xb8, 0x86, 0x79, 0xb1, 0x12, 0xcb, 0x8b, 0x1b, 0xe7, 0x59, 0xb8, 0x73, 0xbd, 0xec, 0x72, + 0x6c, 0xbc, 0xcc, 0x14, 0x1c, 0x59, 0x3c, 0x0b, 0xfd, 0xe2, 0xa7, 0x0a, 0x2c, 0x6f, 0x63, 0x63, + 0x78, 0xdd, 0x91, 0x13, 0x00, 0x7d, 0x5b, 0x81, 0xa5, 0xe0, 0x39, 0x6b, 0xd2, 0x9c, 0xf2, 0x9b, + 0xb0, 0x7a, 0x75, 0x1e, 0x5b, 0x11, 0xcc, 0x4f, 0x6c, 0x2a, 0xe7, 0x95, 0x8b, 0x1f, 0x01, 0x88, + 0x9e, 0xf1, 0xbd, 0xa4, 0x23, 0xf7, 0x94, 0xe7, 0x66, 0xdc, 0x97, 0xaa, 0xb3, 0x1a, 0x48, 0xf6, + 0xef, 0x28, 0x50, 0x16, 0xf4, 0x62, 0x21, 0x7f, 0x02, 0x05, 0xf1, 0x70, 0x69, 0x96, 0x84, 0x4e, + 0x8e, 0x48, 0xbd, 0x3c, 0x9b, 0x91, 0x5c, 0xbe, 0x44, 0x4f, 0xbe, 0x1b, 0x7e, 0xa2, 0x5d, 0x31, + 0x5f, 0xd1, 0x13, 0x58, 0x0a, 0x1e, 0x2f, 0xcf, 0xba, 0x84, 0xb1, 0xc0, 0xad, 0x5e, 0xc8, 0x6e, + 0x15, 0xc4, 0x45, 0xd1, 0x97, 0x5f, 0x2e, 0x40, 0x5d, 0xf4, 0xa5, 0xfd, 0xc4, 0xc7, 0xc4, 0x31, + 0x6c, 0xa1, 0xb2, 0xae, 0x2b, 0x94, 0x53, 0x8e, 0xe9, 0x1a, 0x5d, 0x9d, 0x7b, 0xc2, 0xa9, 0xaf, + 0xcf, 0x63, 0x1a, 0x78, 0x0d, 0x7d, 0x53, 0x01, 0x88, 0x66, 0x00, 0xca, 0xbe, 0xbf, 0x4d, 0x4c, + 0x33, 0xf5, 0xea, 0x1c, 0x96, 0x41, 0x2f, 0xb6, 0xb7, 0xe0, 0x3f, 0xa6, 0x59, 0xc7, 0x8d, 0xb7, + 0x4b, 0xc2, 0xa1, 0x5b, 0x9e, 0xf5, 0xee, 0x4a, 0xac, 0x49, 0x3f, 0xb8, 0xb0, 0xbf, 0xc8, 0x93, + 0x87, 0x4b, 0xff, 0x08, 0x00, 0x00, 0xff, 0xff, 0xf1, 0x22, 0xc0, 0xa7, 0x84, 0x33, 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 79acdb8c7f97..9ffe1197b905 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 @@ const ( // 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 assign the parameterized timestamp, windows and PaneInfo 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 beam:coder:bytes:v1 + // elements coder and the window coder that this param_windowed_value 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 @@ var StandardCoders_Enum_name = map[int32]string{ 6: "LENGTH_PREFIX", 7: "GLOBAL_WINDOW", 8: "WINDOWED_VALUE", + 14: "PARAM_WINDOWED_VALUE", 9: "STATE_BACKED_ITERABLE", 13: "ROW", } @@ -566,6 +576,7 @@ var StandardCoders_Enum_value = map[string]int32{ "LENGTH_PREFIX": 6, "GLOBAL_WINDOW": 7, "WINDOWED_VALUE": 8, + "PARAM_WINDOWED_VALUE": 14, "STATE_BACKED_ITERABLE": 9, "ROW": 13, } @@ -813,7 +824,7 @@ func (x StandardEnvironments_Environments) String() string { } func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{36, 0} + return fileDescriptor_cf57597c3a9659a9, []int{37, 0} } type DisplayData_Type_Enum int32 @@ -856,7 +867,7 @@ func (x DisplayData_Type_Enum) String() string { } func (DisplayData_Type_Enum) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{41, 2, 0} + return fileDescriptor_cf57597c3a9659a9, []int{42, 2, 0} } type BeamConstants struct { @@ -1102,11 +1113,14 @@ type PTransform struct { // (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"` - 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:"-"` + 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{} } @@ -2668,7 +2682,10 @@ type WindowingStrategy struct { // (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:"-"` @@ -3947,6 +3964,62 @@ func (m *SideInput) GetWindowMappingFn() *FunctionSpec { 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 + // beam:coder:bytes:v1 elements coder and the window coder that this + // param_windowed_value 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 { @@ -3964,7 +4037,7 @@ func (m *Environment) Reset() { *m = Environment{} } 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 { @@ -4009,7 +4082,7 @@ func (m *StandardEnvironments) Reset() { *m = StandardEnvironments{} } 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 { @@ -4042,7 +4115,7 @@ func (m *DockerPayload) Reset() { *m = DockerPayload{} } 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 { @@ -4084,7 +4157,7 @@ func (m *ProcessPayload) Reset() { *m = ProcessPayload{} } 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 { @@ -4145,7 +4218,7 @@ func (m *ExternalPayload) Reset() { *m = ExternalPayload{} } 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 { @@ -4225,7 +4298,7 @@ func (m *FunctionSpec) Reset() { *m = FunctionSpec{} } func (m *FunctionSpec) String() string { return proto.CompactTextString(m) } func (*FunctionSpec) ProtoMessage() {} func (*FunctionSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{40} + return fileDescriptor_cf57597c3a9659a9, []int{41} } func (m *FunctionSpec) XXX_Unmarshal(b []byte) error { @@ -4273,7 +4346,7 @@ func (m *DisplayData) Reset() { *m = DisplayData{} } func (m *DisplayData) String() string { return proto.CompactTextString(m) } func (*DisplayData) ProtoMessage() {} func (*DisplayData) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{41} + return fileDescriptor_cf57597c3a9659a9, []int{42} } func (m *DisplayData) XXX_Unmarshal(b []byte) error { @@ -4318,7 +4391,7 @@ func (m *DisplayData_Identifier) Reset() { *m = DisplayData_Identifier{} func (m *DisplayData_Identifier) String() string { return proto.CompactTextString(m) } func (*DisplayData_Identifier) ProtoMessage() {} func (*DisplayData_Identifier) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{41, 0} + return fileDescriptor_cf57597c3a9659a9, []int{42, 0} } func (m *DisplayData_Identifier) XXX_Unmarshal(b []byte) error { @@ -4383,7 +4456,7 @@ func (m *DisplayData_Item) Reset() { *m = DisplayData_Item{} } func (m *DisplayData_Item) String() string { return proto.CompactTextString(m) } func (*DisplayData_Item) ProtoMessage() {} func (*DisplayData_Item) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{41, 1} + return fileDescriptor_cf57597c3a9659a9, []int{42, 1} } func (m *DisplayData_Item) XXX_Unmarshal(b []byte) error { @@ -4456,7 +4529,7 @@ func (m *DisplayData_Type) Reset() { *m = DisplayData_Type{} } func (m *DisplayData_Type) String() string { return proto.CompactTextString(m) } func (*DisplayData_Type) ProtoMessage() {} func (*DisplayData_Type) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{41, 2} + return fileDescriptor_cf57597c3a9659a9, []int{42, 2} } func (m *DisplayData_Type) XXX_Unmarshal(b []byte) error { @@ -4510,7 +4583,7 @@ func (m *MessageWithComponents) Reset() { *m = MessageWithComponents{} } func (m *MessageWithComponents) String() string { return proto.CompactTextString(m) } func (*MessageWithComponents) ProtoMessage() {} func (*MessageWithComponents) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{42} + return fileDescriptor_cf57597c3a9659a9, []int{43} } func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error { @@ -4703,6 +4776,8 @@ type ExecutableStagePayload struct { // 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"` @@ -4733,7 +4808,7 @@ func (m *ExecutableStagePayload) Reset() { *m = ExecutableStagePayload{} func (m *ExecutableStagePayload) String() string { return proto.CompactTextString(m) } func (*ExecutableStagePayload) ProtoMessage() {} func (*ExecutableStagePayload) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{43} + return fileDescriptor_cf57597c3a9659a9, []int{44} } func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error { @@ -4761,6 +4836,13 @@ func (m *ExecutableStagePayload) GetEnvironment() *Environment { 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 @@ -4826,7 +4908,7 @@ func (m *ExecutableStagePayload_SideInputId) Reset() { *m = ExecutableSt func (m *ExecutableStagePayload_SideInputId) String() string { return proto.CompactTextString(m) } func (*ExecutableStagePayload_SideInputId) ProtoMessage() {} func (*ExecutableStagePayload_SideInputId) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{43, 0} + return fileDescriptor_cf57597c3a9659a9, []int{44, 0} } func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error { @@ -4877,7 +4959,7 @@ func (m *ExecutableStagePayload_UserStateId) Reset() { *m = ExecutableSt func (m *ExecutableStagePayload_UserStateId) String() string { return proto.CompactTextString(m) } func (*ExecutableStagePayload_UserStateId) ProtoMessage() {} func (*ExecutableStagePayload_UserStateId) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{43, 1} + return fileDescriptor_cf57597c3a9659a9, []int{44, 1} } func (m *ExecutableStagePayload_UserStateId) XXX_Unmarshal(b []byte) error { @@ -4928,7 +5010,7 @@ func (m *ExecutableStagePayload_TimerId) Reset() { *m = ExecutableStageP func (m *ExecutableStagePayload_TimerId) String() string { return proto.CompactTextString(m) } func (*ExecutableStagePayload_TimerId) ProtoMessage() {} func (*ExecutableStagePayload_TimerId) Descriptor() ([]byte, []int) { - return fileDescriptor_cf57597c3a9659a9, []int{43, 2} + return fileDescriptor_cf57597c3a9659a9, []int{44, 2} } func (m *ExecutableStagePayload_TimerId) XXX_Unmarshal(b []byte) error { @@ -5066,6 +5148,7 @@ func init() { 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") @@ -5090,331 +5173,335 @@ func init() { func init() { proto.RegisterFile("beam_runner_api.proto", fileDescriptor_cf57597c3a9659a9) } var fileDescriptor_cf57597c3a9659a9 = []byte{ - // 5179 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xcf, 0x6f, 0x23, 0xc9, - 0x75, 0xbf, 0x48, 0x8a, 0xbf, 0x1e, 0x29, 0xaa, 0x55, 0xd2, 0xcc, 0x6a, 0xdb, 0xeb, 0x9d, 0xd9, - 0xde, 0xf5, 0xee, 0x78, 0xbf, 0x6b, 0xee, 0x48, 0xb3, 0xb3, 0x3b, 0x23, 0xdb, 0xb3, 0x26, 0xc5, - 0xd6, 0xa8, 0x67, 0xf8, 0xcb, 0x4d, 0x4a, 0x9a, 0x59, 0xdb, 0xdb, 0xdf, 0x16, 0xbb, 0x28, 0x35, - 0xa6, 0xd9, 0x4d, 0x77, 0x37, 0x35, 0x4b, 0xc3, 0x41, 0x80, 0x20, 0xf0, 0xc1, 0x87, 0x04, 0xc9, - 0x61, 0x81, 0x1c, 0x82, 0x00, 0x36, 0x10, 0x24, 0x41, 0x80, 0x04, 0x76, 0x92, 0x3f, 0xc0, 0x49, - 0x8e, 0x39, 0x04, 0x08, 0x10, 0x20, 0xff, 0x43, 0x2e, 0x01, 0x7c, 0x48, 0x4e, 0x41, 0xfd, 0xe8, - 0x66, 0x91, 0x92, 0x66, 0x49, 0xcd, 0x20, 0x37, 0xf6, 0xeb, 0x7a, 0x9f, 0x57, 0xf5, 0xaa, 0xea, - 0xd5, 0xab, 0xf7, 0x5e, 0x13, 0xae, 0x1d, 0x63, 0x73, 0x60, 0xf8, 0x23, 0xd7, 0xc5, 0xbe, 0x61, - 0x0e, 0xed, 0xf2, 0xd0, 0xf7, 0x42, 0x0f, 0xbd, 0xe5, 0xf9, 0x27, 0x65, 0x73, 0x68, 0xf6, 0x4e, - 0x71, 0x99, 0xb4, 0x28, 0x0f, 0x3c, 0x0b, 0x3b, 0xe5, 0xa1, 0x3d, 0xc4, 0x8e, 0xed, 0xe2, 0xf2, - 0xd9, 0x96, 0xbc, 0x8a, 0x5d, 0x6b, 0xe8, 0xd9, 0x6e, 0x18, 0x30, 0x1e, 0xf9, 0xf5, 0x13, 0xcf, - 0x3b, 0x71, 0xf0, 0x87, 0xf4, 0xe9, 0x78, 0xd4, 0xff, 0xd0, 0x74, 0xc7, 0xfc, 0xd5, 0xcd, 0xd9, - 0x57, 0x16, 0x0e, 0x7a, 0xbe, 0x3d, 0x0c, 0x3d, 0x9f, 0xb7, 0xb8, 0x31, 0xdb, 0x22, 0xb4, 0x07, - 0x38, 0x08, 0xcd, 0xc1, 0x90, 0x35, 0x50, 0x7e, 0x93, 0x80, 0x95, 0x2a, 0x36, 0x07, 0xbb, 0x9e, - 0x1b, 0x84, 0xa6, 0x1b, 0x06, 0xca, 0xdf, 0x26, 0x20, 0x1f, 0x3f, 0xa1, 0x2d, 0xd8, 0x68, 0x68, - 0x4d, 0xa3, 0xab, 0x35, 0xd4, 0x4e, 0xb7, 0xd2, 0x68, 0x1b, 0x0d, 0xad, 0x5e, 0xd7, 0x3a, 0xd2, - 0x92, 0xfc, 0xda, 0x5f, 0xfd, 0xdd, 0xff, 0xfc, 0x26, 0xbd, 0xf6, 0xad, 0xfb, 0xdb, 0xdb, 0x77, - 0xee, 0x7c, 0xb2, 0x7d, 0xfb, 0xce, 0xc7, 0xf7, 0xee, 0x7e, 0xf4, 0xc9, 0x27, 0x77, 0xd1, 0x6d, - 0xd8, 0x68, 0x54, 0x9e, 0x9c, 0x67, 0x49, 0xc8, 0xd7, 0x29, 0x8b, 0x74, 0x8e, 0xe3, 0x01, 0x28, - 0x0f, 0xeb, 0xad, 0x6a, 0xa5, 0x6e, 0x1c, 0x69, 0xcd, 0x5a, 0xeb, 0xc8, 0xb8, 0x90, 0x3f, 0x39, - 0xcd, 0xbf, 0x75, 0xff, 0xee, 0xed, 0x8f, 0x28, 0xbf, 0xf2, 0x0f, 0x39, 0x80, 0x5d, 0x6f, 0x30, - 0xf4, 0x5c, 0x4c, 0xfa, 0xfc, 0x23, 0x80, 0xd0, 0x37, 0xdd, 0xa0, 0xef, 0xf9, 0x83, 0x60, 0x33, - 0x71, 0x33, 0x75, 0xab, 0xb0, 0xfd, 0xdd, 0xf2, 0x57, 0xaa, 0xbe, 0x3c, 0x81, 0x28, 0x77, 0x63, - 0x7e, 0xd5, 0x0d, 0xfd, 0xb1, 0x2e, 0x00, 0xa2, 0x1e, 0x14, 0x87, 0x3d, 0xcf, 0x71, 0x70, 0x2f, - 0xb4, 0x3d, 0x37, 0xd8, 0x4c, 0x52, 0x01, 0x9f, 0x2e, 0x26, 0xa0, 0x2d, 0x20, 0x30, 0x11, 0x53, - 0xa0, 0x68, 0x0c, 0x1b, 0xcf, 0x6d, 0xd7, 0xf2, 0x9e, 0xdb, 0xee, 0x89, 0x11, 0x84, 0xbe, 0x19, - 0xe2, 0x13, 0x1b, 0x07, 0x9b, 0x29, 0x2a, 0x6c, 0x6f, 0x31, 0x61, 0x47, 0x11, 0x52, 0x27, 0x06, - 0x62, 0x32, 0xd7, 0x9f, 0x9f, 0x7f, 0x83, 0xbe, 0x0f, 0x99, 0x9e, 0x67, 0x61, 0x3f, 0xd8, 0x5c, - 0xa6, 0xc2, 0xee, 0x2f, 0x26, 0x6c, 0x97, 0xf2, 0x32, 0x7c, 0x0e, 0x44, 0x54, 0x86, 0xdd, 0x33, - 0xdb, 0xf7, 0xdc, 0x01, 0x69, 0xb3, 0x99, 0xbe, 0x8a, 0xca, 0x54, 0x01, 0x81, 0xab, 0x4c, 0x04, - 0x95, 0x1d, 0x58, 0x9d, 0x99, 0x36, 0x24, 0x41, 0xea, 0x19, 0x1e, 0x6f, 0x26, 0x6e, 0x26, 0x6e, - 0xe5, 0x75, 0xf2, 0x13, 0xed, 0x42, 0xfa, 0xcc, 0x74, 0x46, 0x78, 0x33, 0x79, 0x33, 0x71, 0xab, - 0xb0, 0xfd, 0xad, 0x39, 0xba, 0xd0, 0x8e, 0x51, 0x75, 0xc6, 0xbb, 0x93, 0xbc, 0x97, 0x90, 0x3d, - 0x58, 0x3b, 0x37, 0x87, 0x17, 0xc8, 0xab, 0x4d, 0xcb, 0x2b, 0xcf, 0x23, 0x6f, 0x37, 0x86, 0x15, - 0x05, 0xfe, 0x14, 0x36, 0x2f, 0x9b, 0xc7, 0x0b, 0xe4, 0x3e, 0x9a, 0x96, 0xfb, 0xd1, 0x1c, 0x72, - 0x67, 0xd1, 0xc7, 0xa2, 0xf4, 0x1e, 0x14, 0x84, 0x89, 0xbd, 0x40, 0xe0, 0x83, 0x69, 0x81, 0xb7, - 0xe6, 0x9a, 0x5b, 0x0b, 0xfb, 0x33, 0x3a, 0x3d, 0x37, 0xc9, 0xaf, 0x46, 0xa7, 0x02, 0xac, 0x20, - 0x50, 0xf9, 0x8f, 0x04, 0xe4, 0xda, 0xbc, 0x19, 0x6a, 0x00, 0xf4, 0xe2, 0xd5, 0x46, 0xe5, 0xcd, - 0xb7, 0x3e, 0x26, 0x4b, 0x54, 0x17, 0x00, 0xd0, 0x07, 0x80, 0x7c, 0xcf, 0x0b, 0x8d, 0xd8, 0x72, - 0x18, 0xb6, 0xc5, 0x8c, 0x45, 0x5e, 0x97, 0xc8, 0x9b, 0x78, 0x59, 0x69, 0x16, 0xd9, 0x74, 0x45, - 0xcb, 0x0e, 0x86, 0x8e, 0x39, 0x36, 0x2c, 0x33, 0x34, 0x37, 0x53, 0x73, 0x0f, 0xad, 0xc6, 0xd8, - 0x6a, 0x66, 0x68, 0xea, 0x05, 0x6b, 0xf2, 0xa0, 0xfc, 0x6a, 0x19, 0x60, 0xb2, 0x76, 0xd1, 0x0d, - 0x28, 0x8c, 0x5c, 0xfb, 0xc7, 0x23, 0x6c, 0xb8, 0xe6, 0x00, 0x6f, 0xa6, 0xa9, 0x3e, 0x81, 0x91, - 0x9a, 0xe6, 0x00, 0xa3, 0x5d, 0x58, 0x0e, 0x86, 0xb8, 0xc7, 0x47, 0xfe, 0xe1, 0x1c, 0xa2, 0xf7, - 0x46, 0x2e, 0x5d, 0xa6, 0x9d, 0x21, 0xee, 0xe9, 0x94, 0x19, 0xbd, 0x03, 0x2b, 0xc1, 0xe8, 0x58, - 0x30, 0xbf, 0x6c, 0xc0, 0xd3, 0x44, 0x62, 0x62, 0x6c, 0x77, 0x38, 0x0a, 0x23, 0x7b, 0x76, 0x7f, - 0xa1, 0x6d, 0x58, 0xd6, 0x28, 0x2f, 0x37, 0x31, 0x0c, 0x08, 0x75, 0x21, 0xeb, 0x8d, 0x42, 0x8a, - 0xc9, 0xcc, 0xd6, 0xce, 0x62, 0x98, 0x2d, 0xc6, 0xcc, 0x40, 0x23, 0xa8, 0x73, 0xd3, 0x92, 0x79, - 0xe9, 0x69, 0x41, 0xdf, 0x80, 0x92, 0x60, 0xb6, 0x0c, 0xdb, 0xda, 0xcc, 0xd2, 0xa9, 0x58, 0x11, - 0xa8, 0x9a, 0x25, 0xdf, 0x87, 0x82, 0x30, 0xcc, 0x0b, 0x76, 0xc1, 0x86, 0xb8, 0x0b, 0xf2, 0xe2, - 0x36, 0xda, 0x81, 0xa2, 0x38, 0x9a, 0x45, 0x78, 0x95, 0xbf, 0x5f, 0x81, 0xf5, 0x4e, 0x68, 0xba, - 0x96, 0xe9, 0x5b, 0x13, 0xed, 0x04, 0xca, 0x5f, 0xa6, 0x00, 0xda, 0xbe, 0x3d, 0xb0, 0x43, 0xfb, - 0x0c, 0x07, 0xe8, 0x9b, 0x90, 0x69, 0x57, 0x74, 0xa3, 0xd6, 0x92, 0x96, 0xe4, 0xaf, 0xff, 0x82, - 0x9c, 0xca, 0xaf, 0x11, 0x3d, 0xec, 0xc4, 0x73, 0xbc, 0x33, 0x34, 0x7d, 0xcb, 0xdb, 0x39, 0xdb, - 0x42, 0x1f, 0x40, 0x76, 0xaf, 0x5e, 0xe9, 0x76, 0xd5, 0xa6, 0x94, 0x90, 0x6f, 0xd0, 0xb6, 0xaf, - 0xcf, 0xb4, 0xed, 0x3b, 0x66, 0x18, 0x62, 0x97, 0xb4, 0xfe, 0x18, 0x8a, 0x0f, 0xf5, 0xd6, 0x41, - 0xdb, 0xa8, 0x3e, 0x35, 0x1e, 0xab, 0x4f, 0xa5, 0xa4, 0xfc, 0x0e, 0x65, 0x79, 0x73, 0x86, 0xe5, - 0xc4, 0xf7, 0x46, 0x43, 0xe3, 0x78, 0x6c, 0x3c, 0xc3, 0x63, 0x2e, 0x45, 0x6b, 0xb4, 0x0f, 0xea, - 0x1d, 0x55, 0x4a, 0x5d, 0x22, 0xc5, 0x1e, 0x0c, 0x47, 0x4e, 0x80, 0x49, 0xeb, 0x4f, 0xa0, 0x54, - 0xe9, 0x74, 0xb4, 0x87, 0x4d, 0xee, 0x70, 0x74, 0xa4, 0x65, 0xf9, 0x6d, 0xca, 0xf4, 0xf5, 0x19, - 0x26, 0x76, 0x40, 0x1a, 0xb6, 0x1b, 0xd2, 0xc1, 0xdc, 0x81, 0x42, 0x57, 0xed, 0x74, 0x8d, 0x4e, - 0x57, 0x57, 0x2b, 0x0d, 0x29, 0x2d, 0x2b, 0x94, 0xeb, 0x8d, 0x19, 0xae, 0x10, 0x07, 0x61, 0x10, - 0xfa, 0x84, 0x78, 0xb6, 0x85, 0x3e, 0x82, 0x42, 0xa3, 0xd2, 0x8e, 0x45, 0x65, 0x2e, 0x11, 0x35, - 0x30, 0x87, 0x06, 0x13, 0x17, 0x10, 0xae, 0x7b, 0xb0, 0xd2, 0x50, 0xf5, 0x87, 0x6a, 0xcc, 0x97, - 0x95, 0xbf, 0x41, 0xf9, 0x6e, 0xcc, 0xf2, 0x61, 0xff, 0x04, 0x0b, 0x9c, 0x4a, 0x08, 0x1b, 0x35, - 0x3c, 0xf4, 0x71, 0xcf, 0x0c, 0xb1, 0x25, 0x4c, 0xda, 0xbb, 0xb0, 0xac, 0xab, 0x95, 0x9a, 0xb4, - 0x24, 0xbf, 0x41, 0x81, 0xae, 0xcf, 0x00, 0xf9, 0xd8, 0xb4, 0x78, 0x7f, 0x77, 0x75, 0xb5, 0xd2, - 0x55, 0x8d, 0x43, 0x4d, 0x3d, 0x92, 0x12, 0x97, 0xf4, 0xb7, 0xe7, 0x63, 0x33, 0xc4, 0xc6, 0x99, - 0x8d, 0x9f, 0x13, 0xa9, 0xff, 0x95, 0xe0, 0x4e, 0x58, 0x60, 0x87, 0x38, 0x40, 0xdf, 0x81, 0xd5, - 0xdd, 0x56, 0xa3, 0xaa, 0x35, 0x55, 0xa3, 0xad, 0xea, 0x74, 0x2e, 0x97, 0xe4, 0xf7, 0x28, 0xd0, - 0x5b, 0xb3, 0x40, 0xde, 0xe0, 0xd8, 0x76, 0xb1, 0x31, 0xc4, 0x7e, 0x34, 0x9d, 0x0f, 0x40, 0x8a, - 0xb8, 0x99, 0x67, 0x58, 0x7f, 0x2a, 0x25, 0xe4, 0x5b, 0x94, 0x5d, 0xb9, 0x84, 0xfd, 0xc4, 0xf1, - 0x8e, 0x4d, 0xc7, 0xa1, 0xfc, 0xb7, 0x21, 0xaf, 0xab, 0x9d, 0xfd, 0x83, 0xbd, 0xbd, 0xba, 0x2a, - 0x25, 0xe5, 0xb7, 0x28, 0xe3, 0xd7, 0xce, 0x8d, 0x37, 0x38, 0x1d, 0xf5, 0xfb, 0x0e, 0xe6, 0x83, - 0x3e, 0xd2, 0xb5, 0xae, 0x6a, 0xec, 0x69, 0x75, 0xb5, 0x23, 0xa5, 0x2e, 0x5b, 0x0f, 0xbe, 0x1d, - 0x62, 0xa3, 0x6f, 0x3b, 0x98, 0xaa, 0xfa, 0xb7, 0x49, 0x58, 0xdb, 0x65, 0xf2, 0x05, 0x07, 0x54, - 0x07, 0x79, 0x66, 0xec, 0x46, 0x5b, 0x57, 0x39, 0x49, 0x5a, 0x92, 0xb7, 0x29, 0xf4, 0x07, 0x2f, - 0x56, 0x83, 0x41, 0x66, 0x90, 0x91, 0x48, 0xff, 0x8e, 0x41, 0x99, 0xc5, 0x64, 0xcb, 0xa3, 0xb2, - 0xbb, 0x7b, 0xd0, 0x38, 0xa8, 0x57, 0xba, 0x2d, 0x9d, 0xf8, 0xd8, 0x3b, 0x14, 0xfb, 0xa3, 0xaf, - 0xc0, 0x66, 0x6b, 0xc6, 0xec, 0xf5, 0x46, 0x83, 0x91, 0x63, 0x86, 0x9e, 0x4f, 0x97, 0xdc, 0x0f, - 0xe1, 0xc6, 0xac, 0x0c, 0xf5, 0x49, 0x57, 0xaf, 0xec, 0x76, 0x8d, 0xd6, 0x41, 0xb7, 0x7d, 0xd0, - 0x25, 0x4e, 0xf8, 0x27, 0x54, 0xc0, 0xd6, 0x57, 0x08, 0xc0, 0x5f, 0x84, 0xbe, 0xd9, 0x0b, 0x0d, - 0x6e, 0x48, 0x09, 0xfa, 0x23, 0xb8, 0x1e, 0xcf, 0x29, 0xd9, 0xe2, 0x6a, 0xcd, 0x38, 0xac, 0xd4, - 0x0f, 0xa8, 0xb2, 0xcb, 0x14, 0xf4, 0xd6, 0x65, 0x33, 0x4b, 0x36, 0x3b, 0xb6, 0x0c, 0x6a, 0xa6, - 0xa8, 0xde, 0xff, 0x60, 0x19, 0x5e, 0xef, 0x0c, 0x1d, 0x3b, 0x0c, 0xcd, 0x63, 0x07, 0xb7, 0x4d, - 0xbf, 0xe6, 0x09, 0xfa, 0xaf, 0xc3, 0xb5, 0x76, 0x45, 0xd3, 0x8d, 0x23, 0xad, 0xbb, 0x6f, 0xe8, - 0x6a, 0xa7, 0xab, 0x6b, 0xbb, 0x5d, 0xad, 0xd5, 0x94, 0x96, 0xe4, 0x2d, 0x2a, 0xe8, 0xff, 0xcd, - 0x08, 0x0a, 0xac, 0xbe, 0x31, 0x34, 0x6d, 0xdf, 0x78, 0x6e, 0x87, 0xa7, 0x86, 0x8f, 0x83, 0xd0, - 0xb7, 0xe9, 0xc9, 0x46, 0xfa, 0x5d, 0x83, 0xb5, 0x4e, 0xbb, 0xae, 0x75, 0xa7, 0x90, 0x12, 0xf2, - 0xb7, 0x28, 0xd2, 0x7b, 0x17, 0x20, 0x05, 0xa4, 0x63, 0xb3, 0x28, 0x4d, 0xb8, 0xde, 0xd6, 0x5b, - 0xbb, 0x6a, 0xa7, 0x43, 0xf4, 0xaa, 0xd6, 0x0c, 0xb5, 0xae, 0x36, 0xd4, 0x26, 0x55, 0xe9, 0xc5, - 0xeb, 0x81, 0x76, 0xca, 0xf7, 0x7a, 0x38, 0x08, 0x88, 0x4a, 0xb1, 0x65, 0x60, 0x07, 0x53, 0xc7, - 0x88, 0xe0, 0x55, 0x41, 0x8a, 0xf0, 0x62, 0xa4, 0x94, 0xfc, 0x01, 0x45, 0x7a, 0xf7, 0x05, 0x48, - 0x22, 0xc6, 0x13, 0xf8, 0x1a, 0x1b, 0x59, 0xa5, 0x59, 0x33, 0x3a, 0xda, 0x67, 0xaa, 0x38, 0x44, - 0x62, 0x13, 0x2f, 0x9e, 0xeb, 0xc9, 0x18, 0x4d, 0xd7, 0x32, 0x02, 0xfb, 0x27, 0x58, 0x1c, 0x2c, - 0x45, 0xf6, 0xe0, 0xbd, 0xa8, 0x77, 0x04, 0x77, 0x32, 0x5a, 0x2a, 0x6a, 0x4a, 0x4a, 0x5a, 0xae, - 0x52, 0x29, 0xdf, 0x79, 0x41, 0xa7, 0x89, 0x8c, 0x78, 0xf8, 0x54, 0xea, 0x8c, 0x40, 0xe5, 0xf7, - 0x12, 0x70, 0x3d, 0x3a, 0xb7, 0x3a, 0xb6, 0x85, 0xe9, 0xd9, 0xd9, 0x1d, 0x0f, 0x71, 0xa0, 0x9c, - 0xc2, 0xb2, 0xea, 0x8e, 0x06, 0xe8, 0x43, 0xc8, 0x69, 0x5d, 0x55, 0xaf, 0x54, 0xeb, 0x64, 0x0f, - 0x8a, 0x26, 0x21, 0xb0, 0x2d, 0x6c, 0x50, 0x3f, 0x62, 0xc7, 0x0e, 0xb1, 0x4f, 0x96, 0x14, 0x19, - 0xc4, 0x87, 0x90, 0x6b, 0x1c, 0xd4, 0xbb, 0x5a, 0xa3, 0xd2, 0x96, 0x12, 0x97, 0x31, 0x0c, 0x46, - 0x4e, 0x68, 0x0f, 0xcc, 0x21, 0xe9, 0xc4, 0x2f, 0x92, 0x50, 0x10, 0xbc, 0xf7, 0x59, 0x97, 0x2b, - 0x71, 0xce, 0xe5, 0x7a, 0x1d, 0x72, 0xf4, 0x86, 0x44, 0xbc, 0x00, 0x76, 0x14, 0x67, 0xe9, 0xb3, - 0x66, 0xa1, 0x36, 0x80, 0x1d, 0x18, 0xc7, 0xde, 0xc8, 0xb5, 0xb0, 0x45, 0xdd, 0xc1, 0xd2, 0xf6, - 0xd6, 0x1c, 0x7e, 0x87, 0x16, 0x54, 0x19, 0x4f, 0x99, 0x0c, 0x5a, 0xcf, 0xdb, 0xd1, 0x33, 0xda, - 0x86, 0x6b, 0xe7, 0xae, 0x94, 0x63, 0x22, 0x79, 0x99, 0x4a, 0x3e, 0x77, 0x17, 0x1c, 0x6b, 0xd6, - 0x39, 0xff, 0x27, 0xfd, 0xf2, 0x6e, 0xe9, 0x97, 0x59, 0x28, 0xd2, 0x0d, 0xdb, 0x36, 0xc7, 0x8e, - 0x67, 0x5a, 0xc4, 0x9d, 0xb7, 0x3c, 0xa3, 0xef, 0x5e, 0xd9, 0xf1, 0xb4, 0xbc, 0x3d, 0x17, 0xd5, - 0x01, 0x86, 0xa6, 0x6f, 0x0e, 0x70, 0x48, 0x6e, 0xae, 0xec, 0x4e, 0xfe, 0xc1, 0x3c, 0x2e, 0x60, - 0xc4, 0xa4, 0x0b, 0xfc, 0xe8, 0xff, 0x43, 0x61, 0x32, 0xc7, 0x91, 0x97, 0xfa, 0xe9, 0x7c, 0x70, - 0xf1, 0xc8, 0xca, 0xf1, 0x42, 0x8c, 0xa2, 0x08, 0x41, 0x4c, 0xa0, 0x12, 0x42, 0x72, 0x7e, 0x12, - 0xb7, 0x39, 0xf2, 0x59, 0x17, 0x97, 0x40, 0x20, 0x88, 0x16, 0x62, 0x09, 0x31, 0x81, 0x48, 0x08, - 0xed, 0x01, 0xf6, 0xb9, 0x84, 0xf4, 0xd5, 0x24, 0x74, 0x09, 0x84, 0x28, 0x21, 0x8c, 0x09, 0xe8, - 0x4d, 0x80, 0x20, 0x36, 0xc2, 0xd4, 0x37, 0xce, 0xe9, 0x02, 0x05, 0xdd, 0x86, 0x0d, 0x61, 0x9f, - 0x1a, 0xf1, 0x52, 0x67, 0x0e, 0x2f, 0x12, 0xde, 0xed, 0xf2, 0x55, 0x7f, 0x07, 0xae, 0xf9, 0xf8, - 0xc7, 0x23, 0xe2, 0x3e, 0x19, 0x7d, 0xdb, 0x35, 0x1d, 0xfb, 0x27, 0x26, 0x79, 0xbf, 0x99, 0xa3, - 0xe0, 0x1b, 0xd1, 0xcb, 0x3d, 0xe1, 0x9d, 0xfc, 0x0c, 0x56, 0x67, 0x34, 0x7d, 0x81, 0xcb, 0x5b, - 0x9d, 0xbe, 0x34, 0xce, 0xb3, 0x34, 0x62, 0x50, 0xd1, 0xb9, 0x26, 0xc2, 0xa6, 0x95, 0xfe, 0x8a, - 0x84, 0x45, 0xa0, 0x33, 0xc2, 0x66, 0xf4, 0xff, 0x6a, 0x84, 0xc5, 0xa0, 0xa2, 0xeb, 0xff, 0xeb, - 0x04, 0xe4, 0xe3, 0xdd, 0x80, 0x1e, 0xc1, 0x72, 0x38, 0x1e, 0x32, 0xa3, 0x55, 0xda, 0xfe, 0x78, - 0x91, 0x9d, 0x54, 0x26, 0x76, 0x97, 0x99, 0x1f, 0x8a, 0x21, 0x7f, 0x06, 0xcb, 0x84, 0xa4, 0xe8, - 0xdc, 0x12, 0xaf, 0x42, 0xe1, 0xa0, 0xd9, 0x69, 0xab, 0xbb, 0xda, 0x9e, 0xa6, 0xd6, 0xa4, 0x25, - 0x04, 0x90, 0x61, 0x5e, 0xae, 0x94, 0x40, 0x1b, 0x20, 0xb5, 0xb5, 0xb6, 0x5a, 0x27, 0x7e, 0x42, - 0xab, 0xcd, 0xce, 0x88, 0x24, 0x7a, 0x0d, 0xd6, 0x85, 0x53, 0xc3, 0x20, 0x4e, 0xc9, 0x63, 0x55, - 0x97, 0x52, 0xca, 0xbf, 0xa6, 0x20, 0x1f, 0xeb, 0x0e, 0xf9, 0x70, 0x9d, 0x78, 0xb1, 0xc6, 0xc0, - 0xb3, 0xec, 0xfe, 0xd8, 0x60, 0xde, 0x9a, 0x70, 0xab, 0xfd, 0xf6, 0x1c, 0xe3, 0xd0, 0xb1, 0x69, - 0x35, 0x28, 0xff, 0x11, 0x61, 0x8f, 0xc1, 0xf7, 0x97, 0xf4, 0x75, 0x7f, 0xe6, 0x1d, 0x91, 0x59, - 0x87, 0xdc, 0xb1, 0x79, 0xc2, 0xa4, 0x24, 0xe7, 0x36, 0x61, 0x55, 0xf3, 0x44, 0x44, 0xce, 0x1e, - 0x9b, 0x27, 0x14, 0xed, 0x73, 0x28, 0x31, 0xb7, 0x87, 0x5a, 0x69, 0x82, 0xc9, 0x42, 0x01, 0x77, - 0xe7, 0x8b, 0x44, 0x30, 0x46, 0x11, 0x79, 0x25, 0x86, 0x8b, 0x7a, 0x4b, 0x2e, 0x1a, 0x14, 0x79, - 0x79, 0xee, 0xde, 0x36, 0xcc, 0xe1, 0x54, 0x6f, 0x07, 0xe6, 0x30, 0x42, 0x0b, 0x70, 0xc8, 0xd0, - 0xd2, 0x73, 0xa3, 0x75, 0x70, 0x38, 0x85, 0x16, 0xe0, 0x90, 0xfc, 0xac, 0x66, 0x58, 0x04, 0x42, - 0xb9, 0x0b, 0x9b, 0x97, 0x4d, 0xc2, 0xd4, 0x91, 0x99, 0x98, 0x3a, 0x32, 0x95, 0x7b, 0x50, 0x14, - 0xb5, 0x8a, 0x6e, 0x81, 0x14, 0xb9, 0x0c, 0x33, 0x2c, 0x25, 0x4e, 0xe7, 0x66, 0x47, 0xf9, 0x32, - 0x01, 0xe8, 0xbc, 0xf2, 0x88, 0xfd, 0x12, 0x5c, 0xe4, 0x59, 0x10, 0x24, 0xbc, 0x8b, 0xec, 0x57, - 0x93, 0xc6, 0x90, 0xa8, 0xd3, 0xda, 0x77, 0x17, 0x58, 0x0d, 0x53, 0x07, 0x5a, 0x9e, 0x43, 0xec, - 0xb9, 0xca, 0x21, 0x14, 0x45, 0xd5, 0xa3, 0x9b, 0x50, 0x24, 0xde, 0xf5, 0x4c, 0x4f, 0xe0, 0x19, - 0x1e, 0x47, 0x3d, 0x78, 0x07, 0x4a, 0x74, 0x4b, 0x1b, 0x33, 0x8e, 0x45, 0x91, 0x52, 0x77, 0x27, - 0xaa, 0x12, 0x27, 0x61, 0x01, 0x55, 0xfd, 0x2c, 0x01, 0xf9, 0xd8, 0x7c, 0xa0, 0x0e, 0x3b, 0x63, - 0x0c, 0xcb, 0x1b, 0x98, 0xb6, 0xcb, 0x8d, 0xc5, 0xf6, 0x9c, 0x16, 0xa8, 0x46, 0x99, 0x98, 0xa1, - 0xa0, 0xc7, 0x0a, 0x23, 0x90, 0x21, 0xb0, 0x83, 0x6b, 0x76, 0x08, 0x94, 0x1a, 0x75, 0xe4, 0x7b, - 0x90, 0x8f, 0x7d, 0x1d, 0xe5, 0xce, 0x65, 0x96, 0x65, 0x05, 0xf2, 0x07, 0xcd, 0x6a, 0xeb, 0xa0, - 0x59, 0x53, 0x6b, 0x52, 0x02, 0x15, 0x20, 0x1b, 0x3d, 0x24, 0x95, 0xbf, 0x48, 0x40, 0x81, 0xac, - 0xb3, 0xc8, 0x11, 0x79, 0x08, 0x99, 0xc0, 0x1b, 0xf9, 0x3d, 0x7c, 0x55, 0x4f, 0x84, 0xb3, 0xcf, - 0xf8, 0x6e, 0xc9, 0x97, 0xf7, 0xdd, 0x14, 0x13, 0xd6, 0x58, 0x78, 0x56, 0x73, 0xc3, 0xd8, 0x71, - 0xaa, 0x43, 0x9e, 0x87, 0x27, 0xae, 0xee, 0x3c, 0xe5, 0x18, 0xc2, 0x9e, 0xab, 0xfc, 0x71, 0x02, - 0x4a, 0xfc, 0x2a, 0x1b, 0x09, 0x98, 0x5e, 0xcd, 0x89, 0x97, 0x5d, 0xcd, 0x97, 0xee, 0xa7, 0xe4, - 0x65, 0xfb, 0x49, 0xf9, 0x97, 0x2c, 0xac, 0x75, 0x71, 0x10, 0x76, 0x68, 0x30, 0x25, 0xea, 0xd7, - 0xe5, 0x36, 0x00, 0xe9, 0x90, 0xc1, 0x67, 0x34, 0x80, 0x9b, 0x9c, 0x3b, 0x0a, 0x78, 0x4e, 0x40, - 0x59, 0x25, 0x10, 0x3a, 0x47, 0x42, 0x1d, 0xc8, 0x45, 0x49, 0x39, 0x6e, 0x8c, 0x3f, 0x99, 0x03, - 0xb5, 0x32, 0xb4, 0x3b, 0xd8, 0x3f, 0xb3, 0x7b, 0xb8, 0x16, 0x67, 0xe5, 0xf4, 0x18, 0x48, 0xfe, - 0x32, 0x0d, 0x69, 0x2a, 0x06, 0x9d, 0xc1, 0xea, 0x73, 0x33, 0xc4, 0xfe, 0xc0, 0xf4, 0x9f, 0x19, - 0x54, 0x24, 0x57, 0xf5, 0xe3, 0xab, 0xf7, 0xbd, 0x5c, 0xb1, 0xce, 0x4c, 0xb7, 0x87, 0x8f, 0x22, - 0xe0, 0xfd, 0x25, 0xbd, 0x14, 0x4b, 0x61, 0x72, 0x7f, 0x96, 0x80, 0x6b, 0xfc, 0x82, 0x45, 0xce, - 0x1a, 0xba, 0x8f, 0x99, 0x78, 0x66, 0xb7, 0xda, 0x2f, 0x2f, 0xbe, 0x1d, 0xc3, 0x93, 0xfd, 0x4e, - 0x0e, 0xd0, 0xe1, 0x14, 0x85, 0x75, 0x64, 0x00, 0x2b, 0x91, 0xf1, 0x61, 0xf2, 0x99, 0x92, 0xf7, - 0x5e, 0x4a, 0xbe, 0xa5, 0xf2, 0x8b, 0xee, 0xfe, 0x92, 0x5e, 0xe4, 0xf0, 0xf4, 0x9d, 0xac, 0x81, - 0x34, 0xab, 0x1d, 0xf4, 0x36, 0xac, 0xb8, 0xf8, 0xb9, 0x11, 0x6b, 0x88, 0xce, 0x40, 0x4a, 0x2f, - 0xba, 0xf8, 0xf9, 0xa4, 0x91, 0x04, 0xa9, 0xd0, 0x3c, 0xe1, 0xab, 0x95, 0xfc, 0x94, 0xab, 0x70, - 0xed, 0xc2, 0x91, 0xa2, 0x6f, 0x82, 0x64, 0xb2, 0x17, 0x86, 0x35, 0xf2, 0x99, 0x0b, 0xcb, 0x20, - 0x57, 0x39, 0xbd, 0xc6, 0xc9, 0xf2, 0xef, 0x27, 0xa0, 0x20, 0x74, 0x17, 0xf5, 0x20, 0x17, 0xdd, - 0xd1, 0x79, 0xee, 0xf2, 0xe1, 0x95, 0x14, 0xd1, 0x8d, 0x32, 0xbd, 0x38, 0xc2, 0xd6, 0x63, 0xe0, - 0x68, 0x28, 0xa9, 0x78, 0x28, 0xd5, 0x2c, 0xa4, 0xa9, 0xf2, 0xe5, 0x1f, 0x00, 0x3a, 0xcf, 0x8a, - 0xde, 0x83, 0x55, 0xec, 0x92, 0x4d, 0x16, 0x5f, 0xc3, 0xe9, 0x78, 0x8a, 0x7a, 0x89, 0x93, 0xa3, - 0x86, 0x6f, 0x40, 0x3e, 0xce, 0x31, 0x53, 0x55, 0xa5, 0xf4, 0x09, 0x41, 0x59, 0x85, 0x15, 0x3a, - 0x09, 0x81, 0xce, 0x1c, 0x79, 0xe5, 0x3f, 0x53, 0xb0, 0x46, 0x4f, 0xf8, 0x3d, 0xdb, 0xc1, 0x41, - 0xb4, 0xc1, 0x77, 0x61, 0x39, 0xb0, 0xdd, 0x67, 0x57, 0x4f, 0x45, 0xd8, 0xee, 0x33, 0xf4, 0x04, - 0x56, 0xfb, 0x9e, 0x3f, 0x30, 0x43, 0xa3, 0xcf, 0x5f, 0x5e, 0xf5, 0x40, 0x2e, 0x31, 0x9c, 0x88, - 0x46, 0x94, 0xc1, 0xcc, 0x26, 0xb6, 0x98, 0x8b, 0x19, 0x50, 0x4d, 0xe6, 0xf4, 0x52, 0x44, 0xa6, - 0x43, 0x0a, 0xd0, 0x77, 0x40, 0xe6, 0x35, 0x00, 0x16, 0x71, 0x8c, 0x07, 0xb6, 0x8b, 0x2d, 0x23, - 0x38, 0x35, 0x7d, 0xcb, 0x76, 0x4f, 0xa8, 0xfb, 0x95, 0xd3, 0x37, 0x59, 0x8b, 0x5a, 0xdc, 0xa0, - 0xc3, 0xdf, 0x23, 0x3c, 0x7d, 0x09, 0x65, 0x17, 0xb8, 0xda, 0x3c, 0x99, 0xbc, 0x59, 0x85, 0xbe, - 0xe8, 0x26, 0xfa, 0x7f, 0x7a, 0x7d, 0x52, 0x7e, 0x0a, 0x69, 0x6a, 0xdb, 0x5f, 0x4d, 0xb6, 0xa9, - 0x0c, 0xeb, 0x71, 0xc6, 0x2d, 0x3e, 0x4e, 0xa2, 0x9c, 0xd3, 0x5a, 0xfc, 0x8a, 0x9f, 0x26, 0x81, - 0xf2, 0xa7, 0x69, 0x28, 0x45, 0x51, 0x22, 0x96, 0xce, 0x54, 0x7e, 0x9e, 0xe6, 0xae, 0xc3, 0x3b, - 0x90, 0xae, 0x3e, 0xed, 0xaa, 0x1d, 0x69, 0x49, 0x7e, 0x9d, 0x86, 0x7a, 0xd6, 0x69, 0xa8, 0x87, - 0xa2, 0xee, 0x1c, 0x8f, 0x43, 0x1a, 0x78, 0x44, 0xb7, 0xa1, 0x40, 0x6e, 0x21, 0xcd, 0x87, 0xc6, - 0x41, 0x77, 0xef, 0x9e, 0x04, 0x53, 0xb9, 0x06, 0xd6, 0x96, 0x5c, 0x6a, 0xdd, 0x13, 0x63, 0x14, - 0xf6, 0xef, 0x11, 0x8e, 0x37, 0x21, 0xf9, 0xf8, 0x50, 0x4a, 0xc8, 0xd7, 0x69, 0x43, 0x49, 0x68, - 0xf8, 0xec, 0x8c, 0xbc, 0x57, 0x60, 0xb9, 0xda, 0x6a, 0xd5, 0xa5, 0xa2, 0xbc, 0x49, 0x5b, 0x20, - 0x51, 0xac, 0xe7, 0x39, 0xa4, 0xcd, 0xbb, 0x90, 0x39, 0xac, 0xe8, 0x5a, 0xb3, 0x2b, 0x25, 0x65, - 0x99, 0xb6, 0xda, 0x10, 0x5a, 0x9d, 0x99, 0xbe, 0xed, 0x86, 0xbc, 0x5d, 0xad, 0x75, 0x50, 0xad, - 0xab, 0x52, 0xe1, 0x82, 0x76, 0x96, 0x37, 0xe2, 0x91, 0xad, 0xf7, 0x85, 0x50, 0x58, 0x6a, 0x2a, - 0x1b, 0xc0, 0x5a, 0x8a, 0x51, 0xb0, 0x77, 0x20, 0xdd, 0xd5, 0x1a, 0xaa, 0x2e, 0x2d, 0x5f, 0xa0, - 0x17, 0xea, 0x91, 0xb1, 0x6c, 0xc5, 0xaa, 0xd6, 0xec, 0xaa, 0xfa, 0x61, 0x5c, 0xc4, 0x21, 0xa5, - 0xa7, 0x42, 0xe8, 0x1c, 0xd8, 0x0d, 0xb1, 0x7f, 0x66, 0x3a, 0x3c, 0x5d, 0xc1, 0x02, 0xef, 0x2b, - 0x75, 0xb5, 0xf9, 0xb0, 0xbb, 0x6f, 0xb4, 0x75, 0x75, 0x4f, 0x7b, 0x22, 0x65, 0xa6, 0x42, 0x6d, - 0x8c, 0xcf, 0xc1, 0xee, 0x49, 0x78, 0x6a, 0x0c, 0x7d, 0xdc, 0xb7, 0xbf, 0xe0, 0x5c, 0x53, 0x25, - 0x23, 0x52, 0xf6, 0x02, 0x2e, 0x96, 0x11, 0x10, 0x64, 0x7d, 0x0c, 0x25, 0xd6, 0x3c, 0x8a, 0x3d, - 0x4b, 0xb9, 0xa9, 0x0c, 0x0e, 0x63, 0x8b, 0xf7, 0x36, 0x5b, 0xb6, 0x34, 0x04, 0x7c, 0xad, 0xd3, - 0xad, 0x74, 0x55, 0xa3, 0x4a, 0xae, 0x9d, 0x35, 0x23, 0x56, 0x5e, 0x5e, 0xfe, 0x26, 0x65, 0x7f, - 0x7b, 0x6a, 0xfe, 0xcd, 0x10, 0x1b, 0xc7, 0x66, 0xef, 0x19, 0xb6, 0x0c, 0x51, 0x93, 0x37, 0x21, - 0xa5, 0xb7, 0x8e, 0xa4, 0x15, 0xf9, 0x35, 0xca, 0xb3, 0x26, 0xf0, 0xf8, 0xb4, 0x7f, 0xca, 0x5f, - 0x67, 0x22, 0x3f, 0x4f, 0x08, 0xc3, 0xbd, 0x5a, 0x3f, 0x0f, 0x1d, 0x42, 0x91, 0x45, 0xff, 0x49, - 0x3f, 0x47, 0x01, 0x77, 0x4f, 0xef, 0xcc, 0x73, 0x09, 0x24, 0x6c, 0x1d, 0xca, 0xc5, 0x1c, 0xd4, - 0xc2, 0x60, 0x42, 0x41, 0xef, 0x46, 0x46, 0x71, 0xe2, 0xd7, 0xb1, 0xe3, 0x65, 0x85, 0x91, 0xa3, - 0x0b, 0x4a, 0x0d, 0xb2, 0xa1, 0x6f, 0x9f, 0x9c, 0x60, 0x9f, 0xdf, 0x3f, 0xdf, 0x9f, 0xe7, 0x78, - 0x63, 0x1c, 0x7a, 0xc4, 0x8a, 0x30, 0xac, 0xc5, 0xee, 0xa2, 0xed, 0xb9, 0xe4, 0xc2, 0xcf, 0x72, - 0xda, 0xa5, 0xed, 0x7b, 0xf3, 0x38, 0x67, 0x02, 0x6f, 0xc3, 0xb3, 0x78, 0xb4, 0x42, 0x32, 0x67, - 0xc8, 0xe4, 0x7e, 0xc3, 0x32, 0x18, 0xd4, 0x3d, 0xa2, 0x21, 0xae, 0xf9, 0xee, 0x37, 0x2c, 0x01, - 0x4b, 0x0e, 0x52, 0x7e, 0xbf, 0xf1, 0x62, 0x02, 0x3a, 0x06, 0xa9, 0xe7, 0x78, 0xd4, 0xe9, 0x3a, - 0xc6, 0xa7, 0xe6, 0x99, 0xed, 0xf9, 0x34, 0x24, 0x56, 0x9a, 0xcb, 0xaf, 0xdc, 0x65, 0xac, 0x55, - 0xce, 0xc9, 0xe0, 0x57, 0x7b, 0xd3, 0x54, 0xea, 0x80, 0x38, 0x0e, 0x5d, 0xc5, 0x8e, 0x19, 0x62, - 0x17, 0x07, 0x01, 0x8d, 0xa1, 0x11, 0x07, 0x84, 0xd1, 0xeb, 0x9c, 0x8c, 0x3e, 0x87, 0x52, 0xcb, - 0x25, 0x1d, 0x8b, 0x98, 0x37, 0xf3, 0x73, 0xc7, 0x7c, 0xa6, 0x19, 0x59, 0x5f, 0x66, 0xd0, 0xd0, - 0x16, 0x5c, 0x33, 0x83, 0xc0, 0x3e, 0x71, 0x03, 0x23, 0xf4, 0x0c, 0xcf, 0x8d, 0x72, 0x95, 0x9b, - 0x40, 0xcf, 0x3f, 0xc4, 0x5f, 0x76, 0xbd, 0x96, 0x8b, 0xd9, 0xe2, 0xbf, 0x20, 0x47, 0x5e, 0xb8, - 0x20, 0x47, 0xae, 0xfc, 0x10, 0x0a, 0xc2, 0x9a, 0x54, 0x1a, 0x97, 0x5d, 0x02, 0x57, 0xa1, 0xd0, - 0x6c, 0x35, 0x69, 0xbe, 0x4c, 0x6b, 0x3e, 0x94, 0x12, 0x94, 0xa0, 0xaa, 0xb5, 0x0e, 0x4b, 0xa1, - 0x49, 0x49, 0x84, 0xa0, 0x54, 0xa9, 0xeb, 0x6a, 0xa5, 0xc6, 0xb3, 0x6a, 0x35, 0x29, 0xa5, 0xfc, - 0x08, 0xa4, 0xd9, 0x65, 0xa2, 0x68, 0x97, 0x89, 0x28, 0x01, 0xd4, 0xb4, 0xce, 0x6e, 0x45, 0xaf, - 0x31, 0x09, 0x12, 0x14, 0xe3, 0xc4, 0x1c, 0xa1, 0x24, 0x49, 0x0b, 0x5d, 0xa5, 0xc9, 0x34, 0xf2, - 0x9c, 0x52, 0xbe, 0x0f, 0xab, 0x33, 0x53, 0xa9, 0x3c, 0x78, 0xc1, 0x00, 0xd4, 0x86, 0xd6, 0x35, - 0x2a, 0xf5, 0xa3, 0xca, 0xd3, 0x0e, 0x0b, 0x92, 0x51, 0x82, 0xb6, 0x67, 0x34, 0x5b, 0x4d, 0xb5, - 0xd1, 0xee, 0x3e, 0x95, 0x92, 0x4a, 0x7b, 0x76, 0x26, 0x5f, 0x88, 0xb8, 0xa7, 0xe9, 0xea, 0x14, - 0x22, 0x25, 0x4c, 0x23, 0x1e, 0x03, 0x4c, 0x56, 0xb2, 0xd2, 0xbd, 0x0c, 0x6d, 0x0d, 0x56, 0xd4, - 0x66, 0xcd, 0x68, 0xed, 0x19, 0x71, 0x18, 0x0f, 0x41, 0xa9, 0x5e, 0xa1, 0xb9, 0x72, 0xad, 0x69, - 0xb4, 0x2b, 0x4d, 0xa2, 0x65, 0xd2, 0xeb, 0x8a, 0x5e, 0xd7, 0x44, 0x6a, 0x4a, 0x71, 0x00, 0x26, - 0xd1, 0x00, 0xe5, 0xf3, 0x17, 0x68, 0x58, 0x3d, 0x54, 0x9b, 0x5d, 0x5a, 0x18, 0x28, 0x25, 0xd0, - 0x3a, 0xac, 0xf2, 0x14, 0x13, 0x39, 0x8d, 0x29, 0x31, 0x89, 0x6e, 0xc2, 0x1b, 0x9d, 0xa7, 0xcd, - 0xdd, 0x7d, 0xbd, 0xd5, 0xa4, 0x69, 0xa7, 0xd9, 0x16, 0x29, 0xe5, 0x97, 0x12, 0x64, 0xb9, 0x35, - 0x41, 0x3a, 0xe4, 0xcd, 0x7e, 0x88, 0x7d, 0xc3, 0x74, 0x1c, 0x6e, 0x58, 0xef, 0xcc, 0x6f, 0x8c, - 0xca, 0x15, 0xc2, 0x5b, 0x71, 0x9c, 0xfd, 0x25, 0x3d, 0x67, 0xf2, 0xdf, 0x02, 0xa6, 0x3b, 0xe6, - 0xce, 0xd2, 0xe2, 0x98, 0xee, 0x78, 0x82, 0xe9, 0x8e, 0xd1, 0x01, 0x00, 0xc3, 0xc4, 0x66, 0xef, - 0x94, 0xdf, 0x8e, 0x3e, 0x5a, 0x14, 0x54, 0x35, 0x7b, 0xa7, 0xfb, 0x4b, 0x3a, 0xeb, 0x1d, 0x79, - 0x40, 0x0e, 0xac, 0x73, 0x58, 0xd7, 0x32, 0xbc, 0x7e, 0xb4, 0x2d, 0x97, 0xe7, 0x8e, 0x94, 0x4e, - 0xe3, 0xbb, 0x56, 0xab, 0xcf, 0xf6, 0xef, 0xfe, 0x92, 0x2e, 0x99, 0x33, 0x34, 0x14, 0xc2, 0x35, - 0x26, 0x6d, 0xe6, 0xce, 0xc9, 0xe3, 0x86, 0x0f, 0x16, 0x95, 0x77, 0xfe, 0x6e, 0x69, 0x9e, 0x27, - 0xa3, 0x3f, 0x49, 0x80, 0xc2, 0xc4, 0x06, 0x63, 0xb7, 0x77, 0xea, 0x7b, 0x2e, 0x4d, 0x25, 0xce, - 0xf6, 0x81, 0xd5, 0xf5, 0x3c, 0x5a, 0xb4, 0x0f, 0x1d, 0x01, 0xf3, 0x5c, 0x7f, 0x6e, 0x98, 0x2f, - 0x6e, 0x82, 0x1e, 0x43, 0xc6, 0x74, 0x9e, 0x9b, 0xe3, 0x60, 0xb3, 0x48, 0xc5, 0x6f, 0x2d, 0x22, - 0x9e, 0x32, 0xee, 0x2f, 0xe9, 0x1c, 0x02, 0x35, 0x21, 0x6b, 0xe1, 0xbe, 0x39, 0x72, 0x42, 0x7a, - 0x96, 0x14, 0xe6, 0x8b, 0xc2, 0x71, 0xb4, 0x1a, 0xe3, 0xdc, 0x5f, 0xd2, 0x23, 0x10, 0xf4, 0xf9, - 0xe4, 0x52, 0xde, 0xf3, 0x46, 0x6e, 0x48, 0x4f, 0x8f, 0xf9, 0x22, 0x1f, 0x11, 0xaa, 0x1a, 0x45, - 0x0e, 0x47, 0x6e, 0x28, 0xdc, 0xc2, 0xe9, 0x33, 0xda, 0x87, 0xb4, 0x8b, 0xcf, 0x30, 0x3b, 0x6c, - 0x0a, 0xdb, 0xb7, 0x17, 0xc0, 0x6d, 0x12, 0xbe, 0xfd, 0x25, 0x9d, 0x01, 0x90, 0xdd, 0xe1, 0xf9, - 0x2c, 0x5b, 0xe4, 0x8c, 0xe9, 0xa1, 0xb2, 0xd8, 0xee, 0x68, 0xf9, 0x7b, 0x8c, 0x97, 0xec, 0x0e, - 0x2f, 0x7a, 0x20, 0xb3, 0xe3, 0xe3, 0x21, 0x36, 0x43, 0x7a, 0xf6, 0x2c, 0x36, 0x3b, 0x3a, 0x65, - 0x24, 0xb3, 0xc3, 0x20, 0xe4, 0x27, 0x90, 0x8b, 0xac, 0x05, 0xaa, 0x43, 0x81, 0x56, 0xc3, 0xd1, - 0xa6, 0xd1, 0x1d, 0x7f, 0x11, 0x27, 0x48, 0x64, 0x9f, 0x20, 0xbb, 0xe3, 0x57, 0x8c, 0xfc, 0x14, - 0xf2, 0xb1, 0xe1, 0x78, 0xc5, 0xd0, 0xbf, 0x4a, 0x80, 0x34, 0x6b, 0x34, 0x50, 0x0b, 0x56, 0xb0, - 0xe9, 0x3b, 0x63, 0xa3, 0x6f, 0x93, 0x0b, 0x54, 0x54, 0x82, 0xb9, 0x88, 0x90, 0x22, 0x05, 0xd8, - 0x63, 0xfc, 0xa8, 0x01, 0x45, 0xe2, 0xfb, 0xc4, 0x78, 0xc9, 0x85, 0xf1, 0x0a, 0x84, 0x9f, 0xc3, - 0xc9, 0xbf, 0x0b, 0xeb, 0x17, 0x18, 0x1e, 0x74, 0x0a, 0x1b, 0x71, 0x7c, 0xc3, 0x38, 0x57, 0x77, - 0x7e, 0x77, 0xce, 0x58, 0x38, 0x65, 0x9f, 0x14, 0x1a, 0xaf, 0x87, 0xe7, 0x68, 0x81, 0xfc, 0x16, - 0xdc, 0xf8, 0x0a, 0xab, 0x23, 0xe7, 0x21, 0xcb, 0xf7, 0xb2, 0x7c, 0x07, 0x8a, 0xe2, 0x06, 0x44, - 0x6f, 0xcf, 0x6e, 0x68, 0xa2, 0xde, 0xf4, 0xf4, 0xae, 0x94, 0xb3, 0x90, 0xa6, 0xbb, 0x4b, 0xce, - 0x41, 0x86, 0x99, 0x18, 0xf9, 0x8f, 0x12, 0x90, 0x8f, 0xb7, 0x08, 0x7a, 0x00, 0xcb, 0x71, 0xa4, - 0x7f, 0x31, 0x5d, 0x52, 0x3e, 0xe2, 0xfd, 0x47, 0x3b, 0x75, 0xf1, 0xe9, 0x88, 0x58, 0xe5, 0x2e, - 0x64, 0xd8, 0x16, 0x43, 0x8f, 0x00, 0x26, 0x0b, 0xeb, 0x0a, 0xbd, 0x12, 0xb8, 0xab, 0xf9, 0xf8, - 0x66, 0xa2, 0xfc, 0x63, 0x52, 0x88, 0x82, 0x4d, 0x6a, 0x68, 0x3b, 0x90, 0xb6, 0xb0, 0x63, 0x8e, - 0x17, 0xc8, 0x26, 0x9e, 0x47, 0x29, 0xd7, 0x08, 0x04, 0xb1, 0x5f, 0x14, 0x0b, 0x7d, 0x06, 0x39, - 0xd3, 0xb1, 0x4f, 0x5c, 0x23, 0xf4, 0xb8, 0x4e, 0xbe, 0x7b, 0x35, 0xdc, 0x0a, 0x41, 0xe9, 0x7a, - 0xc4, 0x8a, 0x9b, 0xec, 0xa7, 0xfc, 0x3e, 0xa4, 0xa9, 0x34, 0xf4, 0x16, 0x14, 0xa9, 0x34, 0x63, - 0x60, 0x3b, 0x8e, 0x1d, 0xf0, 0x60, 0x64, 0x81, 0xd2, 0x1a, 0x94, 0x24, 0xdf, 0x87, 0x2c, 0x47, - 0x40, 0xd7, 0x21, 0x33, 0xc4, 0xbe, 0xed, 0xb1, 0x2b, 0x5c, 0x4a, 0xe7, 0x4f, 0x84, 0xee, 0xf5, - 0xfb, 0x01, 0x0e, 0xa9, 0x93, 0x90, 0xd2, 0xf9, 0x53, 0xf5, 0x1a, 0xac, 0x5f, 0xb0, 0x07, 0x94, - 0x9f, 0x27, 0x21, 0x1f, 0x47, 0x81, 0xd0, 0x21, 0x94, 0xcc, 0x1e, 0x2d, 0xe7, 0x19, 0x9a, 0x61, - 0x88, 0xfd, 0x2b, 0xdf, 0x65, 0x57, 0x18, 0x4c, 0x9b, 0xa1, 0xa0, 0x7d, 0xc8, 0x9e, 0xd9, 0xf8, - 0xf9, 0x4b, 0x24, 0xdc, 0x32, 0x84, 0x7f, 0xcf, 0x45, 0x3f, 0x80, 0x35, 0x7e, 0x85, 0x1d, 0x98, - 0xc3, 0x21, 0x71, 0x0e, 0xfa, 0x2e, 0x77, 0xb7, 0x16, 0xc6, 0xe4, 0x97, 0xe1, 0x06, 0x03, 0xda, - 0x73, 0x95, 0x4f, 0xa1, 0x20, 0x54, 0xa1, 0x23, 0x09, 0x52, 0x23, 0xdf, 0x8d, 0x82, 0xc9, 0x23, - 0xdf, 0x45, 0x9b, 0x90, 0x1d, 0xb2, 0x70, 0x1d, 0x95, 0x59, 0xd4, 0xa3, 0xc7, 0x47, 0xcb, 0xb9, - 0x84, 0x94, 0x54, 0xfe, 0x2c, 0x01, 0x1b, 0x51, 0xf0, 0x4a, 0x2c, 0x93, 0x57, 0x7e, 0x96, 0x80, - 0xa2, 0x48, 0x40, 0xef, 0x40, 0xa6, 0xd6, 0xa2, 0xf9, 0xf1, 0xa5, 0xa9, 0xa0, 0x12, 0x76, 0xcf, - 0x76, 0x2c, 0xaf, 0xf7, 0x8c, 0x85, 0x6c, 0xde, 0x85, 0x2c, 0x77, 0x8f, 0xa5, 0xc4, 0x54, 0x68, - 0x87, 0x34, 0xe3, 0x0e, 0x12, 0x69, 0x77, 0x0b, 0x72, 0xea, 0x93, 0xae, 0xaa, 0x37, 0x2b, 0xf5, - 0x99, 0xf0, 0x13, 0x69, 0x88, 0xbf, 0x20, 0x93, 0x60, 0x3a, 0x3b, 0x67, 0x5b, 0xca, 0x3d, 0x58, - 0xa9, 0x51, 0xf8, 0x28, 0x8e, 0xfb, 0x1e, 0xac, 0xf6, 0x3c, 0x37, 0x34, 0x6d, 0x17, 0xfb, 0x86, - 0x3d, 0x30, 0x4f, 0xa2, 0x22, 0xa8, 0x52, 0x4c, 0xd6, 0x08, 0x55, 0xf9, 0xb7, 0x04, 0x94, 0xb8, - 0x29, 0x8b, 0x78, 0x4b, 0x90, 0xf4, 0x02, 0xde, 0x3c, 0xe9, 0x05, 0x08, 0xc1, 0xb2, 0xe9, 0xf7, - 0x4e, 0xb9, 0xc6, 0xe8, 0x6f, 0xa2, 0xb2, 0x9e, 0x37, 0x18, 0x98, 0x6e, 0x14, 0x6b, 0x88, 0x1e, - 0x51, 0x1d, 0x52, 0xd8, 0x3d, 0x5b, 0xa4, 0x14, 0x7c, 0x4a, 0x7a, 0x59, 0x75, 0xcf, 0x58, 0xa4, - 0x94, 0xc0, 0xc8, 0x1f, 0x43, 0x2e, 0x22, 0x2c, 0x54, 0x4d, 0xfd, 0xdf, 0x09, 0x58, 0x55, 0xb9, - 0x82, 0xa2, 0x71, 0x89, 0xd9, 0xa4, 0xc4, 0x2b, 0xca, 0x26, 0xa1, 0x43, 0xc8, 0xd0, 0xea, 0xa5, - 0x28, 0xed, 0x35, 0x8f, 0x37, 0x3d, 0xd3, 0x31, 0x56, 0xbf, 0x11, 0x55, 0xd5, 0x33, 0x34, 0xf9, - 0x3e, 0x14, 0x04, 0xf2, 0x42, 0x63, 0xdf, 0x81, 0xa2, 0xb8, 0x21, 0xa2, 0x05, 0x9f, 0x98, 0x63, - 0xc1, 0x2b, 0xbf, 0x5d, 0x86, 0x82, 0x50, 0x40, 0x86, 0x34, 0x48, 0xdb, 0x21, 0x8e, 0x0f, 0xd5, - 0x3b, 0x8b, 0xd5, 0x9f, 0x95, 0xb5, 0x10, 0x0f, 0x74, 0x86, 0x20, 0xf7, 0x01, 0x34, 0x0b, 0xbb, - 0xa1, 0xdd, 0xb7, 0xb1, 0x4f, 0xcc, 0xa2, 0xf8, 0x7d, 0x06, 0xef, 0x5d, 0x21, 0x9c, 0x7c, 0x9a, - 0x41, 0xce, 0xcd, 0x49, 0x93, 0xc9, 0x96, 0x9d, 0xf0, 0x1d, 0xf8, 0x6e, 0xa4, 0x98, 0x54, 0xac, - 0x18, 0xf9, 0xd7, 0x49, 0x58, 0x26, 0x72, 0x91, 0x06, 0x49, 0x0e, 0x3c, 0xdf, 0x77, 0x0e, 0x53, - 0x1d, 0x8f, 0x7b, 0xaa, 0x27, 0x6d, 0xb2, 0xa8, 0x59, 0x4d, 0x4e, 0x72, 0xee, 0x38, 0x97, 0x08, - 0x36, 0x53, 0x95, 0x83, 0xde, 0x8f, 0xa6, 0x8e, 0x59, 0xb8, 0x8d, 0x32, 0xfb, 0x56, 0xb0, 0x1c, - 0x7d, 0x2b, 0x58, 0xae, 0xb8, 0xd1, 0x27, 0x40, 0xe8, 0x2e, 0x14, 0x82, 0x53, 0xcf, 0x0f, 0x59, - 0x48, 0x94, 0x5f, 0x11, 0x2f, 0xe6, 0x00, 0xda, 0xf0, 0x90, 0xb2, 0x6d, 0x40, 0xda, 0x31, 0x8f, - 0xb1, 0xc3, 0xbf, 0x36, 0x61, 0x0f, 0xe8, 0x75, 0xc8, 0x39, 0xb6, 0xfb, 0xcc, 0x18, 0xf9, 0x0e, - 0xbd, 0x78, 0xe5, 0xf5, 0x2c, 0x79, 0x3e, 0xf0, 0x1d, 0xf9, 0x77, 0x78, 0xa5, 0xd0, 0xe8, 0x05, - 0x95, 0x42, 0x2c, 0xfe, 0xce, 0x92, 0xf9, 0x5a, 0xb3, 0xab, 0x3e, 0x54, 0x75, 0x29, 0x89, 0xf2, - 0x90, 0xde, 0xab, 0xb7, 0x2a, 0x5d, 0x29, 0xc5, 0x92, 0xfc, 0xad, 0xba, 0x5a, 0x69, 0x4a, 0xcb, - 0x68, 0x05, 0xf2, 0xf1, 0x97, 0x84, 0x52, 0x1a, 0x15, 0x21, 0x57, 0x3b, 0xd0, 0x2b, 0xb4, 0x86, - 0x37, 0x83, 0x4a, 0x00, 0x8f, 0x2a, 0x87, 0x15, 0x63, 0xb7, 0x5e, 0xe9, 0x74, 0xa4, 0xac, 0xf2, - 0xef, 0x59, 0xb8, 0xd6, 0xc0, 0x41, 0x60, 0x9e, 0xe0, 0x23, 0x3b, 0x3c, 0x15, 0x4a, 0x8a, 0x5f, - 0xf1, 0xc7, 0x41, 0xdf, 0x83, 0x34, 0x8d, 0x92, 0x2e, 0xfa, 0xb5, 0x14, 0xf1, 0x1a, 0x28, 0x23, - 0xfa, 0x21, 0x31, 0xad, 0xbc, 0xe6, 0x5a, 0xd8, 0x44, 0xf3, 0xdd, 0x53, 0xa6, 0xf3, 0xfc, 0xfb, - 0x4b, 0x3a, 0xaf, 0x39, 0x8a, 0x33, 0xff, 0x87, 0xb0, 0x12, 0x25, 0xcd, 0x16, 0x2d, 0x15, 0x12, - 0x37, 0x3d, 0xb9, 0xf5, 0xf5, 0x45, 0x23, 0x70, 0x04, 0xa5, 0xa1, 0xe9, 0x1b, 0x96, 0x17, 0x77, - 0x3a, 0x33, 0x37, 0xb0, 0x58, 0x96, 0x48, 0x80, 0x87, 0x62, 0x11, 0x69, 0x0b, 0x60, 0x18, 0xef, - 0x48, 0x7e, 0x03, 0x5e, 0xec, 0xe3, 0xbe, 0xfd, 0x25, 0x5d, 0x80, 0x40, 0x3a, 0x14, 0x84, 0x0f, - 0x32, 0xf9, 0xed, 0x77, 0xc1, 0xcf, 0xf7, 0xf6, 0x97, 0x74, 0x11, 0x04, 0x75, 0xa0, 0x48, 0xab, - 0xd3, 0xa2, 0xb1, 0xe7, 0xe7, 0x06, 0x15, 0xca, 0x54, 0x08, 0xa8, 0x2f, 0x54, 0xad, 0x34, 0x00, - 0x26, 0x59, 0x42, 0x7e, 0x57, 0x5d, 0x28, 0x3d, 0x47, 0xae, 0xbd, 0x71, 0x3a, 0x10, 0xf5, 0x61, - 0x5d, 0xf8, 0xe6, 0x25, 0xee, 0x6a, 0x71, 0xc1, 0xcf, 0x08, 0x85, 0x3a, 0x95, 0xfd, 0x25, 0x9d, - 0xbb, 0x55, 0x62, 0xf1, 0x0a, 0x06, 0x74, 0xbe, 0x1a, 0x79, 0x73, 0xe5, 0xea, 0x5f, 0x2b, 0x4e, - 0xc4, 0x08, 0xc4, 0x6a, 0x06, 0x96, 0x7d, 0xcf, 0x0b, 0x95, 0x5f, 0x66, 0xe0, 0xba, 0xfa, 0x05, - 0xee, 0x8d, 0x68, 0x65, 0x6a, 0x27, 0x34, 0x4f, 0xe2, 0xb5, 0xde, 0x86, 0x82, 0x10, 0x56, 0xe6, - 0x7b, 0x7b, 0xd1, 0x8f, 0x0a, 0x45, 0x08, 0x62, 0xf6, 0xd8, 0x6c, 0xf0, 0x43, 0xd1, 0xe6, 0x9a, - 0xbd, 0xa0, 0xa6, 0x58, 0x9d, 0xeb, 0xa0, 0xbe, 0xa8, 0xdf, 0x93, 0x09, 0xd4, 0xac, 0xa9, 0xca, - 0xe2, 0x37, 0xa7, 0x3e, 0x7f, 0x5e, 0xa6, 0xb9, 0x50, 0xf1, 0xfb, 0xe5, 0xcd, 0xc9, 0x97, 0x72, - 0x69, 0xfa, 0x32, 0xfe, 0xda, 0x6d, 0xda, 0xc8, 0x65, 0x5e, 0xd6, 0xc8, 0xf5, 0xa1, 0x30, 0x0a, - 0xb0, 0x4f, 0x13, 0x4d, 0x38, 0xd8, 0xcc, 0xbe, 0xec, 0x80, 0x0f, 0x02, 0xec, 0xd3, 0x92, 0x35, - 0x32, 0xe0, 0x51, 0xf4, 0x10, 0xa0, 0xa7, 0x90, 0xa1, 0x79, 0xc8, 0x60, 0x33, 0x47, 0x45, 0x54, - 0xae, 0x2e, 0x82, 0x56, 0xb6, 0x69, 0x96, 0xce, 0x01, 0xe5, 0x16, 0x14, 0x04, 0x35, 0xcf, 0xe3, - 0x2e, 0x7c, 0x1d, 0xc0, 0xf1, 0x7a, 0xa6, 0xc3, 0x4a, 0xfe, 0xd9, 0x02, 0xc8, 0x53, 0x4a, 0xd3, - 0x1c, 0x60, 0x02, 0x28, 0x0c, 0xe3, 0x15, 0x00, 0x3e, 0x86, 0x2c, 0xef, 0xf4, 0xcb, 0x83, 0x6d, - 0xff, 0x61, 0x42, 0x2c, 0xb7, 0xe2, 0x0e, 0x27, 0xfa, 0x09, 0x64, 0x58, 0xd1, 0x06, 0x9a, 0x27, - 0x4a, 0x37, 0x55, 0xdf, 0x21, 0xbf, 0x44, 0xfd, 0x95, 0xb2, 0x74, 0x3b, 0xb1, 0xf3, 0x29, 0xe4, - 0xe8, 0x5f, 0x29, 0x10, 0x7f, 0xf1, 0xad, 0x73, 0xfe, 0x06, 0xf1, 0x11, 0xa8, 0xa7, 0xd1, 0x1a, - 0xb2, 0x6f, 0xf1, 0xff, 0xe9, 0xcf, 0xff, 0xe6, 0x09, 0xf3, 0x28, 0x08, 0xd7, 0x81, 0xef, 0xee, - 0x68, 0xb0, 0x42, 0x01, 0x7a, 0xfc, 0x2f, 0x0d, 0xe6, 0x41, 0xf9, 0xe7, 0x08, 0xa5, 0x78, 0x2c, - 0xfc, 0x35, 0x42, 0xf5, 0xdb, 0xf0, 0xd5, 0xff, 0xdf, 0x50, 0xcd, 0xeb, 0xb4, 0x9c, 0xa3, 0x32, - 0xb4, 0x3f, 0x2b, 0x44, 0x74, 0xe3, 0x6c, 0xeb, 0x38, 0x43, 0xc5, 0xdd, 0xf9, 0xdf, 0x00, 0x00, - 0x00, 0xff, 0xff, 0x6e, 0x30, 0x2c, 0x4d, 0x1a, 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. From c7a04d725e8e488c4eba5f972af4989cb145812c Mon Sep 17 00:00:00 2001 From: Udi Meiri Date: Mon, 30 Dec 2019 15:53:25 -0800 Subject: [PATCH 29/67] [BEAM-8487] Convert forward references to Any (#9888) * [BEAM-8487] Convert forward references to Any * Fix python docs not recognizing typing.Union --- .../typehints/native_type_compatibility.py | 8 +++++++- .../typehints/native_type_compatibility_test.py | 5 +++++ .../typehints/typed_pipeline_test_py3.py | 17 ++++++++++++++++- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py index 435879d6b9b0..f2802e0bbf52 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -189,7 +189,8 @@ def convert_to_beam_type(typ): """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 @@ -209,6 +210,11 @@ def convert_to_beam_type(typ): _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 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 38e75dcf3660..4bfb60bc37a9 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 @@ def test_generator_converted_to_iterator(self): 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 e49b11ca8847..a718c8b7c94d 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 @@ def process(self, element: int, *, side_input: str) -> \ 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 @@ def process(self, element: int, **side_inputs: typehints.Dict[str, str]) \ 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): From 662f8d6e5573a9a137b0349f7b1e626efbe30bf1 Mon Sep 17 00:00:00 2001 From: Tomo Suzuki Date: Mon, 30 Dec 2019 19:01:02 -0500 Subject: [PATCH 30/67] [BEAM-9020] LengthPrefixUnknownCodersTest to avoid relying on AbstractMap's equality (#10467) * GenericJsonAssert to take Objects * Apply assertEqualsAsJson * GenericJsonMatcher * Comment update --- .../graph/LengthPrefixUnknownCodersTest.java | 53 ++++++++------- .../worker/testing/GenericJsonAssert.java | 15 ++-- .../worker/testing/GenericJsonMatcher.java | 68 +++++++++++++++++++ .../testing/GenericJsonMatcherTest.java | 63 +++++++++++++++++ 4 files changed, 169 insertions(+), 30 deletions(-) create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcher.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java 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 ebe0d4e320a2..a8b37b02b90e 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 setup() { public void testLengthPrefixUnknownCoders() throws Exception { Map lengthPrefixedCoderCloudObject = forCodec(CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null), false); - assertEquals( + assertEqualsAsJson( CloudObjects.asCloudObject(prefixedWindowedValueCoder, /*sdkComponents=*/ null), lengthPrefixedCoderCloudObject); } @@ -126,7 +129,7 @@ public void testLengthPrefixForLengthPrefixCoder() throws Exception { KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); - assertEquals( + assertEqualsAsJson( CloudObjects.asCloudObject(expectedCoder, /*sdkComponents=*/ null), lengthPrefixedCoderCloudObject); } @@ -141,7 +144,7 @@ public void testLengthPrefixAndReplaceUnknownCoder() throws Exception { Map lengthPrefixedCoderCloudObject = forCodec(CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null), true); - assertEquals( + assertEqualsAsJson( CloudObjects.asCloudObject(prefixedAndReplacedWindowedValueCoder, /*sdkComponents=*/ null), lengthPrefixedCoderCloudObject); } @@ -153,11 +156,11 @@ public void testLengthPrefixInstructionOutputCoder() throws Exception { 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 @@ public void testLengthPrefixReadInstructionCoder() throws Exception { 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 @@ public void testLengthPrefixWriteInstructionCoder() throws Exception { 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 @@ public void testLengthPrefixParDoInstructionCoder() throws Exception { 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 @@ public void testClone() throws Exception { } @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 @@ public void testLengthPrefixAndReplaceForRunnerNetwork() { MutableNetwork prefixedNetwork = andReplaceForRunnerNetwork(network); - Set prefixedInstructions = new HashSet<>(); + ImmutableSet.Builder prefixedInstructions = ImmutableSet.builder(); for (Node node : prefixedNetwork.nodes()) { if (node instanceof ParallelInstructionNode) { prefixedInstructions.add(((ParallelInstructionNode) node).getParallelInstruction()); @@ -252,11 +255,11 @@ public void testLengthPrefixAndReplaceForRunnerNetwork() { } } - 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 @@ public void testLengthPrefixForInstructionOutputNodeWithGrpcNodeSuccessor() { 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 @@ public void testLengthPrefixForInstructionOutputNodeWithGrpcNodePredecessor() { 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 @@ public void testLengthPrefixForInstructionOutputNodeWithNonGrpcNodeNeighbor() { 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 @@ public void testLengthPrefixForSideInputInfos() { ImmutableList.of( createSideInputInfosWithCoders(windowedValueCoder, prefixedWindowedValueCoder)), false); - assertEquals( + assertEqualsAsJson( ImmutableList.of( createSideInputInfosWithCoders(prefixedWindowedValueCoder, prefixedWindowedValueCoder)), prefixedSideInputInfos); @@ -316,7 +319,7 @@ public void testLengthPrefixForSideInputInfos() { 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/testing/GenericJsonAssert.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonAssert.java index d44f921afb15..f4e726851297 100644 --- 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 @@ -19,6 +19,7 @@ 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; @@ -28,19 +29,23 @@ public class GenericJsonAssert { private static final JacksonFactory jacksonFactory = JacksonFactory.getDefaultInstance(); /** - * Asserts that {@code actual} has the same JSON representation as {@code expectedJsonText}. + * Asserts that {@code actual} has the same JSON representation as {@code expected}. * - * @param expectedJsonText expected JSON string. + * @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(String expectedJsonText, GenericJson actual) { - actual.setFactory(jacksonFactory); + public static void assertEqualsAsJson(Object expected, Object actual) { - String actualJsonText = actual.toString(); 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 000000000000..e576a33cd746 --- /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 { + + 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 000000000000..cac9fe3da23d --- /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"); + } +} From 13945c93c59aaabc01bbad91ee0b941168ae2da6 Mon Sep 17 00:00:00 2001 From: Jincheng Sun Date: Tue, 31 Dec 2019 08:01:42 +0800 Subject: [PATCH 31/67] [BEAM-7951] Improve the docs for beam_runner_api.proto and WindowedValue.java (#10458) --- .../src/main/proto/beam_runner_api.proto | 12 +++++----- .../apache/beam/sdk/util/WindowedValue.java | 22 +++++++++---------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto index 0105e896d26c..df5d59d73f72 100644 --- a/model/pipeline/src/main/proto/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/beam_runner_api.proto @@ -679,12 +679,12 @@ message StandardCoders { // 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 assign the parameterized timestamp, windows and PaneInfo to 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 beam:coder:bytes:v1 - // elements coder and the window coder that this param_windowed_value uses. + // 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. @@ -1089,8 +1089,8 @@ message WireCoderSetting { // 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 - // beam:coder:bytes:v1 elements coder and the window coder that this - // param_windowed_value uses. + // a beam:coder:bytes:v1 element coder and the window coder that this + // param_windowed_value coder uses. bytes payload = 2; } 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 7e1dddc03bf0..55c16013775b 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 @@ -648,7 +648,7 @@ public List> getComponents() { *

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 general purpose implementation of the + * @deprecated Use ParamWindowedValueCoder instead, it is a general purpose implementation of the * same concept but makes timestamp, windows and pane info configurable. */ @Deprecated @@ -709,7 +709,7 @@ public List> getCoderArguments() { } /** - * A parameterized Coder for {@code WindowedValue}. + * A parameterized coder for {@code WindowedValue}. * *

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 @@ -726,8 +726,8 @@ public static class ParamWindowedValueCoder extends FullWindowedValueCoder private static final byte[] EMPTY_BYTES = new byte[0]; /** - * Returns the {@link ParamWindowedValueCoder} from the given valueCoder, windowCoder and the - * supplied parameterized timestamp, windows and pane info for {@link WindowedValue}. + * Returns the {@link ParamWindowedValueCoder} for the given valueCoder and windowCoder using + * the supplied parameterized timestamp, windows and pane info for {@link WindowedValue}. */ public static ParamWindowedValueCoder of( Coder valueCoder, @@ -739,9 +739,9 @@ public static ParamWindowedValueCoder of( } /** - * Returns the {@link ParamWindowedValueCoder} from the given valueCoder, windowCoder and the - * supplied parameterized {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, {@link #GLOBAL_WINDOWS} and - * {@link PaneInfo#NO_FIRING} for {@link WindowedValue}. + * 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 ParamWindowedValueCoder of( Coder valueCoder, Coder windowCoder) { @@ -754,10 +754,10 @@ public static ParamWindowedValueCoder of( } /** - * Returns the {@link ParamWindowedValueCoder} from the given valueCoder, {@link - * GlobalWindow.Coder#INSTANCE} and the supplied parameterized {@link - * BoundedWindow#TIMESTAMP_MIN_VALUE}, {@link #GLOBAL_WINDOWS} and {@link PaneInfo#NO_FIRING} - * for {@link WindowedValue}. + * 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 ParamWindowedValueCoder of(Coder valueCoder) { return ParamWindowedValueCoder.of(valueCoder, GlobalWindow.Coder.INSTANCE); From 5ed285084e282aab1d52f57c0f4c3c37ea01e861 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Wed, 25 Dec 2019 12:15:01 +0800 Subject: [PATCH 32/67] [BEAM-9006] Improve ProcessManager for shutdown hook handling. --- .../environment/ProcessManager.java | 38 +++++++++++++------ .../environment/ProcessManagerTest.java | 30 +++++++++++++++ 2 files changed, 57 insertions(+), 11 deletions(-) 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 65fcdf2a3a2b..f1d88d03309a 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 @@ public class ProcessManager { /** A list of all managers to ensure all processes shutdown on JVM exit . */ private static final List 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 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 @@ public RunningProcess startProcess( return startProcess(id, command, args, env, outputFile); } + @SuppressFBWarnings("ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD") public RunningProcess startProcess( String id, String command, List args, Map env, File outputFile) throws IOException { @@ -149,6 +143,15 @@ public RunningProcess startProcess( 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 @@ public RunningProcess startProcess( } /** 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/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 d0c02c663280..a1377f33a335 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 @@ public void testInheritIO() throws IOException, InterruptedException { // 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); + } } From 2b17befc616cf85b2af375a8dafdee9606b82131 Mon Sep 17 00:00:00 2001 From: Kamil Wasilewski Date: Thu, 2 Jan 2020 12:20:47 +0100 Subject: [PATCH 33/67] [BEAM-8671] Fix Python 3.7 ParDo test job name --- .test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy index d68ddb902e4d..71c5d6f7528a 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Python_37.groovy @@ -66,6 +66,6 @@ PhraseTriggeringPostCommitBuilder.postCommitJob( batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR) } -CronJobBuilder.cronJob('beam_LoadTests_Python_ParDo_Dataflow_Batch', 'H 13 * * *', this) { +CronJobBuilder.cronJob('beam_LoadTests_Python_37_ParDo_Dataflow_Batch', 'H 13 * * *', this) { batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT) } From 5b4ea812e4c56e3fdb12bd4f546b0c118cb77f25 Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Thu, 2 Jan 2020 14:10:06 -0800 Subject: [PATCH 34/67] [BEAM-5600] Add unimplemented split API to Runner side SDF libraries. (#10482) * [BEAM-5600] Add unimplemented split API to Runner side SDF libraries. This creates an unsupported API to the RemoteBundle and a default split handler that throws. --- .../FlinkExecutableStageFunctionTest.java | 5 + .../ExecutableStageDoFnOperatorTest.java | 5 + .../control/BundleSplitHandler.java | 46 +++ .../control/DefaultJobBundleFactory.java | 5 + .../fnexecution/control/RemoteBundle.java | 9 + .../fnexecution/control/SdkHarnessClient.java | 283 +++++++++++------- .../control/RemoteExecutionTest.java | 19 +- .../control/SdkHarnessClientTest.java | 26 +- .../SparkExecutableStageFunctionTest.java | 5 + 9 files changed, 274 insertions(+), 129 deletions(-) create mode 100644 runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleSplitHandler.java 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 89af4d3022c4..61d89062aa17 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 @@ -197,6 +197,11 @@ public Map getInputReceivers() { }); } + @Override + public void split(double fractionOfRemainder) { + throw new UnsupportedOperationException(); + } + @Override public void close() throws Exception { if (once) { 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 69c64283e551..7fdec35504ce 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 @@ -314,6 +314,11 @@ public Map getInputReceivers() { }); } + @Override + public void split(double fractionOfRemainder) { + throw new UnsupportedOperationException(); + } + @Override public void close() throws Exception { if (onceEmitted) { 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 000000000000..cb03238720c6 --- /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. + * + *

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 breaking the fusion barrier 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 a5f70c62344f..006790f213a5 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 @@ -399,6 +399,11 @@ public Map getInputReceivers() { return bundle.getInputReceivers(); } + @Override + public void split(double fractionOfRemainder) { + bundle.split(fractionOfRemainder); + } + @Override public void close() throws Exception { bundle.close(); 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 30fea857b6d3..cd7ac11ffe32 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 @@ -38,6 +38,15 @@ public interface RemoteBundle extends AutoCloseable { */ Map 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. + * + *

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 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 2799e5875d49..08d50b0c24c0 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 @@ public ActiveBundle newBundle( outputReceivers, stateRequestHandler, progressHandler, + BundleSplitHandler.unsupported(), request -> { throw new UnsupportedOperationException( String.format( @@ -174,6 +178,7 @@ public ActiveBundle newBundle( Map> outputReceivers, StateRequestHandler stateRequestHandler, BundleProgressHandler progressHandler, + BundleSplitHandler splitHandler, BundleCheckpointHandler checkpointHandler, BundleFinalizationHandler finalizationHandler) { String bundleId = idGenerator.getId(); @@ -205,14 +210,15 @@ public ActiveBundle newBundle( outputClients.put(receiver.getKey(), outputClient); } - ImmutableMap.Builder dataReceiversBuilder = + ImmutableMap.Builder dataReceiversBuilder = ImmutableMap.builder(); for (Map.Entry 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 @@ public ActiveBundle newBundle( outputClients, stateDelegator.registerForProcessBundleInstructionId(bundleId, stateRequestHandler), progressHandler, + splitHandler, checkpointHandler, finalizationHandler); } @@ -231,71 +238,123 @@ private InboundDataClient attachReceiver( 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 response; - private final Map inputReceivers; - private final Map outputClients; - private final StateDelegator.Registration stateRegistration; - private final BundleProgressHandler progressHandler; - private final BundleCheckpointHandler checkpointHandler; - private final BundleFinalizationHandler finalizationHandler; - - private ActiveBundle( - String bundleId, - CompletionStage response, - Map inputReceivers, - Map 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; - } + /** An active bundle for a particular {@link BeamFnApi.ProcessBundleDescriptor}. */ + public class ActiveBundle implements RemoteBundle { + private final String bundleId; + private final CompletionStage response; + private final Map inputReceivers; + private final Map 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 response, + Map inputReceivers, + Map 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 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 getInputReceivers() { + return (Map) inputReceivers; + } - /** - * Blocks until bundle processing is finished. This is comprised of: - * - *

    - *
  • closing each {@link #getInputReceivers() input receiver}. - *
  • waiting for the SDK to say that processing the bundle is finished. - *
  • waiting for all inbound data clients to complete - *
- * - *

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 splits = new HashMap<>(); + for (Map.Entry 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 response = fnApiControlClient.handle(request); + response.thenAccept( + instructionResponse -> splitHandler.split(instructionResponse.getProcessBundleSplit())); + } + + /** + * Blocks until bundle processing is finished. This is comprised of: + * + *

    + *
  • closing each {@link #getInputReceivers() input receiver}. + *
  • waiting for the SDK to say that processing the bundle is finished. + *
  • waiting for all inbound data clients to complete + *
+ * + *

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(); + // 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; @@ -303,50 +362,11 @@ public void close() throws Exception { 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(); + stateRegistration.deregister(); } else { - outputClient.cancel(); + stateRegistration.abort(); } } catch (Exception e) { if (exception == null) { @@ -355,9 +375,24 @@ public void close() throws Exception { exception.addSuppressed(e); } } - } - if (exception != null) { - throw exception; + 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 @@ public void abort() {} } } + /** + * A {@link CloseableFnDataReceiver} which counts the number of elements that have been accepted. + */ + private static class CountingFnDataReceiver implements CloseableFnDataReceiver { + 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/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 dc732037ea83..d9d51d4c398c 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 @@ public void process(ProcessContext ctxt) { } // 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 @@ public void process(ProcessContext ctxt) throws Exception { (FnDataReceiver>) outputContents::add)); } - try (ActiveBundle bundle = + try (RemoteBundle bundle = processor.newBundle(outputReceivers, BundleProgressHandler.ignored())) { Iterables.getOnlyElement(bundle.getInputReceivers().values()) .accept( @@ -359,7 +358,7 @@ public void process(ProcessContext ctxt) throws Exception { } try { - try (ActiveBundle bundle = + try (RemoteBundle bundle = processor.newBundle(outputReceivers, BundleProgressHandler.ignored())) { Iterables.getOnlyElement(bundle.getInputReceivers().values()) .accept( @@ -372,7 +371,7 @@ public void process(ProcessContext ctxt) throws Exception { 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 @@ public Coder valueCoder() { }); 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 @@ public void onCompleted(ProcessBundleResponse response) { } }; - try (ActiveBundle bundle = + try (RemoteBundle bundle = processor.newBundle(outputReceivers, stateRequestHandler, progressHandler)) { Iterables.getOnlyElement(bundle.getInputReceivers().values()) .accept( @@ -959,7 +958,7 @@ public void clear(ByteString key, BoundedWindow window) { } }); - try (ActiveBundle bundle = + try (RemoteBundle bundle = processor.newBundle( outputReceivers, stateRequestHandler, BundleProgressHandler.ignored())) { Iterables.getOnlyElement(bundle.getInputReceivers().values()) @@ -1102,7 +1101,7 @@ public void processingTimer( // 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 @@ public void process(ProcessContext c) { (Coder>) 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 ef0b2ac84043..57b5cbbda974 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 @@ public void testNewBundleNoDataDoesNotCrash() throws Exception { (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 @@ public void testNewBundleAndProcessElements() throws Exception { SDK_GRPC_READ_TRANSFORM))); Collection> outputs = new ArrayList<>(); - try (ActiveBundle activeBundle = + try (RemoteBundle activeBundle = processor.newBundle( Collections.singletonMap( SDK_GRPC_WRITE_TRANSFORM, @@ -311,7 +311,7 @@ public void handleCleanupWhenInputSenderFails() throws Exception { 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 @@ public void handleCleanupWithStateWhenInputSenderFails() throws Exception { 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 @@ public void handleCleanupWhenProcessingBundleFails() throws Exception { 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 @@ public void handleCleanupWithStateWhenProcessingBundleFails() throws Exception { 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 @@ public void handleCleanupWhenAwaitingOnClosingOutputReceivers() throws Exception 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 @@ public void handleCleanupWithStateWhenAwaitingOnClosingOutputReceivers() throws 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 @@ public void testBundleCheckpointCallback() throws Exception { 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 @@ public void testBundleCheckpointCallback() throws Exception { throw new UnsupportedOperationException(); }, mockProgressHandler, + mockSplitHandler, mockCheckpointHandler, mockFinalizationHandler)) { processBundleResponseFuture.complete( @@ -671,7 +673,7 @@ public void testBundleCheckpointCallback() throws Exception { verify(mockProgressHandler).onCompleted(response); verify(mockCheckpointHandler).onCheckpoint(response); - verifyZeroInteractions(mockFinalizationHandler); + verifyZeroInteractions(mockFinalizationHandler, mockSplitHandler); } @Test @@ -698,6 +700,7 @@ public void testBundleFinalizationCallback() throws Exception { 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 @@ public void testBundleFinalizationCallback() throws Exception { throw new UnsupportedOperationException(); }, mockProgressHandler, + mockSplitHandler, mockCheckpointHandler, mockFinalizationHandler)) { bundleId = activeBundle.getId(); @@ -720,7 +724,7 @@ public void testBundleFinalizationCallback() throws Exception { verify(mockProgressHandler).onCompleted(response); verify(mockFinalizationHandler).requestsFinalization(bundleId); - verifyZeroInteractions(mockCheckpointHandler); + verifyZeroInteractions(mockCheckpointHandler, mockSplitHandler); } private CompletableFuture createRegisterResponse() { 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 5a59fdd8c657..f89f115e3668 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 @@ -160,6 +160,11 @@ public Map getInputReceivers() { }); } + @Override + public void split(double fractionOfRemainder) { + throw new UnsupportedOperationException(); + } + @Override public void close() throws Exception { if (once) { From 20ef57e9a255a00829f8d24c813994390788093f Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Thu, 2 Jan 2020 17:47:59 -0800 Subject: [PATCH 35/67] [BEAM-5605] Fix type used to describe channel splits to match type used on estimated_input_elements. (#10490) --- .../src/main/proto/beam_fn_api.proto | 4 +- .../pkg/beam/core/runtime/harness/harness.go | 4 +- .../model/fnexecution_v1/beam_fn_api.pb.go | 412 +++++++++--------- .../model/pipeline_v1/beam_runner_api.pb.go | 12 +- 4 files changed, 216 insertions(+), 216 deletions(-) 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 f8bcf8bef2c9..3824e9027c8d 100644 --- a/model/fn-execution/src/main/proto/beam_fn_api.proto +++ b/model/fn-execution/src/main/proto/beam_fn_api.proto @@ -481,12 +481,12 @@ message ProcessBundleSplitResponse { // 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/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index 28bdf4c4cb0a..58d4d2dddfea 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 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe 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/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go index c4cb943dad97..19bdfa4f805c 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 @@ -1967,11 +1967,11 @@ type ProcessBundleSplitResponse_ChannelSplit struct { // 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:"-"` @@ -2011,14 +2011,14 @@ func (m *ProcessBundleSplitResponse_ChannelSplit) GetTransformId() string { 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 } @@ -3660,208 +3660,208 @@ func init() { func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_6d954c03a4758710) } var fileDescriptor_6d954c03a4758710 = []byte{ - // 3216 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xcd, 0x8f, 0x1b, 0xc7, - 0x95, 0x57, 0x0f, 0xc9, 0x19, 0xf2, 0x91, 0x33, 0x43, 0xd6, 0xcc, 0x48, 0x54, 0xaf, 0xbc, 0x2b, - 0x73, 0xd7, 0xc0, 0xc0, 0x0b, 0x53, 0x9f, 0xb0, 0x25, 0xaf, 0x2d, 0x7b, 0x86, 0x43, 0x49, 0xb4, - 0x46, 0x12, 0xb7, 0x47, 0x5a, 0xed, 0xda, 0x6b, 0x37, 0x7a, 0xd8, 0x45, 0xaa, 0xa1, 0x66, 0x77, - 0xbb, 0xaa, 0x39, 0xd2, 0x28, 0x46, 0x9c, 0x2f, 0x24, 0x48, 0x90, 0xc4, 0x97, 0x1c, 0x9c, 0xdc, - 0x92, 0x00, 0x01, 0x72, 0xc9, 0x1f, 0xe0, 0x6b, 0x0e, 0x41, 0x4e, 0xf9, 0x07, 0x7c, 0x09, 0x90, - 0x00, 0x49, 0x9c, 0x73, 0x80, 0xdc, 0x82, 0xfa, 0xe8, 0x0f, 0x36, 0x9b, 0x72, 0x93, 0x33, 0xca, - 0xad, 0xab, 0xaa, 0xdf, 0xef, 0x57, 0xf5, 0xfa, 0x57, 0xaf, 0x5e, 0x55, 0x35, 0xd4, 0xf6, 0xb1, - 0x31, 0xd4, 0xfb, 0x8e, 0x6e, 0x78, 0x56, 0xd3, 0x23, 0xae, 0xef, 0xa2, 0x97, 0x5c, 0x32, 0x68, - 0x1a, 0x9e, 0xd1, 0x7b, 0x88, 0x9b, 0xac, 0xb5, 0x39, 0x74, 0x4d, 0x6c, 0x37, 0xfb, 0x8e, 0x8e, - 0x9f, 0xe0, 0xde, 0xc8, 0xb7, 0x5c, 0xa7, 0x79, 0x70, 0x41, 0xdd, 0xe0, 0x96, 0x64, 0xe4, 0x38, - 0x98, 0x44, 0xd6, 0xea, 0x2a, 0x76, 0x4c, 0xcf, 0xb5, 0x1c, 0x9f, 0xca, 0x8a, 0xb3, 0x03, 0xd7, - 0x1d, 0xd8, 0xf8, 0x1c, 0x2f, 0xed, 0x8f, 0xfa, 0xe7, 0x4c, 0x4c, 0x7b, 0xc4, 0xf2, 0x7c, 0x97, - 0xc8, 0x37, 0xfe, 0x2d, 0xf9, 0x86, 0x6f, 0x0d, 0x31, 0xf5, 0x8d, 0xa1, 0x27, 0x5f, 0xf8, 0xd7, - 0x09, 0x88, 0x11, 0x31, 0x78, 0x3f, 0xa6, 0xb4, 0x3f, 0x26, 0x86, 0xe7, 0x61, 0x12, 0x74, 0x61, - 0x79, 0x88, 0x7d, 0x62, 0xf5, 0x64, 0xb1, 0xf1, 0x13, 0x05, 0x56, 0x34, 0x3c, 0x74, 0x7d, 0x7c, - 0x83, 0x78, 0xbd, 0xae, 0x4b, 0x7c, 0x34, 0x84, 0x93, 0x86, 0x67, 0xe9, 0x14, 0x93, 0x03, 0xab, - 0x87, 0xf5, 0xa8, 0x8b, 0x75, 0xe5, 0xac, 0xb2, 0x59, 0xbe, 0xf8, 0x5a, 0x33, 0xdd, 0x29, 0x9e, - 0xe5, 0x61, 0xdb, 0x72, 0x70, 0xf3, 0xe0, 0x42, 0x73, 0xcb, 0xb3, 0xf6, 0x84, 0xfd, 0x4e, 0x68, - 0xae, 0xad, 0x1b, 0x29, 0xb5, 0xe8, 0x34, 0x14, 0x7b, 0xae, 0x89, 0x89, 0x6e, 0x99, 0xf5, 0x85, - 0xb3, 0xca, 0x66, 0x49, 0x5b, 0xe2, 0xe5, 0x8e, 0xd9, 0xf8, 0x63, 0x1e, 0x50, 0xc7, 0xa1, 0x3e, - 0x19, 0xf5, 0xd8, 0x08, 0x35, 0xfc, 0xe1, 0x08, 0x53, 0x1f, 0xbd, 0x04, 0x2b, 0x56, 0x54, 0xcb, - 0xec, 0x14, 0x6e, 0xb7, 0x1c, 0xab, 0xed, 0x98, 0xe8, 0x3e, 0x14, 0x09, 0x1e, 0x58, 0xd4, 0xc7, - 0xa4, 0xfe, 0xa7, 0x25, 0xde, 0xf5, 0x57, 0x9b, 0x99, 0xbe, 0x67, 0x53, 0x93, 0x76, 0x92, 0xf1, - 0xe6, 0x09, 0x2d, 0x84, 0x42, 0x18, 0x56, 0x3c, 0xe2, 0xf6, 0x30, 0xa5, 0xfa, 0xfe, 0xc8, 0x31, - 0x6d, 0x5c, 0xff, 0xb3, 0x00, 0xff, 0xaf, 0x8c, 0xe0, 0x5d, 0x61, 0xbd, 0xcd, 0x8d, 0x23, 0x86, - 0x65, 0x2f, 0x5e, 0x8f, 0xbe, 0x0a, 0xa7, 0xc6, 0x69, 0x74, 0x8f, 0xb8, 0x03, 0x82, 0x29, 0xad, - 0xff, 0x45, 0xf0, 0xb5, 0xe6, 0xe1, 0xeb, 0x4a, 0x90, 0x88, 0x77, 0xc3, 0x4b, 0x6b, 0x47, 0x23, - 0x58, 0x4f, 0xf0, 0x53, 0xcf, 0xb6, 0xfc, 0xfa, 0x17, 0x82, 0xfc, 0xed, 0x79, 0xc8, 0xf7, 0x18, - 0x42, 0xc4, 0x8c, 0xbc, 0x89, 0x46, 0xf4, 0x10, 0x56, 0xfb, 0x96, 0x63, 0xd8, 0xd6, 0x53, 0x1c, - 0xb8, 0xf7, 0xaf, 0x82, 0xf1, 0x8d, 0x8c, 0x8c, 0xd7, 0xa5, 0x79, 0xd2, 0xbf, 0x2b, 0xfd, 0xb1, - 0x86, 0xed, 0x12, 0x2c, 0x11, 0xd1, 0xd8, 0xf8, 0x46, 0x01, 0xd6, 0xc6, 0x74, 0x46, 0x3d, 0xd7, - 0xa1, 0x38, 0xab, 0xd0, 0xd6, 0xa1, 0x80, 0x09, 0x71, 0x89, 0x94, 0xaf, 0x28, 0xa0, 0xff, 0x99, - 0x94, 0xdf, 0x6b, 0x33, 0xcb, 0x4f, 0x74, 0x64, 0x4c, 0x7f, 0xfd, 0x69, 0xfa, 0x7b, 0x63, 0x3e, - 0xfd, 0x85, 0x14, 0x09, 0x01, 0x7e, 0xfc, 0xa5, 0x02, 0xdc, 0x39, 0x9a, 0x00, 0x43, 0xe2, 0x29, - 0x0a, 0x3c, 0x78, 0xb6, 0x02, 0xb7, 0x8e, 0xa0, 0xc0, 0x90, 0x3a, 0x4d, 0x82, 0xd6, 0x54, 0x09, - 0xbe, 0x39, 0xa7, 0x04, 0x43, 0xba, 0xa4, 0x06, 0x81, 0x69, 0x44, 0xb4, 0x36, 0x7e, 0xa0, 0xc0, - 0x6a, 0x22, 0xee, 0xa0, 0xa7, 0x70, 0x3a, 0xe1, 0x82, 0xb1, 0x68, 0x9c, 0xdb, 0x2c, 0x5f, 0xbc, - 0x36, 0x8f, 0x1b, 0x62, 0x41, 0xf9, 0x94, 0x97, 0xde, 0xd0, 0x40, 0x50, 0x4d, 0xea, 0xb0, 0xf1, - 0x73, 0x80, 0x53, 0x53, 0x80, 0xd0, 0x0a, 0x2c, 0x84, 0x13, 0x64, 0xc1, 0x32, 0x91, 0x03, 0xe0, - 0x13, 0xc3, 0xa1, 0x7d, 0x97, 0x0c, 0x69, 0x7d, 0x81, 0x77, 0xf6, 0xce, 0xd1, 0x3a, 0xdb, 0xbc, - 0x17, 0x02, 0xb6, 0x1d, 0x9f, 0x1c, 0x6a, 0x31, 0x06, 0xe4, 0x43, 0xc5, 0xeb, 0xb9, 0xb6, 0x8d, - 0xf9, 0xb4, 0xa4, 0xf5, 0x1c, 0x67, 0xec, 0x1e, 0x91, 0xb1, 0x1b, 0x83, 0x14, 0x9c, 0x63, 0x2c, - 0xe8, 0x7b, 0x0a, 0xac, 0x3f, 0xb6, 0x1c, 0xd3, 0x7d, 0x6c, 0x39, 0x03, 0x9d, 0xfa, 0xc4, 0xf0, - 0xf1, 0xc0, 0xc2, 0xb4, 0x9e, 0xe7, 0xf4, 0x0f, 0x8e, 0x48, 0xff, 0x20, 0x80, 0xde, 0x0b, 0x91, - 0x45, 0x2f, 0xd6, 0x1e, 0x4f, 0xb6, 0xa0, 0x7d, 0x58, 0xe4, 0x4b, 0x27, 0xad, 0x17, 0x38, 0xfb, - 0x3b, 0x47, 0x64, 0x6f, 0x71, 0x30, 0x41, 0x28, 0x91, 0x99, 0x9b, 0xb1, 0x73, 0x60, 0x11, 0xd7, - 0x19, 0x62, 0xc7, 0xa7, 0xf5, 0xc5, 0x63, 0x71, 0x73, 0x3b, 0x06, 0x29, 0xdd, 0x1c, 0x67, 0x41, - 0x4f, 0xe0, 0x0c, 0xf5, 0x0d, 0x1f, 0xeb, 0x53, 0x32, 0x93, 0xa5, 0xa3, 0x65, 0x26, 0xa7, 0x39, - 0x78, 0x5a, 0x93, 0x6a, 0xc3, 0x6a, 0x42, 0x75, 0xa8, 0x0a, 0xb9, 0x47, 0xf8, 0x50, 0x4a, 0x9d, - 0x3d, 0xa2, 0x16, 0x14, 0x0e, 0x0c, 0x7b, 0x84, 0xf9, 0x0a, 0x50, 0xbe, 0xf8, 0x4a, 0x86, 0x7e, - 0x74, 0x43, 0x54, 0x4d, 0xd8, 0xbe, 0xbe, 0x70, 0x45, 0x51, 0x5d, 0xa8, 0x4d, 0x28, 0x2e, 0x85, - 0x6f, 0x67, 0x9c, 0xaf, 0x99, 0x85, 0xaf, 0x15, 0xc2, 0xc6, 0x09, 0x3f, 0x82, 0xfa, 0x34, 0x8d, - 0xa5, 0xf0, 0xbe, 0x33, 0xce, 0x7b, 0x39, 0x03, 0x6f, 0x12, 0xfd, 0x30, 0xce, 0xde, 0x83, 0x72, - 0x4c, 0x63, 0x29, 0x84, 0xd7, 0xc6, 0x09, 0x37, 0x33, 0x10, 0x72, 0xc0, 0x84, 0x4f, 0x27, 0xe4, - 0x75, 0x3c, 0x3e, 0x8d, 0xc1, 0xc6, 0x08, 0x1b, 0x7f, 0xcf, 0x41, 0x4d, 0x28, 0x7c, 0xcb, 0xf3, - 0x6c, 0xab, 0xc7, 0xd3, 0x73, 0xf4, 0x22, 0x54, 0xc2, 0x68, 0x15, 0xa5, 0x12, 0xe5, 0xb0, 0xae, - 0x63, 0xb2, 0x54, 0xd8, 0x72, 0xbc, 0x91, 0x1f, 0x4b, 0x85, 0x79, 0xb9, 0x63, 0xa2, 0x3a, 0x2c, - 0x61, 0x1b, 0x33, 0xa6, 0x7a, 0xee, 0xac, 0xb2, 0x59, 0xd1, 0x82, 0x22, 0xfa, 0x0a, 0xd4, 0xdc, - 0x91, 0xcf, 0xac, 0x1e, 0x1b, 0x3e, 0x26, 0x43, 0x83, 0x3c, 0x0a, 0xa2, 0x4f, 0xd6, 0x70, 0x3b, - 0xd1, 0xd9, 0xe6, 0x5d, 0x8e, 0xf8, 0x20, 0x04, 0x14, 0x73, 0xb2, 0xea, 0x26, 0xaa, 0x51, 0x17, - 0xc0, 0xa2, 0xfa, 0xbe, 0x3b, 0x72, 0x4c, 0x6c, 0xd6, 0x0b, 0x67, 0x95, 0xcd, 0x95, 0x8b, 0x17, - 0x32, 0x78, 0xae, 0x43, 0xb7, 0x85, 0x4d, 0xb3, 0xed, 0x8c, 0x86, 0x5a, 0xc9, 0x0a, 0xca, 0xe8, - 0xff, 0xa1, 0x3a, 0x74, 0x1d, 0xcb, 0x77, 0x09, 0x0b, 0xa8, 0x96, 0xd3, 0x77, 0x83, 0x18, 0x93, - 0x05, 0xf7, 0x76, 0x68, 0xda, 0x71, 0xfa, 0xae, 0xb6, 0x3a, 0x1c, 0x2b, 0x53, 0x55, 0x87, 0x8d, - 0xd4, 0xa1, 0xa5, 0xe8, 0xe1, 0xfc, 0xb8, 0x1e, 0xd4, 0xa6, 0xd8, 0x58, 0x35, 0x83, 0x8d, 0x55, - 0xf3, 0x5e, 0xb0, 0x33, 0x8b, 0x7f, 0xfb, 0x5f, 0x2b, 0x50, 0xdf, 0xc1, 0xb6, 0x71, 0x88, 0xcd, - 0x49, 0x09, 0xbc, 0x0b, 0x65, 0x23, 0x2a, 0xca, 0xed, 0xd4, 0x95, 0x79, 0x3f, 0x92, 0x16, 0x07, - 0x43, 0xb7, 0x60, 0x5d, 0xa6, 0xb3, 0xd8, 0xd4, 0xd9, 0xa6, 0x51, 0x37, 0x59, 0x37, 0x64, 0xef, - 0x4f, 0x4f, 0xf4, 0x7e, 0x47, 0x6e, 0x1b, 0x35, 0x14, 0x9a, 0xb1, 0x01, 0xf1, 0xbe, 0x37, 0x7e, - 0x98, 0x87, 0xf5, 0xb4, 0x6d, 0x0a, 0x7a, 0x0b, 0xce, 0x4c, 0x4d, 0x48, 0x22, 0x51, 0x9f, 0x9e, - 0x92, 0x53, 0x74, 0x4c, 0x64, 0x41, 0xa5, 0xc7, 0x46, 0xaa, 0xfb, 0xee, 0x23, 0xec, 0x04, 0x79, - 0xc1, 0xf5, 0x23, 0x6c, 0x9d, 0x9a, 0x2d, 0x66, 0x75, 0x8f, 0xc1, 0x69, 0xe5, 0x5e, 0xf8, 0x4c, - 0xd5, 0xdf, 0x2e, 0x00, 0x44, 0x6d, 0xe8, 0x43, 0x80, 0x11, 0xc5, 0x44, 0xe7, 0xa1, 0x5e, 0xfa, - 0xbe, 0x7b, 0x3c, 0xbc, 0xcd, 0xfb, 0x14, 0x93, 0x3d, 0x86, 0x7b, 0xf3, 0x84, 0x56, 0x1a, 0x05, - 0x05, 0x46, 0x49, 0x2d, 0x13, 0xeb, 0x7c, 0x12, 0xcb, 0x2f, 0x71, 0x5c, 0x94, 0x7b, 0x96, 0x89, - 0x3b, 0x0c, 0x97, 0x51, 0xd2, 0xa0, 0xc0, 0xf6, 0x22, 0xdc, 0xb3, 0x75, 0xe0, 0x51, 0x42, 0x14, - 0xd4, 0x32, 0x94, 0xc2, 0x2e, 0xaa, 0x2f, 0x43, 0x29, 0x34, 0x46, 0x2f, 0x8c, 0x75, 0x51, 0x7c, - 0xbe, 0x08, 0x6e, 0x7b, 0x11, 0xf2, 0xfe, 0xa1, 0x87, 0x1b, 0x9f, 0x2f, 0xc0, 0x46, 0xea, 0xbe, - 0x01, 0xdd, 0x84, 0x25, 0x79, 0xa2, 0x20, 0x7d, 0xda, 0xcc, 0x38, 0xc0, 0xdb, 0xc2, 0x4a, 0x0b, - 0xcc, 0xd9, 0xc6, 0x86, 0x60, 0x6a, 0x99, 0x23, 0xc3, 0xd6, 0x89, 0xeb, 0xfa, 0x81, 0x38, 0xde, - 0xca, 0x08, 0x38, 0x6d, 0xda, 0x69, 0xcb, 0x01, 0xac, 0xc6, 0x50, 0x53, 0x23, 0x4c, 0xee, 0xb8, - 0x22, 0x0c, 0xba, 0x04, 0x1b, 0x6c, 0x42, 0x59, 0x04, 0x53, 0x5d, 0x66, 0xfb, 0x62, 0xb6, 0xe7, - 0xcf, 0x2a, 0x9b, 0x45, 0x6d, 0x3d, 0x68, 0xbc, 0x1e, 0x6b, 0x6b, 0xb4, 0xe1, 0xcc, 0xb3, 0x76, - 0xe9, 0x19, 0x37, 0xa2, 0x8d, 0x4f, 0xd7, 0x60, 0x49, 0xba, 0x15, 0x19, 0x50, 0xf6, 0x62, 0xf9, - 0xb7, 0x32, 0x93, 0x2b, 0x25, 0x48, 0xb3, 0xeb, 0x27, 0x12, 0xee, 0x38, 0xa6, 0xfa, 0x79, 0x19, - 0x20, 0x4a, 0x63, 0xd0, 0x53, 0x08, 0x76, 0x53, 0xd8, 0xd4, 0xe5, 0xea, 0x14, 0x88, 0xe2, 0xd6, - 0xac, 0xc4, 0x21, 0x6c, 0x30, 0x11, 0xb0, 0xd9, 0x96, 0x90, 0x5a, 0xcd, 0x4b, 0x56, 0xa1, 0x0f, - 0x61, 0xd5, 0xe8, 0xf9, 0xd6, 0x01, 0x8e, 0x88, 0xc5, 0x74, 0xbb, 0x39, 0x3f, 0xf1, 0x16, 0x07, - 0x0c, 0x59, 0x57, 0x8c, 0xb1, 0x32, 0xb2, 0x00, 0x62, 0x0b, 0xae, 0x10, 0x50, 0x67, 0x7e, 0xb6, - 0xe4, 0x5a, 0x1b, 0x03, 0x47, 0x37, 0x20, 0xcf, 0x82, 0x8a, 0x5c, 0xd5, 0x2f, 0xcd, 0x48, 0xc2, - 0x66, 0xbe, 0xc6, 0x01, 0xd4, 0x3f, 0xe4, 0xa0, 0x78, 0x1b, 0x1b, 0x74, 0x44, 0xb0, 0x89, 0xbe, - 0xaf, 0xc0, 0xba, 0x48, 0x37, 0xa4, 0xcf, 0xf4, 0x9e, 0x3b, 0x12, 0x9f, 0x8c, 0xd1, 0xbc, 0x3b, - 0xff, 0x58, 0x02, 0x8a, 0x26, 0x0f, 0x22, 0xd2, 0x63, 0x2d, 0x0e, 0x2e, 0x06, 0x87, 0xac, 0x89, - 0x06, 0xf4, 0x89, 0x02, 0x1b, 0x32, 0x91, 0x49, 0xf4, 0x47, 0x84, 0x81, 0xf7, 0x8e, 0xa1, 0x3f, - 0x62, 0xed, 0x4f, 0xe9, 0xd0, 0x9a, 0x3b, 0xd9, 0x82, 0x36, 0xa1, 0xea, 0xbb, 0xbe, 0x61, 0x8b, - 0xe5, 0x94, 0x7a, 0x41, 0xf2, 0xa5, 0x68, 0x2b, 0xbc, 0x9e, 0xad, 0x97, 0x7b, 0xac, 0x56, 0x6d, - 0xc3, 0xa9, 0x29, 0x43, 0x4d, 0x49, 0x2c, 0xd6, 0xe3, 0x89, 0x45, 0x2e, 0x9e, 0xa9, 0x5e, 0x87, - 0xfa, 0xb4, 0x1e, 0xce, 0x84, 0x43, 0xa1, 0x36, 0x31, 0x6b, 0xd0, 0x07, 0x50, 0x1c, 0x4a, 0x3f, - 0xc8, 0x49, 0xb9, 0x7d, 0x74, 0x8f, 0x6a, 0x21, 0xa6, 0xfa, 0x49, 0x0e, 0x56, 0xc6, 0xa7, 0xcc, - 0xf3, 0xa6, 0x44, 0xaf, 0x00, 0xea, 0x13, 0x43, 0xc4, 0x44, 0x82, 0x87, 0x86, 0xe5, 0x58, 0xce, - 0x80, 0xbb, 0x43, 0xd1, 0x6a, 0x41, 0x8b, 0x16, 0x34, 0xa0, 0x9f, 0x2a, 0x70, 0x7a, 0x5c, 0x61, - 0x34, 0x66, 0x26, 0x66, 0x30, 0x3e, 0xae, 0x78, 0x31, 0xae, 0x35, 0x1a, 0xf6, 0x42, 0xe8, 0xed, - 0x94, 0x9b, 0xde, 0xaa, 0xbe, 0x03, 0x67, 0x9e, 0x65, 0x38, 0x93, 0x0c, 0xde, 0x84, 0xd5, 0x2f, - 0x4f, 0x73, 0xa7, 0x9b, 0xff, 0xae, 0x00, 0x79, 0x16, 0x3b, 0x90, 0x0e, 0x65, 0xb1, 0x46, 0xeb, - 0x8e, 0x31, 0x0c, 0x52, 0xa7, 0x6b, 0x73, 0x44, 0x21, 0x59, 0xb8, 0x63, 0x0c, 0xb1, 0x06, 0xc3, - 0xf0, 0x19, 0x61, 0xa8, 0xf0, 0xa9, 0x8e, 0x89, 0x6e, 0x1a, 0xbe, 0x11, 0x1c, 0x68, 0xbe, 0x35, - 0x0f, 0x45, 0x4b, 0x00, 0xed, 0x18, 0xbe, 0x71, 0xf3, 0x84, 0x56, 0xee, 0x45, 0x45, 0xe4, 0x43, - 0xcd, 0xb4, 0xa8, 0x4f, 0xac, 0x7d, 0x6e, 0x2a, 0xb8, 0x66, 0x3c, 0xcb, 0x1c, 0xe3, 0xda, 0x89, - 0xa1, 0x49, 0xc2, 0xaa, 0x99, 0xa8, 0x43, 0x3a, 0xc0, 0xc0, 0x18, 0x0d, 0xb0, 0xa0, 0xfb, 0x62, - 0xb6, 0x93, 0xc4, 0x31, 0xba, 0x1b, 0x0c, 0x46, 0xf2, 0x94, 0x06, 0x41, 0x41, 0xbd, 0x06, 0x10, - 0xf9, 0x15, 0x9d, 0x81, 0x12, 0xfb, 0x4a, 0xd4, 0x33, 0x7a, 0x58, 0x6e, 0x22, 0xa3, 0x0a, 0x84, - 0x20, 0xcf, 0xbf, 0x61, 0x8e, 0x37, 0xf0, 0x67, 0xf5, 0xdf, 0xd9, 0x26, 0x3c, 0xf2, 0x52, 0x28, - 0x08, 0x25, 0x26, 0x08, 0xf5, 0x03, 0xa8, 0x26, 0x47, 0xcb, 0xde, 0xe4, 0xee, 0x0d, 0xde, 0xe4, - 0x05, 0x26, 0x31, 0x3a, 0x1a, 0x4a, 0x39, 0xb1, 0x47, 0x56, 0x33, 0xb4, 0x1c, 0xce, 0x99, 0xd3, - 0xd8, 0x23, 0xaf, 0x31, 0x9e, 0xf0, 0x94, 0x88, 0xd5, 0x18, 0x4f, 0xd4, 0xf7, 0xa0, 0x14, 0x0e, - 0x2f, 0xbd, 0x0b, 0xe8, 0x0a, 0x94, 0xc2, 0xcb, 0xb0, 0x0c, 0x9b, 0xb2, 0xe8, 0x65, 0x96, 0xc5, - 0x32, 0xe7, 0xab, 0x87, 0x50, 0x4d, 0x66, 0x34, 0x29, 0x33, 0xe2, 0xee, 0xf8, 0xc6, 0xef, 0xea, - 0xdc, 0x11, 0x21, 0xbe, 0x2f, 0xfc, 0xc5, 0x02, 0xbc, 0xf0, 0xcc, 0x73, 0xf0, 0x63, 0x4c, 0xa4, - 0x9f, 0x6f, 0x82, 0xfb, 0x3e, 0x2c, 0x7b, 0xc4, 0x1a, 0x1a, 0xe4, 0x50, 0x66, 0xe9, 0x22, 0x2b, - 0x99, 0x7f, 0x1b, 0x5b, 0x91, 0x70, 0x3c, 0x3b, 0x6f, 0x7c, 0x3d, 0x0f, 0xa7, 0xa7, 0x5e, 0x1a, - 0x65, 0xbd, 0x91, 0x79, 0x0a, 0x2b, 0x26, 0xa6, 0x16, 0xc1, 0xa6, 0xb8, 0x33, 0x08, 0xc6, 0xbf, - 0x77, 0xd4, 0x5b, 0xab, 0xe6, 0x8e, 0x80, 0xe5, 0x75, 0x32, 0x77, 0x58, 0x36, 0xe3, 0x75, 0xea, - 0xaf, 0x14, 0xa8, 0xc4, 0xdf, 0x42, 0x17, 0x61, 0x23, 0x5c, 0xa5, 0xdc, 0xbe, 0x5c, 0x71, 0x4c, - 0x2c, 0xae, 0x53, 0x15, 0x6d, 0x2d, 0x68, 0xbc, 0xdb, 0xd7, 0x82, 0x26, 0x74, 0x1e, 0xd6, 0x0d, - 0xdb, 0x76, 0x1f, 0x07, 0x03, 0xd0, 0xc5, 0x35, 0x32, 0x1f, 0x46, 0x4e, 0x43, 0xb2, 0x8d, 0xe3, - 0x77, 0x79, 0x0b, 0xba, 0x02, 0x75, 0x4c, 0x7d, 0x6b, 0x68, 0xb0, 0xfd, 0xff, 0x58, 0x5a, 0x47, - 0xe5, 0x5c, 0x3c, 0x19, 0xb6, 0xc7, 0x73, 0x15, 0xaa, 0x7e, 0xa2, 0x00, 0x9a, 0x1c, 0x56, 0xca, - 0xc4, 0xe8, 0x8d, 0x4f, 0x8c, 0xdb, 0xc7, 0xea, 0xcc, 0xf8, 0x64, 0xf9, 0x5b, 0x0e, 0xd4, 0xe9, - 0xd7, 0x36, 0x93, 0x0a, 0x54, 0x8e, 0x53, 0x81, 0xff, 0xb4, 0x7d, 0xe8, 0x08, 0x56, 0x7a, 0x0f, - 0x0d, 0xc7, 0xc1, 0xf6, 0xb8, 0x48, 0xef, 0x1c, 0xf9, 0x62, 0xab, 0xd9, 0x12, 0xb8, 0xa2, 0x72, - 0xb9, 0x17, 0x2b, 0x51, 0xf5, 0xc7, 0x0a, 0x54, 0xe2, 0xed, 0x59, 0x0e, 0x26, 0xcf, 0xc3, 0xba, - 0x6d, 0x50, 0x5f, 0x0f, 0xdc, 0x1e, 0x1c, 0x45, 0x32, 0x21, 0x14, 0x34, 0xc4, 0xda, 0xba, 0xa2, - 0x49, 0xaa, 0x0a, 0x5d, 0x86, 0x93, 0x7d, 0x8b, 0x50, 0x5f, 0x0f, 0x5d, 0x19, 0x3f, 0xbe, 0x2c, - 0x68, 0xeb, 0xbc, 0x55, 0x93, 0x8d, 0xd2, 0xaa, 0x71, 0x0d, 0x36, 0x52, 0xaf, 0x6f, 0xb3, 0x6e, - 0x80, 0xeb, 0x70, 0x32, 0xfd, 0xee, 0xad, 0xf1, 0x99, 0x02, 0xc5, 0x30, 0x2f, 0xbd, 0x29, 0xd6, - 0x03, 0xa9, 0x9b, 0xcb, 0x19, 0xfd, 0x1d, 0x66, 0x76, 0x6c, 0x8d, 0xd2, 0xc4, 0x8a, 0x62, 0x42, - 0x9e, 0xaf, 0x58, 0x19, 0xe3, 0x52, 0xd2, 0xd5, 0x0b, 0x93, 0xae, 0x46, 0xb2, 0x6f, 0xe2, 0x94, - 0x97, 0x3f, 0x37, 0x7e, 0x94, 0x83, 0x0a, 0x3f, 0xbb, 0x09, 0xdc, 0x91, 0xbc, 0x6b, 0x9b, 0xa4, - 0x5f, 0x48, 0xa3, 0xdf, 0x85, 0x92, 0xb8, 0x45, 0x61, 0x13, 0x3b, 0xc7, 0x27, 0xf1, 0xb9, 0x8c, - 0x83, 0xe7, 0xf4, 0xb7, 0xf0, 0xa1, 0x56, 0xa4, 0xf2, 0x09, 0xdd, 0x82, 0xdc, 0x00, 0xfb, 0xb3, - 0xfe, 0x5a, 0xc1, 0x81, 0x6e, 0xe0, 0xd8, 0x6f, 0x00, 0x0c, 0x05, 0xdd, 0x83, 0x45, 0xc3, 0xf3, - 0xb0, 0x63, 0x06, 0xc9, 0xdf, 0xd5, 0x59, 0xf0, 0xb6, 0xb8, 0x69, 0x04, 0x29, 0xb1, 0xd0, 0x7f, - 0x43, 0xa1, 0x67, 0x63, 0x83, 0x04, 0x59, 0xde, 0x95, 0x59, 0x40, 0x5b, 0xcc, 0x32, 0xc2, 0x14, - 0x48, 0xf1, 0xdf, 0x06, 0x3e, 0x5b, 0x80, 0x65, 0xf9, 0x59, 0x64, 0x64, 0x4a, 0x7e, 0x97, 0xf4, - 0x3f, 0x03, 0x76, 0xc7, 0x1c, 0xf7, 0xda, 0xcc, 0x8e, 0x0b, 0xaf, 0x93, 0xb9, 0xe7, 0xee, 0x27, - 0x3d, 0xf7, 0xfa, 0x3c, 0x9e, 0x0b, 0x31, 0x03, 0xd7, 0x69, 0x09, 0xd7, 0x5d, 0x9d, 0xc3, 0x75, - 0x21, 0xa8, 0xf4, 0x5d, 0xfc, 0xba, 0xfb, 0x37, 0x45, 0x28, 0x06, 0xa2, 0x42, 0x5d, 0x58, 0x14, - 0x3f, 0x4f, 0xc9, 0xd4, 0xe7, 0xd5, 0x19, 0x55, 0xd9, 0xd4, 0xb8, 0x35, 0xeb, 0xbe, 0xc0, 0x41, - 0x14, 0xd6, 0x86, 0x23, 0x9b, 0xad, 0x77, 0x9e, 0x3e, 0x71, 0x06, 0xbb, 0x35, 0x2b, 0xfc, 0x6d, - 0x09, 0x15, 0x3f, 0x74, 0xad, 0x0d, 0x93, 0x95, 0xc8, 0x84, 0x95, 0x7d, 0x63, 0xa0, 0xc7, 0x8e, - 0x99, 0x73, 0x33, 0xfd, 0x99, 0x11, 0xf2, 0x6d, 0x1b, 0x83, 0xf8, 0x91, 0x72, 0x65, 0x3f, 0x56, - 0x66, 0x43, 0xb3, 0x7c, 0x4c, 0x8c, 0x7d, 0x1b, 0xc7, 0x87, 0x96, 0x9f, 0x6f, 0x68, 0x1d, 0x09, - 0x35, 0x36, 0x34, 0x2b, 0x59, 0x89, 0xbe, 0xa6, 0x40, 0x3d, 0x74, 0xe8, 0x23, 0x7c, 0x48, 0xe3, - 0xd4, 0x05, 0x4e, 0xdd, 0x9e, 0xd7, 0xab, 0xb7, 0xf0, 0x21, 0x8d, 0xd3, 0x6f, 0x0c, 0xd3, 0x1a, - 0x54, 0x15, 0x16, 0xc5, 0x67, 0x8e, 0xa7, 0x26, 0x15, 0x9e, 0x9a, 0xa8, 0x04, 0x6a, 0x13, 0x03, - 0xc9, 0xb2, 0xb0, 0x35, 0x60, 0x39, 0x1a, 0x47, 0x2c, 0x22, 0x87, 0x27, 0xe0, 0x1d, 0x13, 0x9d, - 0x84, 0x45, 0x71, 0xd9, 0x2e, 0x63, 0xb2, 0x2c, 0xa9, 0xdf, 0x52, 0xa0, 0x36, 0x21, 0x8c, 0xe7, - 0x4c, 0x1a, 0x0c, 0x3d, 0x1f, 0x0d, 0xfd, 0x00, 0x36, 0x52, 0x1d, 0xf9, 0xbc, 0x87, 0xff, 0x31, - 0x54, 0xe2, 0x32, 0xcd, 0x48, 0x17, 0xcd, 0x8d, 0x18, 0x5d, 0x78, 0x63, 0x32, 0xcb, 0xc0, 0xc3, - 0xbb, 0x89, 0xb7, 0x61, 0x35, 0xb1, 0xa8, 0xa0, 0x57, 0x00, 0xf5, 0x5c, 0xc7, 0xb7, 0x9c, 0x11, - 0xcf, 0xbc, 0xc4, 0x65, 0x93, 0xd4, 0x4b, 0x2d, 0xde, 0xc2, 0xef, 0x52, 0x1a, 0xf7, 0xa1, 0x9a, - 0x8c, 0xae, 0x33, 0x42, 0x84, 0xcb, 0xf6, 0x42, 0x6c, 0xd9, 0xde, 0x04, 0x34, 0xb9, 0x3a, 0x85, - 0x6f, 0x2a, 0xb1, 0x37, 0x37, 0x60, 0x2d, 0x25, 0x1a, 0x37, 0xd6, 0xa0, 0x36, 0xb1, 0x12, 0x35, - 0xd6, 0x25, 0xea, 0x58, 0x8c, 0x6d, 0xfc, 0x2c, 0x0f, 0xc5, 0x5d, 0x57, 0x1e, 0x12, 0xfd, 0x1f, - 0x14, 0x29, 0x3e, 0xc0, 0xc4, 0xf2, 0xc5, 0x24, 0x59, 0xc9, 0x7c, 0xde, 0x10, 0x40, 0x34, 0xf7, - 0xa4, 0xbd, 0xb8, 0x9f, 0x0d, 0xe1, 0xe6, 0xdf, 0x84, 0xa3, 0x3a, 0xdb, 0xdf, 0x52, 0x6a, 0x0c, - 0x82, 0xd3, 0x87, 0xa0, 0xc8, 0xef, 0xac, 0x88, 0xd1, 0xc3, 0xfc, 0xe3, 0x96, 0x34, 0x51, 0x48, - 0xc9, 0x69, 0x0a, 0x59, 0x52, 0xaa, 0xc5, 0x49, 0xd9, 0xbd, 0x08, 0x15, 0xdb, 0x1d, 0xe8, 0xb6, - 0x2b, 0xef, 0x5d, 0x97, 0xc4, 0x2b, 0xb6, 0x3b, 0xd8, 0x95, 0x55, 0x4c, 0x75, 0xfe, 0x43, 0x82, - 0x0d, 0xb3, 0x5e, 0xe4, 0x8d, 0xb2, 0xa4, 0xfe, 0x2f, 0xe4, 0x77, 0x2d, 0xea, 0xa3, 0x2e, 0xb0, - 0xd7, 0x75, 0xec, 0xf8, 0xc4, 0xc2, 0xc1, 0x86, 0xe3, 0xdc, 0x8c, 0x4e, 0xd5, 0xc0, 0x16, 0x4f, - 0x16, 0xa6, 0x2a, 0x81, 0x62, 0xe0, 0xe3, 0x46, 0x1f, 0xf2, 0xcc, 0xcd, 0x68, 0x15, 0xca, 0xf7, - 0xef, 0xec, 0x75, 0xdb, 0xad, 0xce, 0xf5, 0x4e, 0x7b, 0xa7, 0x7a, 0x02, 0x95, 0xa0, 0x70, 0x4f, - 0xdb, 0x6a, 0xb5, 0xab, 0x0a, 0x7b, 0xdc, 0x69, 0x6f, 0xdf, 0xbf, 0x51, 0x5d, 0x40, 0x45, 0xc8, - 0x77, 0xee, 0x5c, 0xbf, 0x5b, 0xcd, 0x21, 0x80, 0xc5, 0x3b, 0x77, 0xef, 0x75, 0x5a, 0xed, 0x6a, - 0x9e, 0xd5, 0x3e, 0xd8, 0xd2, 0xee, 0x54, 0x0b, 0xec, 0xd5, 0xb6, 0xa6, 0xdd, 0xd5, 0xaa, 0x8b, - 0xa8, 0x02, 0xc5, 0x96, 0xd6, 0xb9, 0xd7, 0x69, 0x6d, 0xed, 0x56, 0x97, 0x1a, 0x15, 0x80, 0x5d, - 0x77, 0xd0, 0x72, 0x1d, 0x9f, 0xb8, 0x76, 0xe3, 0xf7, 0x79, 0xae, 0x24, 0xe2, 0x3f, 0x70, 0xc9, - 0xa3, 0xe8, 0x9f, 0xb3, 0x7f, 0x81, 0xd2, 0x63, 0x5e, 0x11, 0x4d, 0xe2, 0xa2, 0xa8, 0xe8, 0x98, - 0x68, 0x1f, 0xaa, 0x3d, 0x61, 0xae, 0x07, 0xff, 0x36, 0x4b, 0x15, 0xcc, 0xfd, 0xef, 0xcd, 0xaa, - 0x04, 0x6c, 0x4b, 0x3c, 0xc6, 0x61, 0xbb, 0x83, 0x81, 0xe5, 0x0c, 0x22, 0x8e, 0xdc, 0x11, 0x39, - 0x24, 0x60, 0xc8, 0x61, 0x42, 0xcd, 0x20, 0xbe, 0xd5, 0x37, 0x7a, 0x7e, 0x44, 0x92, 0x3f, 0x1a, - 0x49, 0x35, 0x40, 0x0c, 0x59, 0xfa, 0xfc, 0x46, 0xec, 0xc0, 0xa2, 0x4c, 0xc0, 0x21, 0x4d, 0xe1, - 0x68, 0x34, 0xb5, 0x10, 0x32, 0xe4, 0x79, 0x1f, 0x16, 0x3d, 0x83, 0x18, 0x43, 0x5a, 0x07, 0x2e, - 0xcc, 0x19, 0x56, 0xe2, 0xc4, 0xd7, 0x6f, 0x76, 0x39, 0x8e, 0xfc, 0xe5, 0x4b, 0x80, 0xaa, 0x57, - 0xa1, 0x1c, 0xab, 0xfe, 0xb2, 0x33, 0xe4, 0x52, 0x7c, 0x27, 0xff, 0x9f, 0x3c, 0xb0, 0x45, 0x24, - 0x32, 0xb8, 0x86, 0x79, 0xb1, 0x12, 0xcb, 0x8b, 0x1b, 0xe7, 0x59, 0xb8, 0x73, 0xbd, 0xec, 0x72, - 0x6c, 0xbc, 0xcc, 0x14, 0x1c, 0x59, 0x3c, 0x0b, 0xfd, 0xe2, 0xa7, 0x0a, 0x2c, 0x6f, 0x63, 0x63, - 0x78, 0xdd, 0x91, 0x13, 0x00, 0x7d, 0x5b, 0x81, 0xa5, 0xe0, 0x39, 0x6b, 0xd2, 0x9c, 0xf2, 0x9b, - 0xb0, 0x7a, 0x75, 0x1e, 0x5b, 0x11, 0xcc, 0x4f, 0x6c, 0x2a, 0xe7, 0x95, 0x8b, 0x1f, 0x01, 0x88, - 0x9e, 0xf1, 0xbd, 0xa4, 0x23, 0xf7, 0x94, 0xe7, 0x66, 0xdc, 0x97, 0xaa, 0xb3, 0x1a, 0x48, 0xf6, - 0xef, 0x28, 0x50, 0x16, 0xf4, 0x62, 0x21, 0x7f, 0x02, 0x05, 0xf1, 0x70, 0x69, 0x96, 0x84, 0x4e, - 0x8e, 0x48, 0xbd, 0x3c, 0x9b, 0x91, 0x5c, 0xbe, 0x44, 0x4f, 0xbe, 0x1b, 0x7e, 0xa2, 0x5d, 0x31, - 0x5f, 0xd1, 0x13, 0x58, 0x0a, 0x1e, 0x2f, 0xcf, 0xba, 0x84, 0xb1, 0xc0, 0xad, 0x5e, 0xc8, 0x6e, - 0x15, 0xc4, 0x45, 0xd1, 0x97, 0x5f, 0x2e, 0x40, 0x5d, 0xf4, 0xa5, 0xfd, 0xc4, 0xc7, 0xc4, 0x31, - 0x6c, 0xa1, 0xb2, 0xae, 0x2b, 0x94, 0x53, 0x8e, 0xe9, 0x1a, 0x5d, 0x9d, 0x7b, 0xc2, 0xa9, 0xaf, - 0xcf, 0x63, 0x1a, 0x78, 0x0d, 0x7d, 0x53, 0x01, 0x88, 0x66, 0x00, 0xca, 0xbe, 0xbf, 0x4d, 0x4c, - 0x33, 0xf5, 0xea, 0x1c, 0x96, 0x41, 0x2f, 0xb6, 0xb7, 0xe0, 0x3f, 0xa6, 0x59, 0xc7, 0x8d, 0xb7, - 0x4b, 0xc2, 0xa1, 0x5b, 0x9e, 0xf5, 0xee, 0x4a, 0xac, 0x49, 0x3f, 0xb8, 0xb0, 0xbf, 0xc8, 0x93, - 0x87, 0x4b, 0xff, 0x08, 0x00, 0x00, 0xff, 0xff, 0xf1, 0x22, 0xc0, 0xa7, 0x84, 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/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go index 9ffe1197b905..7959fba9419c 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 @@ -483,12 +483,12 @@ const ( 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 assign the parameterized timestamp, windows and PaneInfo to 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 beam:coder:bytes:v1 - // elements coder and the window coder that this param_windowed_value uses. + // 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. // @@ -3973,8 +3973,8 @@ type WireCoderSetting struct { // 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 - // beam:coder:bytes:v1 elements coder and the window coder that this - // param_windowed_value uses. + // 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:"-"` From 0851849dcfe9b8088d69a5bdd3b55f7806b83ea9 Mon Sep 17 00:00:00 2001 From: Lukasz Cwik Date: Thu, 2 Jan 2020 17:48:27 -0800 Subject: [PATCH 36/67] [BEAM-5605] Ensure that split calls are routed to the active bundle processor for the bundle id. (#10489) * [BEAM-5605] Ensure that split calls are routed to the active bundle processor for the bundle id. Maintain a mapping from bundle id to active bundle processor. * fix test, clarify comments * Address comments --- .../org/apache/beam/fn/harness/FnHarness.java | 3 + .../harness/control/ProcessBundleHandler.java | 73 ++++++++++++++++--- .../control/ProcessBundleHandlerTest.java | 27 ++++++- 3 files changed, 89 insertions(+), 14 deletions(-) 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 1aa5ba58ad2d..6ec167353ce2 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 @@ public static void main( 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/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 11222859c495..a258e01d47ba 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. * - *

Finally executes the DAG based graph by starting all runners in reverse topological order, and - * finishing all runners in forward topological order. + *

{@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. + * + *

{@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 breaking the + * fusion barrier for further details. */ public class ProcessBundleHandler { @@ -231,6 +241,7 @@ public BeamFnApi.InstructionResponse.Builder processBundle(BeamFnApi.Instruction BundleProcessor bundleProcessor = bundleProcessorCache.get( request.getProcessBundle().getProcessBundleDescriptorId(), + request.getInstructionId(), () -> { try { return createBundleProcessor( @@ -240,7 +251,6 @@ public BeamFnApi.InstructionResponse.Builder processBundle(BeamFnApi.Instruction throw new RuntimeException(e); } }); - bundleProcessor.setInstructionId(request.getInstructionId()); PTransformFunctionRegistry startFunctionRegistry = bundleProcessor.getStartFunctionRegistry(); PTransformFunctionRegistry finishFunctionRegistry = bundleProcessor.getFinishFunctionRegistry(); Multimap allResiduals = bundleProcessor.getAllResiduals(); @@ -294,6 +304,19 @@ public BeamFnApi.InstructionResponse.Builder processBundle(BeamFnApi.Instruction 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 @@ private BundleProcessor createBundleProcessor( public static class BundleProcessorCache { private final Map> cachedBundleProcessors; + private final Map 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> getCachedBundleProcessors() { @@ -417,26 +449,43 @@ Map> getCachedBundleProcessors() /** * 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. + * + *

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 bundleProcessorSupplier) { + String bundleDescriptorId, + String instructionId, + Supplier bundleProcessorSupplier) { ConcurrentLinkedQueue 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/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 58d8b776321b..a7a233da11ac 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 @@ private static class TestBundleProcessorCache extends BundleProcessorCache { @Override BundleProcessor get( - String bundleDescriptorId, Supplier bundleProcessorSupplier) { - return new TestBundleProcessor(super.get(bundleDescriptorId, bundleProcessorSupplier)); + String bundleDescriptorId, + String instructionId, + Supplier bundleProcessorSupplier) { + return new TestBundleProcessor( + super.get(bundleDescriptorId, instructionId, bundleProcessorSupplier)); } } @@ -482,6 +487,24 @@ public void testBundleProcessorIsResetWhenAddedBackToCache() throws Exception { handler.bundleProcessorCache.getCachedBundleProcessors().get("1L").size(), equalTo(1)); } + @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); From 8653a7685c47a358bfe3860b58746613ddc7d867 Mon Sep 17 00:00:00 2001 From: Tomo Suzuki Date: Fri, 3 Jan 2020 14:32:53 -0500 Subject: [PATCH 37/67] protobuf 3.11.1 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f3ba610ce3b9..0d28c6ba4291 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 @@ class BeamModulePlugin implements Plugin { 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" From 4996c14811dc5d4e6104952e9937893b378cd24f Mon Sep 17 00:00:00 2001 From: Kyle Weaver Date: Fri, 3 Jan 2020 12:49:16 -0800 Subject: [PATCH 38/67] Make Spark REST URL a separate pipeline option. --- sdks/python/apache_beam/options/pipeline_options.py | 10 ++++++---- .../apache_beam/runners/portability/spark_runner.py | 8 ++++---- .../runners/portability/spark_uber_jar_job_server.py | 12 ++++++------ 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 1b4e822b1826..278e5f31805e 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1005,10 +1005,12 @@ def _add_argparse_args(cls, parser): parser.add_argument('--spark_submit_uber_jar', default=False, action='store_true', - help='Create and upload an uber jar to the Spark master' - ' directly, rather than starting up a job server.' - ' Only applies when Spark master is set to a' - ' cluster address. Requires Python 3.6+.') + 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): diff --git a/sdks/python/apache_beam/runners/portability/spark_runner.py b/sdks/python/apache_beam/runners/portability/spark_runner.py index c292ea9b4ecd..ce33acde0c82 100644 --- a/sdks/python/apache_beam/runners/portability/spark_runner.py +++ b/sdks/python/apache_beam/runners/portability/spark_runner.py @@ -44,15 +44,15 @@ def run_pipeline(self, pipeline, options): def default_job_server(self, options): spark_options = options.view_as(pipeline_options.SparkRunnerOptions) - if (spark_options.spark_submit_uber_jar - and not re.match(LOCAL_MASTER_PATTERN, spark_options.spark_master_url)): + 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( - re.sub('^spark://', 'http://', spark_options.spark_master_url), - options) + spark_options.spark_rest_url, options) return job_server.StopOnExitJobServer(SparkJarJobServer(options)) 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 index 7bad8db1a443..88f7cb7a2ecb 100644 --- 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 @@ -44,9 +44,9 @@ class SparkUberJarJobServer(abstract_job_service.AbstractJobServiceServicer): the pipeline artifacts. """ - def __init__(self, master_url, options): + def __init__(self, rest_url, options): super(SparkUberJarJobServer, self).__init__() - self._master_url = master_url + 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) @@ -67,7 +67,7 @@ def executable_jar(self): def create_beam_job(self, job_id, job_name, pipeline, options): return SparkBeamJob( - self._master_url, + self._rest_url, self.executable_jar(), job_id, job_name, @@ -84,17 +84,17 @@ class SparkBeamJob(abstract_job_service.UberJarBeamJob): setting the configuration property spark.master.rest.enabled to true.""" def __init__( - self, master_url, executable_jar, job_id, job_name, pipeline, options, + 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._master_url = master_url + 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._master_url, path) + 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" % From d7ef497b2360ea68f2bad9fb695a539b12cb1ddd Mon Sep 17 00:00:00 2001 From: kirillkozlov Date: Fri, 3 Jan 2020 14:27:32 -0800 Subject: [PATCH 39/67] Address comments --- .../bigquery/BeamSqlUnparseContext.java | 19 ++++++++----------- .../meta/provider/bigquery/BigQueryTable.java | 3 +-- .../sql/zetasql/BeamZetaSqlCalcRel.java | 2 +- .../extensions/sql/zetasql/ZetaSqlUtils.java | 2 ++ 4 files changed, 12 insertions(+), 14 deletions(-) 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 index ebdacf78ff23..432b3b5db282 100644 --- 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 @@ -24,7 +24,6 @@ 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.SqlDialect; 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; @@ -37,8 +36,8 @@ public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext { - public BeamSqlUnparseContext(SqlDialect dialect, IntFunction field) { - super(dialect, field); + public BeamSqlUnparseContext(IntFunction field) { + super(BeamBigQuerySqlDialect.DEFAULT, field); } @Override @@ -46,14 +45,12 @@ public SqlNode toSql(RexProgram program, RexNode rex) { if (rex.getKind().equals(SqlKind.LITERAL)) { final RexLiteral literal = (RexLiteral) rex; SqlTypeFamily family = literal.getTypeName().getFamily(); - if (family != null) { - if (family.equals(SqlTypeFamily.BINARY)) { - BitString bitString = BitString.createFromBytes(literal.getValueAs(byte[].class)); - return new SqlByteStringLiteral(bitString, POS); - } else if (family.equals(SqlTypeFamily.CHARACTER)) { - String escaped = StringEscapeUtils.escapeJava(literal.getValueAs(String.class)); - return SqlLiteral.createCharString(escaped, POS); - } + 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); } } 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 789d8ec1b73b..71f42359bfdf 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 @@ private String generateRowRestrictions(Schema schema, List supported) { // 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/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 index 8cc8d3ed2daa..330fb2df3550 100644 --- 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 @@ -74,7 +74,7 @@ public BeamZetaSqlCalcRel( i -> new SqlIdentifier( getProgram().getInputRowType().getFieldList().get(i).getName(), SqlParserPos.ZERO); - context = new BeamSqlUnparseContext(DIALECT, fn); + context = new BeamSqlUnparseContext(fn); } @Override 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 index 391041a7edb8..d771857e1ed2 100644 --- 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 @@ -158,6 +158,8 @@ public static Object zetaSqlValueToJavaObject(Value value, FieldType fieldType) 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(); } From 16757ef9a6da4d0ac218c6c4d6b19e2a49ccca45 Mon Sep 17 00:00:00 2001 From: Ning Kang Date: Fri, 3 Jan 2020 14:53:24 -0800 Subject: [PATCH 40/67] [BEAM-8335] On Unbounded Source change (#10442) [BEAM-8335] On Unbounded Source change (#10442) 1. Evict all cache when unbounded source changes in a pipeline executed by the InteractiveRunner; 2. Invalidate existing background caching job on unbounded source change; 3. Detects unbounded source addition/deletion/mutation through signature generated from `to_runner_api`; 4. The change detection and following routines are carried out at pipeline execution time. --- .../interactive/background_caching_job.py | 54 ++++++- .../background_caching_job_test.py | 144 ++++++++++++++++++ .../display/pipeline_graph_test.py | 12 +- .../interactive/interactive_environment.py | 7 + .../interactive/testing/mock_ipython.py | 24 +-- 5 files changed, 221 insertions(+), 20 deletions(-) diff --git a/sdks/python/apache_beam/runners/interactive/background_caching_job.py b/sdks/python/apache_beam/runners/interactive/background_caching_job.py index 5084cf69d0dd..f157f9b0720c 100644 --- a/sdks/python/apache_beam/runners/interactive/background_caching_job.py +++ b/sdks/python/apache_beam/runners/interactive/background_caching_job.py @@ -51,6 +51,8 @@ def attempt_to_run_background_caching_job(runner, user_pipeline, options=None): 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 @@ -75,14 +77,16 @@ def is_background_caching_job_needed(user_pipeline): 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 valid previous job. + # 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))) + 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): @@ -105,3 +109,49 @@ def attempt_to_cancel_background_caching_job(user_pipeline): 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 index 64f8fce66a04..19e4889d1e0c 100644 --- a/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py +++ b/sdks/python/apache_beam/runners/interactive/background_caching_job_test.py @@ -21,10 +21,13 @@ 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 @@ -35,6 +38,9 @@ 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() @@ -48,6 +54,19 @@ def _build_a_test_stream_pipeline(): 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): @@ -99,6 +118,131 @@ def test_background_caching_job_not_start_when_such_job_is_done(self): 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/pipeline_graph_test.py b/sdks/python/apache_beam/runners/interactive/display/pipeline_graph_test.py index e73dbd61727c..7e4aa13a01a4 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 @@ def test_get_dot(self): '}\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 b70551553c8e..31bb736c3795 100644 --- a/sdks/python/apache_beam/runners/interactive/interactive_environment.py +++ b/sdks/python/apache_beam/runners/interactive/interactive_environment.py @@ -89,6 +89,7 @@ def __init__(self, cache_manager=None): # 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__') @@ -240,6 +241,12 @@ def is_terminated(self, pipeline, is_main_job=True): 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/testing/mock_ipython.py b/sdks/python/apache_beam/runners/interactive/testing/mock_ipython.py index ee3acd89f0f6..fc11e4d6b678 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 @@ def mock_get_ipython(): 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) + + # 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: - with mock_get_ipython(): + # 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 @@ class MockedGetIpython(object): 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 @@ def __exit__(self, exc_type, exc_value, traceback): """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() From 952c60873384158038c3625eb89b9a6d3a1d73cb Mon Sep 17 00:00:00 2001 From: Sam sam Date: Fri, 3 Jan 2020 16:31:41 -0800 Subject: [PATCH 41/67] [BEAM-9013] TestStream fix for DataflowRunner (#10445) [BEAM-9013] TestStream fix for DataflowRunner (#10445) * Change fix to instead move the TestStream and WatermarkController implementation to the runner --- .../runners/dataflow/dataflow_runner.py | 2 +- .../runners/direct/direct_runner.py | 45 +++- .../runners/direct/test_stream_impl.py | 173 +++++++++++++ .../runners/direct/transform_evaluator.py | 4 +- .../python/apache_beam/testing/test_stream.py | 174 +------------ .../testing/test_stream_it_test.py | 237 ++++++++++++++++++ 6 files changed, 462 insertions(+), 173 deletions(-) create mode 100644 sdks/python/apache_beam/runners/direct/test_stream_impl.py create mode 100644 sdks/python/apache_beam/testing/test_stream_it_test.py diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 02543664b148..678676043bff 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 @@ def run_TestStream(self, transform_node, options): # 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/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 4a6634149748..ef81a2786257 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 @@ class SwitchingDirectRunner(PipelineRunner): 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 @@ def run_pipeline(self, pipeline, options): 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 000000000000..aa1405d199bf --- /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 0d7ddaa98431..ff73aab674ca 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/testing/test_stream.py b/sdks/python/apache_beam/testing/test_stream.py index 8429f20235c9..ff54fe2d5715 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 @@ class TestStream(PTransform): 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 @@ def from_runner_api_parameter(payload, context): 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 000000000000..8e724f654fff --- /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() From e2bb239f0418f1c4949227ba3f51a5f4eb7235df Mon Sep 17 00:00:00 2001 From: liumomo315 <56744666+liumomo315@users.noreply.github.com> Date: Fri, 3 Jan 2020 16:34:12 -0800 Subject: [PATCH 42/67] [BEAM-8575] Refactor test_do_fn_with_windowing_in_finish_bundle to work with Dataflow runner (#10447) * [BEAM-8575] Refactor test_do_fn_with_windowing_in_finish_bundle to work with Dataflow runner * Update sdks/python/apache_beam/transforms/ptransform_test.py Co-Authored-By: Yichi Zhang Co-authored-by: Yichi Zhang --- sdks/python/apache_beam/transforms/ptransform_test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index fbf797d8537a..fac509efe7fa 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -347,14 +347,14 @@ def finish_bundle(self): 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)) From aaaf6ea2eca84c945de416979da8d7b328b22bd5 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Mon, 6 Jan 2020 14:38:47 +0800 Subject: [PATCH 43/67] [BEAM-9055] Unify the config names of Fn Data API across languages. --- .../BeamFnDataBufferingOutboundObserver.java | 22 ++++++++++++++----- ...izeBasedBufferingOutboundObserverTest.java | 2 +- ...imeBasedBufferingOutboundObserverTest.java | 4 ++-- .../data/BeamFnDataGrpcClientTest.java | 2 +- 4 files changed, 21 insertions(+), 9 deletions(-) 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 72ab5d683f1e..bbc2916a2622 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. * *

The default size-based buffer threshold can be overridden by specifying the experiment {@code - * beam_fn_api_data_buffer_size_limit=} + * data_buffer_size_limit=} * *

The default time-based buffer threshold can be overridden by specifying the experiment {@code - * beam_fn_api_data_buffer_time_limit=} + * data_buffer_time_limit_ms=} */ public interface BeamFnDataBufferingOutboundObserver extends CloseableFnDataReceiver { // 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 BeamFnDataSizeBasedBufferingOutboundObserver forLocation( @@ -68,6 +74,9 @@ static BeamFnDataSizeBasedBufferingOutboundObserver forLocation( static int getSizeLimit(PipelineOptions options) { List experiments = options.as(ExperimentalOptions.class).getExperiments(); for (String experiment : experiments == null ? Collections.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 int getSizeLimit(PipelineOptions options) { static long getTimeLimit(PipelineOptions options) { List experiments = options.as(ExperimentalOptions.class).getExperiments(); for (String experiment : experiments == null ? Collections.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/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 0e53b26c3411..ed2f700d7c29 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 @@ public void testConfiguredBufferLimit() throws Exception { 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> 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 f4effa846147..eaf6290efdb5 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 @@ public void testConfiguredTimeLimit() throws Exception { 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> consumer = BeamFnDataBufferingOutboundObserver.forLocation( @@ -80,7 +80,7 @@ public void testConfiguredTimeLimitExceptionPropagation() throws Exception { 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> consumer = (BeamFnDataTimeBasedBufferingOutboundObserver>) BeamFnDataBufferingOutboundObserver.forLocation( 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 672d41befaa4..deb6218cc3ee 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 @@ public StreamObserver data( 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()); From 15a7ef16419308f14258c64c6381a0e734a9404c Mon Sep 17 00:00:00 2001 From: Rehman Date: Mon, 6 Jan 2020 17:08:36 +0500 Subject: [PATCH 44/67] onTimer/setTimer signature updates --- .../operators/ApexParDoOperator.java | 7 +++++- .../apache/beam/runners/core/DoFnRunner.java | 8 +++++- .../core/LateDataDroppingDoFnRunner.java | 9 +++++-- .../beam/runners/core/ProcessFnRunner.java | 7 +++++- .../core/PushbackSideInputDoFnRunner.java | 8 +++++- .../beam/runners/core/SimpleDoFnRunner.java | 7 +++++- .../SimplePushbackSideInputDoFnRunner.java | 9 +++++-- .../beam/runners/core/StatefulDoFnRunner.java | 9 +++++-- .../runners/core/SimpleDoFnRunnerTest.java | 11 +++++++- ...SimplePushbackSideInputDoFnRunnerTest.java | 25 ++++++++++++++++--- .../runners/core/StatefulDoFnRunnerTest.java | 8 +++++- .../beam/runners/direct/ParDoEvaluator.java | 8 +++++- .../metrics/DoFnRunnerWithMetricsUpdate.java | 4 ++- .../functions/FlinkStatefulDoFnFunction.java | 8 +++++- .../wrappers/streaming/DoFnOperator.java | 7 +++++- .../ExecutableStageDoFnOperator.java | 7 +++++- .../stableinput/BufferedElements.java | 18 +++++++++++-- .../stableinput/BufferingDoFnRunner.java | 10 ++++++-- .../stableinput/BufferedElementsTest.java | 7 +++++- .../worker/DataflowProcessFnRunner.java | 7 +++++- .../worker/GroupAlsoByWindowFnRunner.java | 7 +++++- .../dataflow/worker/SimpleParDoFn.java | 8 +++++- ...amingKeyedWorkItemSideInputDoFnRunner.java | 7 +++++- .../worker/StreamingSideInputDoFnRunner.java | 7 +++++- .../jet/processors/StatefulParDoP.java | 8 +++++- .../samza/metrics/DoFnRunnerWithMetrics.java | 12 +++++++-- .../beam/runners/samza/runtime/DoFnOp.java | 8 +++++- .../runtime/DoFnRunnerWithKeyedInternals.java | 17 ++++++++++--- .../samza/runtime/SamzaDoFnRunners.java | 7 +++++- .../batch/DoFnRunnerWithMetrics.java | 4 ++- .../translation/DoFnRunnerWithMetrics.java | 4 ++- .../translation/SparkProcessContext.java | 8 +++++- 32 files changed, 239 insertions(+), 42 deletions(-) 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 79bb6efce2e0..4841c6a558bb 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 @@ -384,7 +384,12 @@ public void fireTimer(Object key, Collection timerDataSet) { checkArgument(namespace instanceof WindowNamespace); BoundedWindow window = ((WindowNamespace) namespace).getWindow(); pushbackDoFnRunner.onTimer( - timerData.getTimerId(), window, timerData.getTimestamp(), timerData.getDomain()); + timerData.getTimerId(), + timerData.getTimerFamilyId(), + window, + timerData.getTimestamp(), + timerData.getOutputTimestamp(), + timerData.getDomain()); } pushbackDoFnRunner.finishBundle(); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index cf2843666c08..e2fc262aff6b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -38,7 +38,13 @@ public interface DoFnRunner { * Calls a {@link DoFn DoFn's} {@link DoFn.OnTimer @OnTimer} method for the given timer in the * given window. */ - void onTimer(String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain); + void onTimer( + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain); /** * Calls a {@link DoFn DoFn's} {@link DoFn.FinishBundle @FinishBundle} method and performs diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 4865e82be9e2..8f19b5fc4d3e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -82,8 +82,13 @@ public void processElement(WindowedValue> elem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { - doFnRunner.onTimer(timerId, window, timestamp, timeDomain); + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { + doFnRunner.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java index d65b5f4d5a5c..c310c49e7269 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java @@ -83,7 +83,12 @@ public void finishBundle() { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { throw new UnsupportedOperationException("User timers unsupported in ProcessFn"); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index cc2e86aff18e..32a61aff2a2d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -43,7 +43,13 @@ public interface PushbackSideInputDoFnRunner { Iterable> processElementInReadyWindows(WindowedValue elem); /** Calls the underlying {@link DoFn.OnTimer} method. */ - void onTimer(String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain); + void onTimer( + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain); /** Calls the underlying {@link DoFn.FinishBundle} method. */ void finishBundle(); 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 472a9d26072d..68f780d0f7d7 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 @@ -186,7 +186,12 @@ public void processElement(WindowedValue compressedElem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { // The effective timestamp is when derived elements will have their timestamp set, if not // otherwise specified. If this is an event time timer, then they have the timestamp of the diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java index 36a89fe44785..b27e046ea327 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java @@ -108,8 +108,13 @@ private boolean isReady(BoundedWindow mainInputWindow) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { - underlying.onTimer(timerId, window, timestamp, timeDomain); + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { + underlying.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } @Override 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 f69c74a90905..670f3a50a27a 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 @@ -117,7 +117,12 @@ private boolean isLate(BoundedWindow window) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { if (cleanupTimer.isForWindow(timerId, window, timestamp, timeDomain)) { stateCleaner.clearForWindow(window); // There should invoke the onWindowExpiration of DoFn @@ -134,7 +139,7 @@ public void onTimer( window, cleanupTimer.currentInputWatermarkTime()); } else { - doFnRunner.onTimer(timerId, window, timestamp, timeDomain); + doFnRunner.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } } } 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 10972d6396db..90bb5aa2de55 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 @@ -119,7 +119,12 @@ public void testOnTimerExceptionsWrappedAsUserCodeException() { thrown.expectCause(is(fn.exceptionToThrow)); runner.onTimer( - ThrowingDoFn.TIMER_ID, GlobalWindow.INSTANCE, new Instant(0), TimeDomain.EVENT_TIME); + ThrowingDoFn.TIMER_ID, + ThrowingDoFn.TIMER_ID, + GlobalWindow.INSTANCE, + new Instant(0), + new Instant(0), + TimeDomain.EVENT_TIME); } /** @@ -238,18 +243,22 @@ public void testOnTimerCalled() { // Mocking is not easily compatible with annotation analysis, so we manually record // the method call. runner.onTimer( + DoFnWithTimers.TIMER_ID, DoFnWithTimers.TIMER_ID, GlobalWindow.INSTANCE, currentTime.plus(offset), + currentTime.plus(offset), TimeDomain.EVENT_TIME); assertThat( fn.onTimerInvocations, contains( TimerData.of( + DoFnWithTimers.TIMER_ID, DoFnWithTimers.TIMER_ID, StateNamespaces.window(windowFn.windowCoder(), GlobalWindow.INSTANCE), currentTime.plus(offset), + currentTime.plus(offset), TimeDomain.EVENT_TIME))); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index 28b387e448e2..10ac7a82b4eb 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -283,7 +283,13 @@ public void testOnTimerCalled() { // Mocking is not easily compatible with annotation analysis, so we manually record // the method call. - runner.onTimer(timerId, window, new Instant(timestamp), TimeDomain.EVENT_TIME); + runner.onTimer( + timerId, + timerId, + window, + new Instant(timestamp), + new Instant(timestamp), + TimeDomain.EVENT_TIME); assertThat( underlying.firedTimers, @@ -320,12 +326,19 @@ public void processElement(WindowedValue elem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { firedTimers.add( TimerData.of( timerId, + timerFamilyId, StateNamespaces.window(IntervalWindow.getCoder(), (IntervalWindow) window), timestamp, + outputTimestamp, timeDomain)); } @@ -458,7 +471,13 @@ private static void advanceInputWatermark( StateNamespace namespace = timer.getNamespace(); checkArgument(namespace instanceof StateNamespaces.WindowNamespace); BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); - toTrigger.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + toTrigger.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index 85b3c0b6cec2..be4e321fe5d3 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -220,7 +220,13 @@ private static void advanceInputWatermark( StateNamespace namespace = timer.getNamespace(); checkArgument(namespace instanceof StateNamespaces.WindowNamespace); BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); - toTrigger.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + toTrigger.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 31eb80b43a0f..5f4117504d9c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -222,7 +222,13 @@ public void processElement(WindowedValue element) { public void onTimer(TimerData timer, BoundedWindow window) { try { - fnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + fnRunner.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } catch (Exception e) { throw UserCodeException.wrap(e); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index 9d853a2ca16e..ce54d5be1fa3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -68,12 +68,14 @@ public void processElement(final WindowedValue elem) { @Override public void onTimer( final String timerId, + final String timerFamilyId, final BoundedWindow window, final Instant timestamp, + final Instant outputTimestamp, final TimeDomain timeDomain) { try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(container.getMetricsContainer(stepName))) { - delegate.onTimer(timerId, window, timestamp, timeDomain); + delegate.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } catch (IOException e) { throw new RuntimeException(e); } 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 6aee09f0bb38..08e0a90a6b9b 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 @@ -215,7 +215,13 @@ private void fireTimer(TimerInternals.TimerData timer, DoFnRunner, Outp StateNamespace namespace = timer.getNamespace(); checkArgument(namespace instanceof StateNamespaces.WindowNamespace); BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); - doFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + doFnRunner.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } @Override 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 8e63679cbb45..5956ef65657d 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 @@ -813,7 +813,12 @@ protected void fireTimer(InternalTimer timer) { BoundedWindow window = ((WindowNamespace) namespace).getWindow(); timerInternals.cleanupPendingTimer(timer.getNamespace()); pushbackDoFnRunner.onTimer( - timerData.getTimerId(), window, timerData.getTimestamp(), timerData.getDomain()); + timerData.getTimerId(), + timerData.getTimerFamilyId(), + window, + timerData.getTimestamp(), + timerData.getOutputTimestamp(), + timerData.getDomain()); } private void setCurrentInputWatermark(long currentInputWatermark) { 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 891cef841f21..45fc2a128a5a 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 @@ -661,7 +661,12 @@ public void processElement(WindowedValue element) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { Object timerKey = keyForTimer.get(); Preconditions.checkNotNull(timerKey, "Key for timer needs to be set before calling onTimer"); Preconditions.checkNotNull(remoteBundle, "Call to onTimer outside of a bundle"); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java index b0f9304e858e..5c5ca6a5df5f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java @@ -67,20 +67,30 @@ public int hashCode() { static final class Timer implements BufferedElement { private final String timerId; + private final String timerFamilyId; private final BoundedWindow window; private final Instant timestamp; + private final Instant outputTimestamp; private final TimeDomain timeDomain; - Timer(String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + Timer( + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { this.timerId = timerId; this.window = window; this.timestamp = timestamp; this.timeDomain = timeDomain; + this.outputTimestamp = outputTimestamp; + this.timerFamilyId = timerFamilyId; } @Override public void processWith(DoFnRunner doFnRunner) { - doFnRunner.onTimer(timerId, window, timestamp, timeDomain); + doFnRunner.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } @Override @@ -130,8 +140,10 @@ public void encode(BufferedElement value, OutputStream outStream) throws IOExcep outStream.write(TIMER_MAGIC_BYTE); Timer timer = (Timer) value; STRING_CODER.encode(timer.timerId, outStream); + STRING_CODER.encode(timer.timerFamilyId, outStream); windowCoder.encode(timer.window, outStream); INSTANT_CODER.encode(timer.timestamp, outStream); + INSTANT_CODER.encode(timer.outputTimestamp, outStream); outStream.write(timer.timeDomain.ordinal()); } else { throw new IllegalStateException("Unexpected element " + value); @@ -146,9 +158,11 @@ public BufferedElement decode(InputStream inStream) throws IOException { return new Element(elementCoder.decode(inStream)); case TIMER_MAGIC_BYTE: return new Timer( + STRING_CODER.decode(inStream), STRING_CODER.decode(inStream), windowCoder.decode(inStream), INSTANT_CODER.decode(inStream), + INSTANT_CODER.decode(inStream), TimeDomain.values()[inStream.read()]); default: throw new IllegalStateException( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 80aabc2290e9..367ed32a6481 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -117,9 +117,15 @@ public void processElement(WindowedValue elem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { currentBufferingElementsHandler.buffer( - new BufferedElements.Timer(timerId, window, timestamp, timeDomain)); + new BufferedElements.Timer( + timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain)); } @Override diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java index 9ebdefcc09ee..0828a22a2603 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java @@ -52,7 +52,12 @@ public void testCoder() throws IOException { WindowedValue.of("test", new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); BufferedElement timerElement = new BufferedElements.Timer( - "timerId", GlobalWindow.INSTANCE, new Instant(1), TimeDomain.EVENT_TIME); + "timerId", + "timerId", + GlobalWindow.INSTANCE, + new Instant(1), + new Instant(1), + TimeDomain.EVENT_TIME); testRoundTrip(ImmutableList.of(element), coder); testRoundTrip(ImmutableList.of(timerElement), coder); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index 2903cc083d60..3cb92c980b60 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -109,7 +109,12 @@ private static BoundedWindow getUnderlyingWindow(KeyedWorkItem kw @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { throw new UnsupportedOperationException("Unsupported for ProcessFn"); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index e50f1bd93cf6..62964615e8b6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -82,7 +82,12 @@ public void processElement(WindowedValue elem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { throw new UnsupportedOperationException( String.format("Timers are not supported by %s", GroupAlsoByWindowFn.class.getSimpleName())); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 712a017fd27d..4c011edc4724 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -357,7 +357,13 @@ public void processTimers() throws Exception { private void processUserTimer(TimerData timer) throws Exception { if (fnSignature.timerDeclarations().containsKey(timer.getTimerId())) { BoundedWindow window = ((WindowNamespace) timer.getNamespace()).getWindow(); - fnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + fnRunner.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index bf5efee468e5..41da4b311412 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -133,7 +133,12 @@ public void processElement(WindowedValue> elem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { throw new UnsupportedOperationException( "Attempt to deliver a timer to a DoFn, but timers are not supported in Dataflow."); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index 02a12923dd7f..f7e86c7dc4b4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -77,7 +77,12 @@ public void processElement(WindowedValue compressedElem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { throw new UnsupportedOperationException( "Attempt to deliver a timer to a DoFn, but timers are not supported in Dataflow."); } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java index e291117f1ad6..76e837549a42 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java @@ -92,7 +92,13 @@ private static void fireTimer( TimerInternals.TimerData timer, DoFnRunner, ?> doFnRunner) { StateNamespace namespace = timer.getNamespace(); BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); - doFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + doFnRunner.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } @Override diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java index 101ee80f9c84..aefcf6d471ab 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java @@ -57,8 +57,16 @@ public void processElement(WindowedValue elem) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { - withMetrics(() -> underlying.onTimer(timerId, window, timestamp, timeDomain)); + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { + withMetrics( + () -> + underlying.onTimer( + timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain)); } @Override diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java index 795b8c0fd80d..cd140c7e3769 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java @@ -452,7 +452,13 @@ private void fireTimer(KeyedTimerData keyedTimerData) { // Need to pass in the keyed TimerData here ((DoFnRunnerWithKeyedInternals) fnRunner).onTimer(keyedTimerData, window); } else { - pushbackFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + pushbackFnRunner.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java index 6fb2bd3087e1..3b2d1cbfa460 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java @@ -62,7 +62,13 @@ public void onTimer(KeyedTimerData keyedTimerData, BoundedWindow window) { try { final TimerInternals.TimerData timer = keyedTimerData.getTimerData(); - onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } finally { clearKeyedInternals(); } @@ -70,10 +76,15 @@ public void onTimer(KeyedTimerData keyedTimerData, BoundedWindow window) { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) { + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) { checkState(keyedInternals.getKey() != null, "Key is not set for timer"); - underlying.onTimer(timerId, window, timestamp, timeDomain); + underlying.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } @Override diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java index 3b1b938ec0ce..99439a22018a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java @@ -262,7 +262,12 @@ private void emitResults() { @Override public void onTimer( - String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {} + String timerId, + String timerFamilyId, + BoundedWindow window, + Instant timestamp, + Instant outputTimestamp, + TimeDomain timeDomain) {} @Override public void finishBundle() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index 46dc282b8ee0..55d97ba17a1b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -71,11 +71,13 @@ public void processElement(final WindowedValue elem) { @Override public void onTimer( final String timerId, + final String timerFamilyId, final BoundedWindow window, final Instant timestamp, + final Instant outputTimestamp, final TimeDomain timeDomain) { try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) { - delegate.onTimer(timerId, window, timestamp, timeDomain); + delegate.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index 845dc63b488e..013f860060a3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -71,11 +71,13 @@ public void processElement(final WindowedValue elem) { @Override public void onTimer( final String timerId, + final String timerFamilyId, final BoundedWindow window, final Instant timestamp, + final Instant outputTimestamp, final TimeDomain timeDomain) { try (Closeable ignored = MetricsEnvironment.scopedMetricsContainer(metricsContainer())) { - delegate.onTimer(timerId, window, timestamp, timeDomain); + delegate.onTimer(timerId, timerFamilyId, window, timestamp, outputTimestamp, timeDomain); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index e978f4677963..9cbbedae8aad 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -161,7 +161,13 @@ private void fireTimer(TimerInternals.TimerData timer) { StateNamespace namespace = timer.getNamespace(); checkArgument(namespace instanceof StateNamespaces.WindowNamespace); BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); - doFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + doFnRunner.onTimer( + timer.getTimerId(), + timer.getTimerFamilyId(), + window, + timer.getTimestamp(), + timer.getOutputTimestamp(), + timer.getDomain()); } } } From 1172b8d437d2480c93d0a4ec6e5403888e8dd6cb Mon Sep 17 00:00:00 2001 From: David Sabater Date: Mon, 6 Jan 2020 14:09:44 +0000 Subject: [PATCH 45/67] [BEAM-9053] Improve error message when unable to get the correct filesystem for specified path in Python SDK --- sdks/python/apache_beam/io/filesystems.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/filesystems.py b/sdks/python/apache_beam/io/filesystems.py index d8b3a4a54542..b41952fc9374 100644 --- a/sdks/python/apache_beam/io/filesystems.py +++ b/sdks/python/apache_beam/io/filesystems.py @@ -89,7 +89,9 @@ def get_filesystem(path): 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). From 2dc923b0aeb627d0db5d7b2a41f56865659fd629 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Thu, 7 Nov 2019 15:39:14 +0100 Subject: [PATCH 46/67] [BEAM-8577] Initialize FileSystems during Coder deserialization in Reshuffle reduce phase. --- .../types/CoderTypeSerializer.java | 19 +++++++++ .../types/CoderTypeSerializer.java | 19 +++++++++ .../flink/FlinkBatchTransformTranslators.java | 18 ++++++-- .../functions/FlinkIdentityFunction.java | 42 +++++++++++++++++++ .../types/CoderTypeInformation.java | 26 +++++++++++- 5 files changed, 118 insertions(+), 6 deletions(-) create mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkIdentityFunction.java 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 e29f97e550e8..807faf541e5c 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 @@ public class CoderTypeSerializer extends TypeSerializer { private final Coder 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 coder) { Preconditions.checkNotNull(coder); this.coder = coder; + this.pipelineOptions = null; + } + + public CoderTypeSerializer( + Coder 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 2ff1cdaa9d6e..276e49cd53ab 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 @@ public class CoderTypeSerializer extends TypeSerializer { private final Coder 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 coder) { Preconditions.checkNotNull(coder); this.coder = coder; + this.pipelineOptions = null; + } + + public CoderTypeSerializer( + Coder coder, @Nullable SerializablePipelineOptions pipelineOptions) { + Preconditions.checkNotNull(coder); + this.coder = coder; + this.pipelineOptions = pipelineOptions; } @Override 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 229eca50c02e..27c9fbaaaaa7 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,19 @@ private static class ReshuffleTranslatorBatch @Override public void translateNode( Reshuffle transform, FlinkBatchTranslationContext context) { - - DataSet>> inputDataSet = + final DataSet>> inputDataSet = context.getInputDataSet(context.getInput(transform)); - - context.setOutputDataSet(context.getOutput(transform), inputDataSet.rebalance()); + @SuppressWarnings("unchecked") + final CoderTypeInformation>> outputType = + ((CoderTypeInformation) inputDataSet.getType()) + .withPipelineOptions(context.getPipelineOptions()); + final DataSet>> 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/translation/functions/FlinkIdentityFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkIdentityFunction.java new file mode 100644 index 000000000000..be3db7c3e20a --- /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 Input element type. + */ +public class FlinkIdentityFunction implements MapFunction { + + private static FlinkIdentityFunction INSTANCE = new FlinkIdentityFunction<>(); + + @SuppressWarnings("unchecked") + public static FlinkIdentityFunction 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/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index c03bef91fb21..5e76923e357a 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 extends TypeInformation implements AtomicType { private final Coder coder; + @Nullable private final SerializablePipelineOptions pipelineOptions; public CoderTypeInformation(Coder coder) { checkNotNull(coder); this.coder = coder; + this.pipelineOptions = null; + } + + private CoderTypeInformation(Coder coder, PipelineOptions pipelineOptions) { + checkNotNull(coder); + this.coder = coder; + this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions); } public Coder getCoder() { @@ -70,9 +81,8 @@ public boolean isKeyType() { } @Override - @SuppressWarnings("unchecked") public TypeSerializer createSerializer(ExecutionConfig config) { - return new CoderTypeSerializer<>(coder); + return new CoderTypeSerializer<>(coder, pipelineOptions); } @Override @@ -80,6 +90,18 @@ public int getTotalFields() { return 2; } + /** + * Creates a new {@link CoderTypeInformation} with {@link PipelineOptions}, that can be used for + * {@link org.apache.beam.sdk.io.FileSystems} registration. + * + * @see Jira issue. + * @param pipelineOptions Options of current pipeline. + * @return New type information. + */ + public CoderTypeInformation withPipelineOptions(PipelineOptions pipelineOptions) { + return new CoderTypeInformation<>(getCoder(), pipelineOptions); + } + @Override public boolean equals(Object o) { if (this == o) { From 39f7048c4f37f20c4b1e11eb3dba8e6fd3b4fdce Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Mon, 6 Jan 2020 10:13:33 -0800 Subject: [PATCH 47/67] Update _posts_2019-12-16-beam-2.17.0.md - Dropped "Make beam_fn_api flag opt-out rather than opt-in for " -- It is an internal change does not need to be a highlight. - Minor edits. --- website/src/_posts_2019-12-16-beam-2.17.0.md | 50 +++++++++----------- 1 file changed, 22 insertions(+), 28 deletions(-) diff --git a/website/src/_posts_2019-12-16-beam-2.17.0.md b/website/src/_posts_2019-12-16-beam-2.17.0.md index 1a10f136c44d..756c6618376a 100644 --- a/website/src/_posts_2019-12-16-beam-2.17.0.md +++ b/website/src/_posts_2019-12-16-beam-2.17.0.md @@ -1,7 +1,7 @@ --- layout: post title: "Apache Beam 2.17.0" -date: 2019-12-16 00:00:01 -0800 +date: 2020-01-06 00:00:01 -0800 # Date above corrected but keep the old URL: permalink: /blog/2019/12/16/beam-2.17.0.html excerpt_separator: @@ -32,42 +32,36 @@ For more information on changes in 2.17.0, check out the ## Highlights * [BEAM-7962](https://issues.apache.org/jira/browse/BEAM-7962) - Drop support for Flink 1.5 and 1.6 * [BEAM-7635](https://issues.apache.org/jira/browse/BEAM-7635) - Migrate SnsIO to AWS SDK for Java 2 -* [BEAM-8483](https://issues.apache.org/jira/browse/BEAM-8483) - Make beam_fn_api flag opt-out rather than opt-in for runners. +* Improved usability for portable Flink Runner with [BEAM-8183](https://issues.apache.org/jira/browse/BEAM-8183) - Optionally bundle multiple pipelines into a single Flink jar AND [BEAM-8372](https://issues.apache.org/jira/browse/BEAM-8372) - Allow submission of Flink UberJar directly to flink cluster AND [BEAM-8471](https://issues.apache.org/jira/browse/BEAM-8471) - Flink native job submission for portable pipelines AND [BEAM-8312](https://issues.apache.org/jira/browse/BEAM-8312) - Flink portable pipeline jars do not need to stage artifacts remotely. ### New Features / Improvements -* [BEAM-7730](https://issues.apache.org/jira/browse/BEAM-7730) - Add Flink 1.9 build target and Make FlinkRunner compatible with Flink 1.9 -* [BEAM-7990](https://issues.apache.org/jira/browse/BEAM-7990) - Add ability to read parquet files into PCollection -* [BEAM-8183](https://issues.apache.org/jira/browse/BEAM-8183) - Optionally bundle multiple pipelines into a single Flink jar -* [BEAM-8355](https://issues.apache.org/jira/browse/BEAM-8355) - Make BooleanCoder a standard coder -* [BEAM-8372](https://issues.apache.org/jira/browse/BEAM-8372) - Allow submission of Flink UberJar directly to flink cluster. -* [BEAM-8394](https://issues.apache.org/jira/browse/BEAM-8394) - Add withDataSourceConfiguration() method in JdbcIO.ReadRows class -* [BEAM-5428](https://issues.apache.org/jira/browse/BEAM-5428) - Implement cross-bundle state caching. -* [BEAM-5967](https://issues.apache.org/jira/browse/BEAM-5967) - Add handling of DynamicMessage in ProtoCoder -* [BEAM-7473](https://issues.apache.org/jira/browse/BEAM-7473) - Update RestrictionTracker within Python to not be required to be thread safe -* [BEAM-7920](https://issues.apache.org/jira/browse/BEAM-7920) - AvroTableProvider -* [BEAM-8098](https://issues.apache.org/jira/browse/BEAM-8098) - Improve documentation on BigQueryIO -* [BEAM-8100](https://issues.apache.org/jira/browse/BEAM-8100) - Add exception handling to Json transforms in Java SDK -* [BEAM-8138](https://issues.apache.org/jira/browse/BEAM-8138) - Fix code snippets in FileIO Java docs -* [BEAM-8306](https://issues.apache.org/jira/browse/BEAM-8306) - improve estimation of data byte size reading from source in ElasticsearchIO -* [BEAM-8312](https://issues.apache.org/jira/browse/BEAM-8312) - Flink portable pipeline jars do not need to stage artifacts remotely -* [BEAM-8351](https://issues.apache.org/jira/browse/BEAM-8351) - Support passing in arbitrary KV pairs to sdk worker via external environment config +* [BEAM-7730](https://issues.apache.org/jira/browse/BEAM-7730) - Add Flink 1.9 build target and Make FlinkRunner compatible with Flink 1.9. +* [BEAM-7990](https://issues.apache.org/jira/browse/BEAM-7990) - Add ability to read parquet files into PCollection. +* [BEAM-8355](https://issues.apache.org/jira/browse/BEAM-8355) - Make BooleanCoder a standard coder. +* [BEAM-8394](https://issues.apache.org/jira/browse/BEAM-8394) - Add withDataSourceConfiguration() method in JdbcIO.ReadRows class. +* [BEAM-5428](https://issues.apache.org/jira/browse/BEAM-5428) - Implement cross-bundle state caching.. +* [BEAM-5967](https://issues.apache.org/jira/browse/BEAM-5967) - Add handling of DynamicMessage in ProtoCoder. +* [BEAM-7473](https://issues.apache.org/jira/browse/BEAM-7473) - Update RestrictionTracker within Python to not be required to be thread safe. +* [BEAM-7920](https://issues.apache.org/jira/browse/BEAM-7920) - Added AvroTableProvider to Beam SQL. +* [BEAM-8098](https://issues.apache.org/jira/browse/BEAM-8098) - Improve documentation on BigQueryIO. +* [BEAM-8100](https://issues.apache.org/jira/browse/BEAM-8100) - Add exception handling to Json transforms in Java SDK. +* [BEAM-8306](https://issues.apache.org/jira/browse/BEAM-8306) - Improve estimation of data byte size reading from source in ElasticsearchIO. +* [BEAM-8351](https://issues.apache.org/jira/browse/BEAM-8351) - Support passing in arbitrary KV pairs to sdk worker via external environment config. * [BEAM-8396](https://issues.apache.org/jira/browse/BEAM-8396) - Default to LOOPBACK mode for local flink (spark, ...) runner. -* [BEAM-8410](https://issues.apache.org/jira/browse/BEAM-8410) - JdbcIO should support setConnectionInitSqls in its DataSource -* [BEAM-8417](https://issues.apache.org/jira/browse/BEAM-8417) - Expose ExternalWorkerHandler hostname -* [BEAM-8471](https://issues.apache.org/jira/browse/BEAM-8471) - Flink native job submission for portable pipelines -* [BEAM-8609](https://issues.apache.org/jira/browse/BEAM-8609) - Add HllCount to Java transform catalog -* [BEAM-8861](https://issues.apache.org/jira/browse/BEAM-8861) - Disallow self-signed certificates by default in ElasticsearchIO +* [BEAM-8410](https://issues.apache.org/jira/browse/BEAM-8410) - JdbcIO should support setConnectionInitSqls in its DataSource. +* [BEAM-8609](https://issues.apache.org/jira/browse/BEAM-8609) - Add HllCount to Java transform catalog. +* [BEAM-8861](https://issues.apache.org/jira/browse/BEAM-8861) - Disallow self-signed certificates by default in ElasticsearchIO. ### Dependency Changes -* [BEAM-8285](https://issues.apache.org/jira/browse/BEAM-8285) - Upgrade ZetaSQL to 2019.09.1 -* [BEAM-8392](https://issues.apache.org/jira/browse/BEAM-8392) - Upgrade pyarrow version bounds: 0.15.1<= to <0.16.0 -* [BEAM-5895](https://issues.apache.org/jira/browse/BEAM-5895) - Upgrade com.rabbitmq:amqp-client to 5.7.3 -* [BEAM-6896](https://issues.apache.org/jira/browse/BEAM-6896) - Upgrade PyYAML version bounds: 3.12<= to <6.0.0 +* [BEAM-8285](https://issues.apache.org/jira/browse/BEAM-8285) - Upgrade ZetaSQL to 2019.09.1. +* [BEAM-8392](https://issues.apache.org/jira/browse/BEAM-8392) - Upgrade pyarrow version bounds: 0.15.1<= to <0.16.0. +* [BEAM-5895](https://issues.apache.org/jira/browse/BEAM-5895) - Upgrade com.rabbitmq:amqp-client to 5.7.3. +* [BEAM-6896](https://issues.apache.org/jira/browse/BEAM-6896) - Upgrade PyYAML version bounds: 3.12<= to <6.0.0. ### Bugfixes -* Various bug fixes and performance improvements. * [BEAM-8819] - AvroCoder for SpecificRecords is not serialized correctly since 2.13.0 +* Various bug fixes and performance improvements. ### Known Issues From 7439b4fd3d1e394e7a283f72bb604c9ce0e30d30 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin <12602502+Ardagan@users.noreply.github.com> Date: Mon, 6 Jan 2020 10:26:38 -0800 Subject: [PATCH 48/67] Cleanup formatting. --- website/src/_posts_2019-12-16-beam-2.17.0.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/website/src/_posts_2019-12-16-beam-2.17.0.md b/website/src/_posts_2019-12-16-beam-2.17.0.md index 756c6618376a..7997f9e79fbb 100644 --- a/website/src/_posts_2019-12-16-beam-2.17.0.md +++ b/website/src/_posts_2019-12-16-beam-2.17.0.md @@ -32,14 +32,18 @@ For more information on changes in 2.17.0, check out the ## Highlights * [BEAM-7962](https://issues.apache.org/jira/browse/BEAM-7962) - Drop support for Flink 1.5 and 1.6 * [BEAM-7635](https://issues.apache.org/jira/browse/BEAM-7635) - Migrate SnsIO to AWS SDK for Java 2 -* Improved usability for portable Flink Runner with [BEAM-8183](https://issues.apache.org/jira/browse/BEAM-8183) - Optionally bundle multiple pipelines into a single Flink jar AND [BEAM-8372](https://issues.apache.org/jira/browse/BEAM-8372) - Allow submission of Flink UberJar directly to flink cluster AND [BEAM-8471](https://issues.apache.org/jira/browse/BEAM-8471) - Flink native job submission for portable pipelines AND [BEAM-8312](https://issues.apache.org/jira/browse/BEAM-8312) - Flink portable pipeline jars do not need to stage artifacts remotely. +* Improved usability for portable Flink Runner + * [BEAM-8183](https://issues.apache.org/jira/browse/BEAM-8183) - Optionally bundle multiple pipelines into a single Flink jar. + * [BEAM-8372](https://issues.apache.org/jira/browse/BEAM-8372) - Allow submission of Flink UberJar directly to flink cluster. + * [BEAM-8471](https://issues.apache.org/jira/browse/BEAM-8471) - Flink native job submission for portable pipelines. + * [BEAM-8312](https://issues.apache.org/jira/browse/BEAM-8312) - Flink portable pipeline jars do not need to stage artifacts remotely. ### New Features / Improvements * [BEAM-7730](https://issues.apache.org/jira/browse/BEAM-7730) - Add Flink 1.9 build target and Make FlinkRunner compatible with Flink 1.9. * [BEAM-7990](https://issues.apache.org/jira/browse/BEAM-7990) - Add ability to read parquet files into PCollection. * [BEAM-8355](https://issues.apache.org/jira/browse/BEAM-8355) - Make BooleanCoder a standard coder. * [BEAM-8394](https://issues.apache.org/jira/browse/BEAM-8394) - Add withDataSourceConfiguration() method in JdbcIO.ReadRows class. -* [BEAM-5428](https://issues.apache.org/jira/browse/BEAM-5428) - Implement cross-bundle state caching.. +* [BEAM-5428](https://issues.apache.org/jira/browse/BEAM-5428) - Implement cross-bundle state caching. * [BEAM-5967](https://issues.apache.org/jira/browse/BEAM-5967) - Add handling of DynamicMessage in ProtoCoder. * [BEAM-7473](https://issues.apache.org/jira/browse/BEAM-7473) - Update RestrictionTracker within Python to not be required to be thread safe. * [BEAM-7920](https://issues.apache.org/jira/browse/BEAM-7920) - Added AvroTableProvider to Beam SQL. From 11fbac273613d2e550f1407c2bc8d91286b99496 Mon Sep 17 00:00:00 2001 From: Mikhail Gryzykhin <12602502+Ardagan@users.noreply.github.com> Date: Mon, 6 Jan 2020 15:11:41 -0800 Subject: [PATCH 49/67] Update release date. --- website/src/get-started/downloads.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/src/get-started/downloads.md b/website/src/get-started/downloads.md index 9f4e2465e0c3..d70f7d73fb7f 100644 --- a/website/src/get-started/downloads.md +++ b/website/src/get-started/downloads.md @@ -90,7 +90,7 @@ versions denoted `0.x.y`. ## Releases -## 2.17.0 (2019-12-16) +## 2.17.0 (2020-01-06) Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.17.0/apache-beam-2.17.0-source-release.zip). [SHA-512](https://www.apache.org/dist/beam/2.17.0/apache-beam-2.17.0-source-release.zip.sha512). [signature](https://www.apache.org/dist/beam/2.17.0/apache-beam-2.17.0-source-release.zip.asc). From d0eb6dd0cf8305c9b5c0bd53a31783337d9ed939 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Mon, 6 Jan 2020 16:12:57 -0800 Subject: [PATCH 50/67] [BEAM-9059] Migrate PTransformTranslation to use string constants --- .../construction/PTransformTranslation.java | 146 +++++++++++++----- 1 file changed, 105 insertions(+), 41 deletions(-) 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 c21cd388011b..95f50eaa18c5 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 @@ -19,6 +19,7 @@ import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; 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 java.io.IOException; import java.util.Collection; @@ -60,68 +61,131 @@ * buffers}. */ public class PTransformTranslation { - - public static final String PAR_DO_TRANSFORM_URN = getUrn(StandardPTransforms.Primitives.PAR_DO); - public static final String FLATTEN_TRANSFORM_URN = getUrn(StandardPTransforms.Primitives.FLATTEN); - public static final String GROUP_BY_KEY_TRANSFORM_URN = - getUrn(StandardPTransforms.Primitives.GROUP_BY_KEY); - public static final String IMPULSE_TRANSFORM_URN = getUrn(StandardPTransforms.Primitives.IMPULSE); - public static final String ASSIGN_WINDOWS_TRANSFORM_URN = - getUrn(StandardPTransforms.Primitives.ASSIGN_WINDOWS); - public static final String TEST_STREAM_TRANSFORM_URN = - getUrn(StandardPTransforms.Primitives.TEST_STREAM); - public static final String MAP_WINDOWS_TRANSFORM_URN = - getUrn(StandardPTransforms.Primitives.MAP_WINDOWS); - + // We specifically copy the values here so that they can be used in switch case statements + // and we validate that the value matches the actual URN in the static block below. + + // Primitives + public static final String PAR_DO_TRANSFORM_URN = "beam:transform:pardo:v1"; + public static final String FLATTEN_TRANSFORM_URN = "beam:transform:flatten:v1"; + public static final String GROUP_BY_KEY_TRANSFORM_URN = "beam:transform:group_by_key:v1"; + public static final String IMPULSE_TRANSFORM_URN = "beam:transform:impulse:v1"; + public static final String ASSIGN_WINDOWS_TRANSFORM_URN = "beam:transform:window_into:v1"; + public static final String TEST_STREAM_TRANSFORM_URN = "beam:transform:teststream:v1"; + public static final String MAP_WINDOWS_TRANSFORM_URN = "beam:transform:map_windows:v1"; + + // DeprecatedPrimitives /** * @deprecated SDKs should move away from creating `Read` transforms and migrate to using Impulse * + SplittableDoFns. */ - @Deprecated - public static final String READ_TRANSFORM_URN = - getUrn(StandardPTransforms.DeprecatedPrimitives.READ); + @Deprecated public static final String READ_TRANSFORM_URN = "beam:transform:read:v1"; + /** * @deprecated runners should move away from translating `CreatePCollectionView` and treat this as * part of the translation for a `ParDo` side input. */ @Deprecated - public static final String CREATE_VIEW_TRANSFORM_URN = - getUrn(StandardPTransforms.DeprecatedPrimitives.CREATE_VIEW); + public static final String CREATE_VIEW_TRANSFORM_URN = "beam:transform:create_view:v1"; - public static final String COMBINE_PER_KEY_TRANSFORM_URN = - getUrn(StandardPTransforms.Composites.COMBINE_PER_KEY); - public static final String COMBINE_GLOBALLY_TRANSFORM_URN = - getUrn(StandardPTransforms.Composites.COMBINE_GLOBALLY); - public static final String COMBINE_GROUPED_VALUES_TRANSFORM_URN = - getUrn(CombineComponents.COMBINE_GROUPED_VALUES); + // Composites + public static final String COMBINE_PER_KEY_TRANSFORM_URN = "beam:transform:combine_per_key:v1"; + public static final String COMBINE_GLOBALLY_TRANSFORM_URN = "beam:transform:combine_globally:v1"; + public static final String RESHUFFLE_URN = "beam:transform:reshuffle:v1"; + public static final String WRITE_FILES_TRANSFORM_URN = "beam:transform:write_files:v1"; + + // CombineComponents public static final String COMBINE_PER_KEY_PRECOMBINE_TRANSFORM_URN = - getUrn(CombineComponents.COMBINE_PER_KEY_PRECOMBINE); + "beam:transform:combine_per_key_precombine:v1"; public static final String COMBINE_PER_KEY_MERGE_ACCUMULATORS_TRANSFORM_URN = - getUrn(CombineComponents.COMBINE_PER_KEY_MERGE_ACCUMULATORS); + "beam:transform:combine_per_key_merge_accumulators:v1"; public static final String COMBINE_PER_KEY_EXTRACT_OUTPUTS_TRANSFORM_URN = - getUrn(CombineComponents.COMBINE_PER_KEY_EXTRACT_OUTPUTS); - public static final String RESHUFFLE_URN = getUrn(StandardPTransforms.Composites.RESHUFFLE); - public static final String WRITE_FILES_TRANSFORM_URN = - getUrn(StandardPTransforms.Composites.WRITE_FILES); + "beam:transform:combine_per_key_extract_outputs:v1"; + public static final String COMBINE_GROUPED_VALUES_TRANSFORM_URN = + "beam:transform:combine_grouped_values:v1"; // SplittableParDoComponents public static final String SPLITTABLE_PAIR_WITH_RESTRICTION_URN = - getUrn(SplittableParDoComponents.PAIR_WITH_RESTRICTION); + "beam:transform:sdf_pair_with_restriction:v1"; public static final String SPLITTABLE_SPLIT_RESTRICTION_URN = - getUrn(SplittableParDoComponents.SPLIT_RESTRICTION); + "beam:transform:sdf_split_restriction:v1"; + /** + * @deprecated runners should move away from using `SplittableProcessKeyedElements` and prefer to + * internalize any necessary SplittableDoFn expansion. + */ + @Deprecated public static final String SPLITTABLE_PROCESS_KEYED_URN = - getUrn(SplittableParDoComponents.PROCESS_KEYED_ELEMENTS); + "beam:transform:sdf_process_keyed_elements:v1"; + public static final String SPLITTABLE_PROCESS_ELEMENTS_URN = - getUrn(SplittableParDoComponents.PROCESS_ELEMENTS); + "beam:transform:sdf_process_elements:v1"; public static final String SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN = - getUrn(SplittableParDoComponents.SPLIT_AND_SIZE_RESTRICTIONS); + "beam:transform:sdf_split_and_size_restrictions:v1"; public static final String SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN = - getUrn(SplittableParDoComponents.PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS); - - public static final String ITERABLE_SIDE_INPUT = - getUrn(RunnerApi.StandardSideInputTypes.Enum.ITERABLE); - public static final String MULTIMAP_SIDE_INPUT = - getUrn(RunnerApi.StandardSideInputTypes.Enum.MULTIMAP); + "beam:transform:sdf_process_sized_element_and_restrictions:v1"; + + static { + checkState(PAR_DO_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.PAR_DO))); + checkState(FLATTEN_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.FLATTEN))); + checkState( + GROUP_BY_KEY_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.GROUP_BY_KEY))); + checkState(IMPULSE_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.IMPULSE))); + checkState( + ASSIGN_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.ASSIGN_WINDOWS))); + checkState( + TEST_STREAM_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.TEST_STREAM))); + checkState( + MAP_WINDOWS_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Primitives.MAP_WINDOWS))); + + // DeprecatedPrimitives + checkState(READ_TRANSFORM_URN.equals(getUrn(StandardPTransforms.DeprecatedPrimitives.READ))); + checkState( + CREATE_VIEW_TRANSFORM_URN.equals( + getUrn(StandardPTransforms.DeprecatedPrimitives.CREATE_VIEW))); + + // Composites + checkState( + COMBINE_PER_KEY_TRANSFORM_URN.equals( + getUrn(StandardPTransforms.Composites.COMBINE_PER_KEY))); + checkState( + COMBINE_GLOBALLY_TRANSFORM_URN.equals( + getUrn(StandardPTransforms.Composites.COMBINE_GLOBALLY))); + checkState(RESHUFFLE_URN.equals(getUrn(StandardPTransforms.Composites.RESHUFFLE))); + checkState( + WRITE_FILES_TRANSFORM_URN.equals(getUrn(StandardPTransforms.Composites.WRITE_FILES))); + + // CombineComponents + checkState( + COMBINE_PER_KEY_PRECOMBINE_TRANSFORM_URN.equals( + getUrn(CombineComponents.COMBINE_PER_KEY_PRECOMBINE))); + checkState( + COMBINE_PER_KEY_MERGE_ACCUMULATORS_TRANSFORM_URN.equals( + getUrn(CombineComponents.COMBINE_PER_KEY_MERGE_ACCUMULATORS))); + checkState( + COMBINE_PER_KEY_EXTRACT_OUTPUTS_TRANSFORM_URN.equals( + getUrn(CombineComponents.COMBINE_PER_KEY_EXTRACT_OUTPUTS))); + checkState( + COMBINE_GROUPED_VALUES_TRANSFORM_URN.equals( + getUrn(CombineComponents.COMBINE_GROUPED_VALUES))); + + // SplittableParDoComponents + checkState( + SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals( + getUrn(SplittableParDoComponents.PAIR_WITH_RESTRICTION))); + checkState( + SPLITTABLE_SPLIT_RESTRICTION_URN.equals( + getUrn(SplittableParDoComponents.SPLIT_RESTRICTION))); + checkState( + SPLITTABLE_PROCESS_KEYED_URN.equals( + getUrn(SplittableParDoComponents.PROCESS_KEYED_ELEMENTS))); + checkState( + SPLITTABLE_PROCESS_ELEMENTS_URN.equals(getUrn(SplittableParDoComponents.PROCESS_ELEMENTS))); + checkState( + SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals( + getUrn(SplittableParDoComponents.SPLIT_AND_SIZE_RESTRICTIONS))); + checkState( + SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals( + getUrn(SplittableParDoComponents.PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS))); + } private static final Collection> KNOWN_TRANSLATORS = loadKnownTranslators(); From 66a2481ecbee5689fbad7d3abbdf6071cdae8b2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 09:40:08 +0100 Subject: [PATCH 51/67] [BEAM-5546] Update commons-codec to version 1.14 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- sdks/java/io/amazon-web-services/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 0d28c6ba4291..1236f06ca876 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -428,7 +428,7 @@ class BeamModulePlugin implements Plugin { cassandra_driver_core : "com.datastax.cassandra:cassandra-driver-core:$cassandra_driver_version", cassandra_driver_mapping : "com.datastax.cassandra:cassandra-driver-mapping:$cassandra_driver_version", classgraph : "io.github.classgraph:classgraph:$classgraph_version", - commons_codec : "commons-codec:commons-codec:1.10", + commons_codec : "commons-codec:commons-codec:1.14", commons_compress : "org.apache.commons:commons-compress:1.19", commons_csv : "org.apache.commons:commons-csv:1.4", commons_io_1x : "commons-io:commons-io:1.3.2", diff --git a/sdks/java/io/amazon-web-services/build.gradle b/sdks/java/io/amazon-web-services/build.gradle index d7e413941632..6948a58208d6 100644 --- a/sdks/java/io/amazon-web-services/build.gradle +++ b/sdks/java/io/amazon-web-services/build.gradle @@ -38,7 +38,7 @@ dependencies { compile library.java.jackson_annotations compile library.java.jackson_databind compile library.java.slf4j_api - runtime 'commons-codec:commons-codec:1.9' + runtime library.java.commons_codec runtime "org.apache.httpcomponents:httpclient:4.5.6" testCompile project(path: ":sdks:java:core", configuration: "shadowTest") testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime") From f94bbc100eddd8ef5f99631c61785acf02735cd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 09:54:20 +0100 Subject: [PATCH 52/67] [BEAM-8701] Remove unused commons-io_1x dependency --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 1 - .../io/elasticsearch-tests/elasticsearch-tests-2/build.gradle | 1 - .../io/elasticsearch-tests/elasticsearch-tests-5/build.gradle | 1 - .../io/elasticsearch-tests/elasticsearch-tests-6/build.gradle | 1 - .../io/elasticsearch-tests/elasticsearch-tests-7/build.gradle | 1 - .../elasticsearch-tests/elasticsearch-tests-common/build.gradle | 1 - 6 files changed, 6 deletions(-) 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 0d28c6ba4291..85acd7606c99 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -431,7 +431,6 @@ class BeamModulePlugin implements Plugin { commons_codec : "commons-codec:commons-codec:1.10", commons_compress : "org.apache.commons:commons-compress:1.19", commons_csv : "org.apache.commons:commons-csv:1.4", - commons_io_1x : "commons-io:commons-io:1.3.2", commons_io_2x : "commons-io:commons-io:2.5", commons_lang3 : "org.apache.commons:commons-lang3:3.6", commons_math3 : "org.apache.commons:commons-math3:3.6.1", diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle index 0c4411d81920..3b64569bb8e6 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle @@ -42,7 +42,6 @@ dependencies { testCompile "org.apache.logging.log4j:log4j-core:$log4j_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:5.6.3" testCompile "org.elasticsearch:elasticsearch:$elastic_search_version" diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle index 2e137004d9ff..c1560ac92831 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle @@ -61,7 +61,6 @@ dependencies { 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 diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle index b7bf6d08cb1f..6d7ae8aecff4 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle @@ -61,7 +61,6 @@ dependencies { 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 diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle index 924b3fca3de8..f731cd0951e7 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle @@ -61,7 +61,6 @@ dependencies { 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 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 a77a29a5cb80..830abacf2758 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle @@ -49,7 +49,6 @@ dependencies { testCompile "org.apache.logging.log4j:log4j-core:$log4j_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 From 71e16f82c3eb7a0f63277d087415d334204e7954 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 09:55:15 +0100 Subject: [PATCH 53/67] [BEAM-8701] Update commons-io to version 2.6 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- runners/samza/build.gradle | 2 +- runners/spark/build.gradle | 2 +- sdks/java/io/cassandra/build.gradle | 1 + sdks/java/io/hadoop-format/build.gradle | 2 +- sdks/java/io/hcatalog/build.gradle | 2 +- 6 files changed, 6 insertions(+), 5 deletions(-) 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 85acd7606c99..ee917eaa9ac3 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -431,7 +431,7 @@ class BeamModulePlugin implements Plugin { commons_codec : "commons-codec:commons-codec:1.10", commons_compress : "org.apache.commons:commons-compress:1.19", commons_csv : "org.apache.commons:commons-csv:1.4", - commons_io_2x : "commons-io:commons-io:2.5", + commons_io : "commons-io:commons-io:2.6", commons_lang3 : "org.apache.commons:commons-lang3:3.6", commons_math3 : "org.apache.commons:commons-math3:3.6.1", error_prone_annotations : "com.google.errorprone:error_prone_annotations:2.0.15", diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle index 18dbea4d9929..2628bdfcd321 100644 --- a/runners/samza/build.gradle +++ b/runners/samza/build.gradle @@ -46,7 +46,7 @@ dependencies { compile library.java.slf4j_api compile library.java.joda_time compile library.java.commons_compress - compile library.java.commons_io_2x + compile library.java.commons_io compile library.java.args4j compile "org.apache.samza:samza-api:$samza_version" compile "org.apache.samza:samza-core_2.11:$samza_version" diff --git a/runners/spark/build.gradle b/runners/spark/build.gradle index 3de3bc1a0c35..592e06d6e150 100644 --- a/runners/spark/build.gradle +++ b/runners/spark/build.gradle @@ -71,7 +71,7 @@ dependencies { provided library.java.spark_network_common provided library.java.hadoop_common provided library.java.commons_lang3 - provided library.java.commons_io_2x + provided library.java.commons_io provided library.java.hamcrest_core provided library.java.hamcrest_library provided "com.esotericsoftware.kryo:kryo:2.21" diff --git a/sdks/java/io/cassandra/build.gradle b/sdks/java/io/cassandra/build.gradle index 36dbede6b728..6f029048b1bb 100644 --- a/sdks/java/io/cassandra/build.gradle +++ b/sdks/java/io/cassandra/build.gradle @@ -43,6 +43,7 @@ dependencies { // for embedded cassandra testCompile group: 'info.archinnov', name: 'achilles-junit', version: "$achilles_version" + testCompile library.java.commons_io testCompile library.java.jackson_jaxb_annotations testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle index 1e0cf94d3fec..2856fbbbb611 100644 --- a/sdks/java/io/hadoop-format/build.gradle +++ b/sdks/java/io/hadoop-format/build.gradle @@ -75,6 +75,7 @@ dependencies { } // elasticsearch-hadoop 5.0.0 uses commons-httpclient's URIException testCompile "commons-httpclient:commons-httpclient:3.1" + testCompile library.java.commons_io testCompile library.java.cassandra_driver_core testCompile library.java.cassandra_driver_mapping testCompile "org.apache.cassandra:cassandra-all:3.11.3" @@ -85,7 +86,6 @@ dependencies { testCompile library.java.hamcrest_library testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") - compile library.java.commons_io_2x delegate.add("sparkRunner", project(":sdks:java:io:hadoop-format")) delegate.add("sparkRunner", project(path: ":sdks:java:io:hadoop-format", configuration: "testRuntime")) diff --git a/sdks/java/io/hcatalog/build.gradle b/sdks/java/io/hcatalog/build.gradle index e32277d10e44..e0b43a2932da 100644 --- a/sdks/java/io/hcatalog/build.gradle +++ b/sdks/java/io/hcatalog/build.gradle @@ -59,7 +59,7 @@ dependencies { exclude group: "com.google.protobuf", module: "protobuf-java" } testCompile project(":sdks:java:io:common").sourceSets.test.output - testCompile library.java.commons_io_2x + testCompile library.java.commons_io testCompile library.java.junit testCompile library.java.hamcrest_core testCompile library.java.hamcrest_library From cc52294c84bfb6b17f6a99c4f5fc9b9967b4e1ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 10:49:24 +0100 Subject: [PATCH 54/67] [BEAM-5544] Update cassandra-all dependency to version 3.11.5 --- sdks/java/io/hadoop-format/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle index 1e0cf94d3fec..e75a42c5b8b7 100644 --- a/sdks/java/io/hadoop-format/build.gradle +++ b/sdks/java/io/hadoop-format/build.gradle @@ -77,7 +77,7 @@ dependencies { testCompile "commons-httpclient:commons-httpclient:3.1" testCompile library.java.cassandra_driver_core testCompile library.java.cassandra_driver_mapping - testCompile "org.apache.cassandra:cassandra-all:3.11.3" + testCompile "org.apache.cassandra:cassandra-all:3.11.5" testCompile library.java.postgres testCompile "org.apache.logging.log4j:log4j-core:$log4j_version" testCompile library.java.junit From 9392f5f9a8b96970a65952fb1ba31be5dba43593 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 10:51:58 +0100 Subject: [PATCH 55/67] [BEAM-8749] Update cassandra-driver-mapping to version 3.8.0 --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0d28c6ba4291..742ee9b9d9d4 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -362,7 +362,7 @@ class BeamModulePlugin implements Plugin { def apex_malhar_version = "3.4.0" def aws_java_sdk_version = "1.11.519" def aws_java_sdk2_version = "2.5.71" - def cassandra_driver_version = "3.6.0" + def cassandra_driver_version = "3.8.0" def classgraph_version = "4.8.56" def generated_grpc_beta_version = "0.44.0" def generated_grpc_ga_version = "1.43.0" From 0aa98761b8ae04b0bc571272686450ed7cc16439 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Tue, 7 Jan 2020 09:33:27 -0800 Subject: [PATCH 56/67] Restrict the upper bound for pyhamcrest, since new version does not work on py2. --- sdks/python/setup.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index fcb4fd4ccf2d..cabaa1f6c290 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -178,7 +178,8 @@ def get_version(): 'nose_xunitmp>=0.4.1', 'pandas>=0.23.4,<0.25', 'parameterized>=0.6.0,<0.8.0', - 'pyhamcrest>=1.9,<2.0', + # pyhamcrest==1.10.0 requires Py3. Beam still supports Py2. + 'pyhamcrest>=1.9,<1.10.0', 'pyyaml>=3.12,<6.0.0', 'requests_mock>=1.7,<2.0', 'tenacity>=5.0.2,<6.0', From ae6444df04b784f1d65ab660bce7c3b10156765c Mon Sep 17 00:00:00 2001 From: Andrew Pilloud Date: Mon, 6 Jan 2020 16:43:57 -0800 Subject: [PATCH 57/67] [BEAM-9027] [SQL] Fix ZetaSQL Byte Literals --- .../bigquery/BeamSqlUnparseContext.java | 4 +++- .../sql/zetasql/ZetaSQLDialectSpecTest.java | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) 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 index 432b3b5db282..60fd1bc7ea41 100644 --- 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 @@ -20,6 +20,7 @@ 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.avatica.util.ByteString; 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; @@ -46,7 +47,8 @@ public SqlNode toSql(RexProgram program, RexNode rex) { final RexLiteral literal = (RexLiteral) rex; SqlTypeFamily family = literal.getTypeName().getFamily(); if (SqlTypeFamily.BINARY.equals(family)) { - BitString bitString = BitString.createFromBytes(literal.getValueAs(byte[].class)); + ByteString byteString = literal.getValueAs(ByteString.class); + BitString bitString = BitString.createFromHexString(byteString.toString(16)); return new SqlByteStringLiteral(bitString, POS); } else if (SqlTypeFamily.CHARACTER.equals(family)) { String escaped = StringEscapeUtils.escapeJava(literal.getValueAs(String.class)); 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 e140c02da04f..4e78749f796e 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 @@ -140,6 +140,23 @@ public void testSimpleSelect() { pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); } + @Test + public void testByteLiterals() { + String sql = "SELECT b'abc'"; + + byte[] byteString = new byte[] {'a', 'b', 'c'}; + + ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); + BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); + PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); + + final Schema schema = Schema.builder().addNullableField("ColA", FieldType.BYTES).build(); + + PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(byteString).build()); + + pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); + } + @Test public void testByteString() { String sql = "SELECT @p0 IS NULL AS ColA"; From f34be7c64b8009b24f1210a3d68ffe7f31928249 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Tue, 7 Jan 2020 10:52:50 -0800 Subject: [PATCH 58/67] [BEAM-9058] Fix line-too-long exclusion regex and re-enable line-too-long linter check. (#10510) --- sdks/python/.pylintrc | 2 +- sdks/python/apache_beam/io/filesystems.py | 7 ++++--- .../apache_beam/runners/portability/fn_api_runner.py | 3 ++- sdks/python/apache_beam/testing/test_stream_it_test.py | 4 ++-- sdks/python/apache_beam/transforms/core.py | 8 +++++--- sdks/python/setup.py | 5 ++++- 6 files changed, 18 insertions(+), 11 deletions(-) diff --git a/sdks/python/.pylintrc b/sdks/python/.pylintrc index 26b39dfaa6b3..fb226b789658 100644 --- a/sdks/python/.pylintrc +++ b/sdks/python/.pylintrc @@ -179,7 +179,7 @@ indent-after-paren=4 ignore-long-lines=(?x) (^\s*(import|from)\s |^\s*(\#\ )??$ - |# type: + |^.*\#\ type\: ) [VARIABLES] diff --git a/sdks/python/apache_beam/io/filesystems.py b/sdks/python/apache_beam/io/filesystems.py index 39e61005b892..bf562c834809 100644 --- a/sdks/python/apache_beam/io/filesystems.py +++ b/sdks/python/apache_beam/io/filesystems.py @@ -96,9 +96,10 @@ def get_filesystem(path): systems = [fs for fs in FileSystem.get_all_subclasses() if fs.scheme() == path_scheme] if len(systems) == 0: - 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) + 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/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py index 7c45847fc174..8f2bf7545837 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py @@ -112,7 +112,8 @@ DataSideInput = Dict[Tuple[str, str], Tuple[bytes, beam_runner_api_pb2.FunctionSpec]] DataOutput = Dict[str, bytes] -BundleProcessResult = Tuple[beam_fn_api_pb2.InstructionResponse, List[beam_fn_api_pb2.ProcessBundleSplitResponse]] +BundleProcessResult = Tuple[beam_fn_api_pb2.InstructionResponse, + List[beam_fn_api_pb2.ProcessBundleSplitResponse]] # This module is experimental. No backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/testing/test_stream_it_test.py b/sdks/python/apache_beam/testing/test_stream_it_test.py index 8e724f654fff..1b95b4fea76a 100644 --- a/sdks/python/apache_beam/testing/test_stream_it_test.py +++ b/sdks/python/apache_beam/testing/test_stream_it_test.py @@ -47,8 +47,8 @@ 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)) + self.skipTest('The "{}", does not support the TestStream transform. ' + 'Supported runners: {}'.format(self.runner_name, runners)) else: return fn(self) return wrapped diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 25cc91f6142d..520dc6f944eb 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1304,7 +1304,9 @@ def to_runner_api_parameter(self, context): return ( common_urns.primitives.PAR_DO.urn, beam_runner_api_pb2.ParDoPayload( - do_fn=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) @@ -2268,8 +2270,8 @@ class Windowing(object): def __init__(self, windowfn, # type: WindowFn 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] + accumulation_mode=None, # type: typing.Optional[beam_runner_api_pb2.AccumulationMode] + timestamp_combiner=None, # type: typing.Optional[beam_runner_api_pb2.OutputTime] allowed_lateness=0, # type: typing.Union[int, float] ): """Class representing the window strategy. diff --git a/sdks/python/setup.py b/sdks/python/setup.py index cabaa1f6c290..8599b7f226ef 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -138,7 +138,10 @@ def get_version(): 'avro>=1.8.1,<2.0.0; python_version < "3.0"', 'avro-python3>=1.8.1,<2.0.0; python_version >= "3.0"', 'crcmod>=1.7,<2.0', - # Dill doesn't guarantee compatibility between releases within minor version. + # Dill doesn't have forwards-compatibility guarantees within minor version. + # Pickles created with a new version of dill may not unpickle using older + # version of dill. It is best to use the same version of dill on client and + # server, therefore list of allowed versions is very narrow. # See: https://github.com/uqfoundation/dill/issues/341. 'dill>=0.3.1.1,<0.3.2', 'fastavro>=0.21.4,<0.22', From 80faf6cac13c8b9dd543944c92c86f3814037d67 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 7 Jan 2020 11:23:01 -0800 Subject: [PATCH 59/67] Readability/Lint fixes --- .../io/aws/clients/s3/client_test.py | 20 +++++++++---------- .../apache_beam/io/aws/s3filesystem_test.py | 4 ++-- .../interactive/pipeline_instrument_test.py | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) 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 index e78c484ee18f..2457234f8566 100644 --- a/sdks/python/apache_beam/io/aws/clients/s3/client_test.py +++ b/sdks/python/apache_beam/io/aws/clients/s3/client_test.py @@ -68,8 +68,8 @@ 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, + self.assertRaises(messages.S3ClientError, + self.client.get_range, request, 0, 10) try: @@ -128,9 +128,9 @@ 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, + request = messages.CopyRequest(self.test_bucket, + src_key, + self.test_bucket, dest_key) with self.assertRaises(messages.S3ClientError) as e: @@ -150,14 +150,14 @@ def test_upload_part_bad_number(self): upload_id = response.upload_id part_number = 0.5 - request = messages.UploadPartRequest(self.test_bucket, - object, - upload_id, - part_number, + request = messages.UploadPartRequest(self.test_bucket, + object, + upload_id, + part_number, contents) self.assertRaises(messages.S3ClientError, - self.client.upload_part, + self.client.upload_part, request) try: diff --git a/sdks/python/apache_beam/io/aws/s3filesystem_test.py b/sdks/python/apache_beam/io/aws/s3filesystem_test.py index 4bddbf2528f8..48b70d690062 100644 --- a/sdks/python/apache_beam/io/aws/s3filesystem_test.py +++ b/sdks/python/apache_beam/io/aws/s3filesystem_test.py @@ -129,7 +129,7 @@ def test_match_multiples_error(self, unused_mock_arg): with self.assertRaises(BeamIOError) as error: self.fs.match(['s3://bucket/']) - self.assertTrue('Match operation failed' in str(error.exception)) + self.assertIn('Match operation failed', str(error.exception)) s3io_mock.list_prefix.assert_called_once_with('s3://bucket/') @mock.patch('apache_beam.io.aws.s3filesystem.s3io') @@ -241,7 +241,7 @@ def test_delete_error(self, unused_mock_arg): # Issue batch delete. with self.assertRaises(BeamIOError) as error: self.fs.delete(files) - self.assertTrue('Delete operation failed' in str(error.exception)) + self.assertIn('Delete operation failed', str(error.exception)) self.assertEqual(error.exception.exception_details, expected_results) s3io_mock.delete_paths.assert_called() 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 f06c27128f71..62cbd7342f55 100644 --- a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py +++ b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py @@ -288,7 +288,7 @@ def test_find_out_correct_user_pipeline(self): 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) + self.assertIs(pipeline_instrument.user_pipeline, user_pipeline) if __name__ == '__main__': From ef8545f8cf5c2170c21099eaaeb360c10e3d898a Mon Sep 17 00:00:00 2001 From: Hannah Jiang Date: Mon, 6 Jan 2020 17:05:52 -0800 Subject: [PATCH 60/67] BEAM-8780 reuse RC images instead of recreate images --- .../src/main/scripts/publish_docker_images.sh | 87 ++++++++++++------- website/src/contribute/release-guide.md | 4 +- 2 files changed, 55 insertions(+), 36 deletions(-) mode change 100644 => 100755 release/src/main/scripts/publish_docker_images.sh diff --git a/release/src/main/scripts/publish_docker_images.sh b/release/src/main/scripts/publish_docker_images.sh old mode 100644 new mode 100755 index 44a133e5ff7c..fb6ba0db9aa6 --- a/release/src/main/scripts/publish_docker_images.sh +++ b/release/src/main/scripts/publish_docker_images.sh @@ -24,43 +24,78 @@ set -e -source release/src/main/scripts/build_release_candidate.sh +PYTHON_VER=("python2.7" "python3.5" "python3.6" "python3.7") +FLINK_VER=("$(ls -1 runners/flink | awk '/^[0-9]+\.[0-9]+$/{print}')") echo "Publish SDK docker images to Docker Hub." + +echo "================Setting Up Environment Variables===========" +echo "Which release version are you working on: " +read RELEASE + +echo "================Setting Up RC candidate Variables===========" +echo "From which RC candidate do you create publish docker image? (ex: rc0, rc1) " +read RC_VERSION + +echo "================Confirmimg Release and RC version===========" +echo "We are using ${RC_VERSION} to create docker images for ${RELEASE}." echo "Do you want to proceed? [y|N]" read confirmation if [[ $confirmation = "y" ]]; then - echo "============Publishing SDK docker images on docker hub=========" - cd ~ - if [[ -d ${LOCAL_PYTHON_STAGING_DIR} ]]; then - rm -rf ${LOCAL_PYTHON_STAGING_DIR} - fi - mkdir -p ${LOCAL_PYTHON_STAGING_DIR} - cd ${LOCAL_PYTHON_STAGING_DIR} - - echo '-------------------Cloning Beam Release Branch-----------------' - git clone ${GIT_REPO_URL} - cd ${BEAM_ROOT_DIR} - git checkout ${RELEASE_BRANCH} echo '-------------------Generating and Pushing Python images-----------------' - ./gradlew :sdks:python:container:buildAll -Pdocker-tag=${RELEASE} for ver in "${PYTHON_VER[@]}"; do + # Pull varified RC from dockerhub. + docker pull apachebeam/${ver}_sdk:${RELEASE}_${RC_VERSION} + + # Tag with ${RELEASE} and push to dockerhub. + docker tag apachebeam/${ver}_sdk:${RELEASE}_${RC_VERSION} apachebeam/${ver}_sdk:${RELEASE} docker push apachebeam/${ver}_sdk:${RELEASE} - docker tag apachebeam/${ver}_sdk:${RELEASE} apachebeam/${ver}_sdk:latest + + # Tag with latest and push to dockerhub. + docker tag apachebeam/${ver}_sdk:${RELEASE}_${RC_VERSION} apachebeam/${ver}_sdk:latest docker push apachebeam/${ver}_sdk:latest + + # Cleanup images from local + docker rmi -f apachebeam/${ver}_sdk:${RELEASE}_${RC_VERSION} + docker rmi -f apachebeam/${ver}_sdk:${RELEASE} + docker rmi -f apachebeam/${ver}_sdk:latest done echo '-------------------Generating and Pushing Java images-----------------' - ./gradlew :sdks:java:container:dockerPush -Pdocker-tag=${RELEASE} - docker tag apachebeam/java_sdk:${RELEASE} apachebeam/java_sdk:latest + # Pull varified RC from dockerhub. + docker pull apachebeam/java_sdk:${RELEASE}_${RC_VERSION} + + # Tag with ${RELEASE} and push to dockerhub. + docker tag apachebeam/java_sdk:${RELEASE}_${RC_VERSION} apachebeam/java_sdk:${RELEASE} + docker push apachebeam/java_sdk:${RELEASE} + + # Tag with latest and push to dockerhub. + docker tag apachebeam/java_sdk:${RELEASE}_${RC_VERSION} apachebeam/java_sdk:latest docker push apachebeam/java_sdk:latest + # Cleanup images from local + docker rmi -f apachebeam/java_sdk:${RELEASE}_${RC_VERSION} + docker rmi -f apachebeam/java_sdk:${RELEASE} + docker rmi -f apachebeam/java_sdk:latest + echo '-------------------Generating and Pushing Go images-----------------' - ./gradlew :sdks:go:container:dockerPush -Pdocker-tag=${RELEASE} - docker tag apachebeam/go_sdk:${RELEASE} apachebeam/go_sdk:latest + # Pull varified RC from dockerhub. + docker pull apachebeam/go_sdk:${RELEASE}_${RC_VERSION} + + # Tag with ${RELEASE} and push to dockerhub. + docker tag apachebeam/go_sdk:${RELEASE}_${RC_VERSION} apachebeam/go_sdk:${RELEASE} + docker push apachebeam/go_sdk:${RELEASE} + + # Tag with latest and push to dockerhub. + docker tag apachebeam/go_sdk:${RELEASE}_${RC_VERSION} apachebeam/go_sdk:latest docker push apachebeam/go_sdk:latest + # Cleanup images from local + docker rmi -f apachebeam/go_sdk:${RELEASE}_${RC_VERSION} + docker rmi -f apachebeam/go_sdk:${RELEASE} + docker rmi -f apachebeam/go_sdk:latest + echo '-------------Generating and Pushing Flink job server images-------------' echo "Building containers for the following Flink versions:" "${FLINK_VER[@]}" for ver in "${FLINK_VER[@]}"; do @@ -71,20 +106,6 @@ if [[ $confirmation = "y" ]]; then docker push "${FLINK_IMAGE_NAME}:latest" done - rm -rf ~/${PYTHON_ARTIFACTS_DIR} - - echo "-------------------Clean up SDK docker images at local-------------------" - for ver in "${PYTHON_VER[@]}"; do - docker rmi -f apachebeam/${ver}_sdk:${RELEASE} - docker rmi -f apachebeam/${ver}_sdk:latest - done - - docker rmi -f apachebeam/java_sdk:${RELEASE} - docker rmi -f apachebeam/java_sdk:latest - - docker rmi -f apachebeam/go_sdk:${RELEASE} - docker rmi -f apachebeam/go_sdk:latest - for ver in "${FLINK_VER[@]}"; do FLINK_IMAGE_NAME=apachebeam/flink${ver}_job_server docker rmi -f "${FLINK_IMAGE_NAME}:${RELEASE}" diff --git a/website/src/contribute/release-guide.md b/website/src/contribute/release-guide.md index 2a27bded8794..40dd143aca93 100644 --- a/website/src/contribute/release-guide.md +++ b/website/src/contribute/release-guide.md @@ -1259,9 +1259,7 @@ __NOTE__: Only PMC members have permissions to do it, ping [dev@](mailto:dev@bea Make sure the download address for last release version is upldaed, [example PR](https://github.com/apache/beam-site/pull/478). ### Deploy SDK docker images to DockerHub -TODO(hannahjiang): change link to master branch after #9560 is merged. - -* Script: [publish_docker_images.sh](https://github.com/Hannah-Jiang/beam/blob/release_script_for_containers/release/src/main/scripts/publish_docker_images.sh) +* Script: [publish_docker_images.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/publish_docker_images.sh) * Usage ``` ./beam/release/src/main/scripts/publish_docker_images.sh From 3e4490cca2b902f719839a42a1ff482fb11c2444 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 17:43:11 +0100 Subject: [PATCH 61/67] [BEAM-9041] Add missing equals methods for GenericRecord <-> Row conversion --- .../beam/sdk/schemas/utils/AvroUtils.java | 65 ++++++++++++++++++- 1 file changed, 63 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java index 80c337ebb0a5..8b54e41b1f7a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -422,7 +423,37 @@ public static Schema getSchema(Class clazz, @Nullable org.apache.avro.Sch */ public static SerializableFunction getGenericRecordToRowFunction( @Nullable Schema schema) { - return g -> toBeamRowStrict(g, schema); + return new GenericRecordToRowFn(schema); + } + + private static class GenericRecordToRowFn implements SerializableFunction { + private final Schema schema; + + GenericRecordToRowFn(Schema schema) { + this.schema = schema; + } + + @Override + public Row apply(GenericRecord input) { + return toBeamRowStrict(input, schema); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + GenericRecordToRowFn that = (GenericRecordToRowFn) other; + return schema.equals(that.schema); + } + + @Override + public int hashCode() { + return Objects.hash(schema); + } } /** @@ -431,7 +462,37 @@ public static SerializableFunction getGenericRecordToRowFunc */ public static SerializableFunction getRowToGenericRecordFunction( @Nullable org.apache.avro.Schema avroSchema) { - return g -> toGenericRecord(g, avroSchema); + return new RowToGenericRecordFn(avroSchema); + } + + private static class RowToGenericRecordFn implements SerializableFunction { + private final org.apache.avro.Schema avroSchema; + + RowToGenericRecordFn(@Nullable org.apache.avro.Schema avroSchema) { + this.avroSchema = avroSchema; + } + + @Override + public GenericRecord apply(Row input) { + return toGenericRecord(input, avroSchema); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (other == null || getClass() != other.getClass()) { + return false; + } + RowToGenericRecordFn that = (RowToGenericRecordFn) other; + return avroSchema.equals(that.avroSchema); + } + + @Override + public int hashCode() { + return Objects.hash(avroSchema); + } } /** From 0e63b5af96ae3ad903dabf1ace2629b85766757c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 18:02:36 +0100 Subject: [PATCH 62/67] [BEAM-9042] Fix RowToGenericRecordFn Avro schema serialization --- .../apache/beam/sdk/schemas/utils/AvroUtils.java | 15 ++++++++++++++- .../beam/sdk/schemas/utils/AvroUtilsTest.java | 3 +++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java index 8b54e41b1f7a..4b158a08a8fc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java @@ -20,6 +20,9 @@ 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.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.lang.reflect.Method; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -466,7 +469,7 @@ public static SerializableFunction getRowToGenericRecordFunc } private static class RowToGenericRecordFn implements SerializableFunction { - private final org.apache.avro.Schema avroSchema; + private transient org.apache.avro.Schema avroSchema; RowToGenericRecordFn(@Nullable org.apache.avro.Schema avroSchema) { this.avroSchema = avroSchema; @@ -493,6 +496,16 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(avroSchema); } + + private void writeObject(ObjectOutputStream out) throws IOException { + final String avroSchemaAsString = (avroSchema == null) ? null : avroSchema.toString(); + out.writeObject(avroSchemaAsString); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + final String avroSchemaAsString = (String) in.readObject(); + avroSchema = new org.apache.avro.Schema.Parser().parse(avroSchemaAsString); + } } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java index 3679e2163429..229e3de6c7de 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java @@ -48,6 +48,7 @@ import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.utils.AvroGenerators.RecordSchemaGenerator; import org.apache.beam.sdk.schemas.utils.AvroUtils.TypeWithNullability; +import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; @@ -533,6 +534,7 @@ public void testAvroSchemaCoders() { assertFalse(records.hasSchema()); records.setCoder(AvroUtils.schemaCoder(schema)); assertTrue(records.hasSchema()); + CoderProperties.coderSerializable(records.getCoder()); AvroGeneratedUser user = new AvroGeneratedUser("foo", 42, "green"); PCollection users = @@ -540,6 +542,7 @@ public void testAvroSchemaCoders() { assertFalse(users.hasSchema()); users.setCoder(AvroUtils.schemaCoder((AvroCoder) users.getCoder())); assertTrue(users.hasSchema()); + CoderProperties.coderSerializable(users.getCoder()); } public static ContainsField containsField(Function predicate) { From 71de81f7d47bf4c93601d5216b2efadb7ce6aaee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 18:09:36 +0100 Subject: [PATCH 63/67] [BEAM-9042] Update SchemaCoder doc with info about functions requiring equals --- .../src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java index 9359c752a489..889925a1f234 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java @@ -102,7 +102,8 @@ protected SchemaCoder( /** * Returns a {@link SchemaCoder} for the specified class. If no schema is registered for this - * class, then throws {@link NoSuchSchemaException}. + * class, then throws {@link NoSuchSchemaException}. The parameter functions to convert from and + * to Rows must implement the equals contract. */ public static SchemaCoder of( Schema schema, From fe17671fb3da74a8880ef9c4f204f2a099a401a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 7 Jan 2020 22:51:26 +0100 Subject: [PATCH 64/67] [BEAM-9042] Test serializability and equality of Row<->GenericRecord functions --- .../apache/beam/sdk/schemas/utils/AvroUtils.java | 5 ++++- .../beam/sdk/schemas/utils/AvroUtilsTest.java | 13 +++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java index 4b158a08a8fc..3740de671692 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java @@ -504,7 +504,10 @@ private void writeObject(ObjectOutputStream out) throws IOException { private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { final String avroSchemaAsString = (String) in.readObject(); - avroSchema = new org.apache.avro.Schema.Parser().parse(avroSchemaAsString); + avroSchema = + (avroSchemaAsString == null) + ? null + : new org.apache.avro.Schema.Parser().parse(avroSchemaAsString); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java index 229e3de6c7de..cd27cb13f73b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java @@ -50,6 +50,7 @@ import org.apache.beam.sdk.schemas.utils.AvroUtils.TypeWithNullability; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; @@ -501,6 +502,12 @@ public void testBeamRowToGenericRecord() { assertEquals(getGenericRecord(), genericRecord); } + @Test + public void testRowToGenericRecordFunction() { + SerializableUtils.ensureSerializable(AvroUtils.getRowToGenericRecordFunction(NULL_SCHEMA)); + SerializableUtils.ensureSerializable(AvroUtils.getRowToGenericRecordFunction(null)); + } + @Test public void testGenericRecordToBeamRow() { GenericRecord genericRecord = getGenericRecord(); @@ -513,6 +520,12 @@ public void testGenericRecordToBeamRow() { assertEquals(getBeamRow(), row); } + @Test + public void testGenericRecordToRowFunction() { + SerializableUtils.ensureSerializable(AvroUtils.getGenericRecordToRowFunction(Schema.of())); + SerializableUtils.ensureSerializable(AvroUtils.getGenericRecordToRowFunction(null)); + } + @Test public void testAvroSchemaCoders() { Pipeline pipeline = Pipeline.create(); From 383357757294c84fd905b412a5314e4830d1493b Mon Sep 17 00:00:00 2001 From: Jeffrey Sorensen Date: Tue, 7 Jan 2020 17:46:39 -0500 Subject: [PATCH 65/67] [BEAM-9062] Improve assertion error for equal_to (#10504) --- sdks/python/apache_beam/testing/util.py | 14 ++++++---- sdks/python/apache_beam/testing/util_test.py | 27 ++++++++++++++++++++ 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/testing/util.py b/sdks/python/apache_beam/testing/util.py index 5b6bc854f4f8..cf108f9bcd87 100644 --- a/sdks/python/apache_beam/testing/util.py +++ b/sdks/python/apache_beam/testing/util.py @@ -174,15 +174,19 @@ def _equal(actual): # 2) As a fallback if we encounter a TypeError in python 3. this method # works on collections that have different types. except (BeamAssertException, TypeError): + unexpected = [] for element in actual: try: expected_list.remove(element) except ValueError: - raise BeamAssertException( - 'Failed assert: %r == %r' % (expected, actual)) - if expected_list: - raise BeamAssertException( - 'Failed assert: %r == %r' % (expected, actual)) + unexpected.append(element) + if unexpected or expected_list: + msg = 'Failed assert: %r == %r' % (expected, actual) + if unexpected: + msg = msg + ', unexpected elements %r' % unexpected + if expected_list: + msg = msg + ', missing elements %r' % expected_list + raise BeamAssertException(msg) return _equal diff --git a/sdks/python/apache_beam/testing/util_test.py b/sdks/python/apache_beam/testing/util_test.py index 72c9205802a7..fe2812a10cd0 100644 --- a/sdks/python/apache_beam/testing/util_test.py +++ b/sdks/python/apache_beam/testing/util_test.py @@ -42,6 +42,12 @@ class UtilTest(unittest.TestCase): + def setUp(self): + try: # Python 3 + _ = self.assertRaisesRegex + except AttributeError: # Python 2 + self.assertRaisesRegex = self.assertRaisesRegexp + def test_assert_that_passes(self): with TestPipeline() as p: assert_that(p | Create([1, 2, 3]), equal_to([1, 2, 3])) @@ -67,6 +73,27 @@ def test_assert_that_fails(self): with TestPipeline() as p: assert_that(p | Create([1, 10, 100]), equal_to([1, 2, 3])) + def test_assert_missing(self): + with self.assertRaisesRegex(BeamAssertException, + r"missing elements \['c'\]"): + with TestPipeline() as p: + assert_that(p | Create(['a', 'b']), equal_to(['a', 'b', 'c'])) + + def test_assert_unexpected(self): + with self.assertRaisesRegex(BeamAssertException, + r"unexpected elements \['c', 'd'\]|" + r"unexpected elements \['d', 'c'\]"): + with TestPipeline() as p: + assert_that(p | Create(['a', 'b', 'c', 'd']), equal_to(['a', 'b'])) + + def test_assert_missing_and_unexpected(self): + with self.assertRaisesRegex( + BeamAssertException, + r"unexpected elements \['c'\].*missing elements \['d'\]"): + with TestPipeline() as p: + assert_that(p | Create(['a', 'b', 'c']), + equal_to(['a', 'b', 'd'])) + def test_reified_value_passes(self): expected = [TestWindowedValue(v, MIN_TIMESTAMP, [GlobalWindow()]) for v in [1, 2, 3]] From ea99d6e4befd310061e9b88e7ee3a0945d5dc1b3 Mon Sep 17 00:00:00 2001 From: yirutang Date: Tue, 7 Jan 2020 14:52:48 -0800 Subject: [PATCH 66/67] [BEAM-8960]: Add an option for user to opt out of using insert id for BigQuery streaming insert. (#10427) * [BEAM-8960]: Add an option for user to opt out of using insert id for Bigquery streaming insert. Expose an option so that user can opt out of using insert id while streaming into BigQuery. Insert id only guarantees best effort insert rows deduplication, without it, user will be able to opt into using new streaming backend with higher quotas. * Address review comments * After running spotless --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 15 ++++++++ .../sdk/io/gcp/bigquery/BigQueryServices.java | 3 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 14 +++++--- .../sdk/io/gcp/bigquery/StreamingInserts.java | 26 ++++++++++++++ .../sdk/io/gcp/bigquery/StreamingWriteFn.java | 6 +++- .../io/gcp/bigquery/StreamingWriteTables.java | 24 +++++++++++++ .../io/gcp/testing/FakeDatasetService.java | 36 ++++++++++++++----- .../sdk/io/gcp/testing/TableContainer.java | 8 ++++- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 29 +++++++++++++++ .../bigquery/BigQueryServicesImplTest.java | 22 +++++++++--- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 2 +- 11 files changed, 163 insertions(+), 22 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 3bd9d8c81813..2301a5b3e9e2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -1643,6 +1643,7 @@ public static Write write() { .setExtendedErrorInfo(false) .setSkipInvalidRows(false) .setIgnoreUnknownValues(false) + .setIgnoreInsertIds(false) .setMaxFilesPerPartition(BatchLoads.DEFAULT_MAX_FILES_PER_PARTITION) .setMaxBytesPerPartition(BatchLoads.DEFAULT_MAX_BYTES_PER_PARTITION) .setOptimizeWrites(false) @@ -1774,6 +1775,8 @@ public enum Method { abstract Boolean getIgnoreUnknownValues(); + abstract Boolean getIgnoreInsertIds(); + @Nullable abstract String getKmsKey(); @@ -1846,6 +1849,8 @@ abstract Builder setAvroSchemaFactory( abstract Builder setIgnoreUnknownValues(Boolean ignoreUnknownValues); + abstract Builder setIgnoreInsertIds(Boolean ignoreInsertIds); + abstract Builder setKmsKey(String kmsKey); abstract Builder setOptimizeWrites(Boolean optimizeWrites); @@ -2241,6 +2246,15 @@ public Write ignoreUnknownValues() { return toBuilder().setIgnoreUnknownValues(true).build(); } + /** + * Setting this option to true disables insertId based data deduplication offered by BigQuery. + * For more information, please see + * https://cloud.google.com/bigquery/streaming-data-into-bigquery#disabling_best_effort_de-duplication. + */ + public Write ignoreInsertIds() { + return toBuilder().setIgnoreInsertIds(true).build(); + } + public Write withKmsKey(String kmsKey) { return toBuilder().setKmsKey(kmsKey).build(); } @@ -2600,6 +2614,7 @@ private WriteResult continueExpandTyped( .withExtendedErrorInfo(getExtendedErrorInfo()) .withSkipInvalidRows(getSkipInvalidRows()) .withIgnoreUnknownValues(getIgnoreUnknownValues()) + .withIgnoreInsertIds(getIgnoreInsertIds()) .withKmsKey(getKmsKey()); return input.apply(streamingInserts); } else { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index ecd4a853b13b..ce02423cf43c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -158,7 +158,8 @@ long insertAll( List> failedInserts, ErrorContainer errorContainer, boolean skipInvalidRows, - boolean ignoreUnknownValues) + boolean ignoreUnknownValues, + boolean ignoreInsertIds) throws IOException, InterruptedException; /** Patch BigQuery {@link Table} description. */ 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 218de4715c33..bf2bd3c62503 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 @@ -713,7 +713,8 @@ long insertAll( List> failedInserts, ErrorContainer errorContainer, boolean skipInvalidRows, - boolean ignoreUnkownValues) + boolean ignoreUnkownValues, + boolean ignoreInsertIds) throws IOException, InterruptedException { checkNotNull(ref, "ref"); if (executor == null) { @@ -733,7 +734,10 @@ long insertAll( // These lists contain the rows to publish. Initially the contain the entire list. // If there are failures, they will contain only the failed rows to be retried. List> rowsToPublish = rowList; - List idsToPublish = insertIdList; + List idsToPublish = null; + if (!ignoreInsertIds) { + idsToPublish = insertIdList; + } while (true) { List> retryRows = new ArrayList<>(); List retryIds = (idsToPublish != null) ? new ArrayList<>() : null; @@ -871,7 +875,8 @@ public long insertAll( List> failedInserts, ErrorContainer errorContainer, boolean skipInvalidRows, - boolean ignoreUnknownValues) + boolean ignoreUnknownValues, + boolean ignoreInsertIds) throws IOException, InterruptedException { return insertAll( ref, @@ -883,7 +888,8 @@ public long insertAll( failedInserts, errorContainer, skipInvalidRows, - ignoreUnknownValues); + ignoreUnknownValues, + ignoreInsertIds); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index 6dad9898473c..d00adbb29b94 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -38,6 +38,7 @@ public class StreamingInserts private boolean extendedErrorInfo; private final boolean skipInvalidRows; private final boolean ignoreUnknownValues; + private final boolean ignoreInsertIds; private final String kmsKey; private final Coder elementCoder; private final SerializableFunction toTableRow; @@ -56,6 +57,7 @@ public StreamingInserts( false, false, false, + false, elementCoder, toTableRow, null); @@ -70,6 +72,7 @@ private StreamingInserts( boolean extendedErrorInfo, boolean skipInvalidRows, boolean ignoreUnknownValues, + boolean ignoreInsertIds, Coder elementCoder, SerializableFunction toTableRow, String kmsKey) { @@ -80,6 +83,7 @@ private StreamingInserts( this.extendedErrorInfo = extendedErrorInfo; this.skipInvalidRows = skipInvalidRows; this.ignoreUnknownValues = ignoreUnknownValues; + this.ignoreInsertIds = ignoreInsertIds; this.elementCoder = elementCoder; this.toTableRow = toTableRow; this.kmsKey = kmsKey; @@ -96,6 +100,7 @@ public StreamingInserts withInsertRetryPolicy( extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow, kmsKey); @@ -111,6 +116,7 @@ public StreamingInserts withExtendedErrorInfo(boolean ex extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow, kmsKey); @@ -125,6 +131,7 @@ StreamingInserts withSkipInvalidRows(boolean skipInvalid extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow, kmsKey); @@ -139,6 +146,22 @@ StreamingInserts withIgnoreUnknownValues(boolean ignoreU extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, + elementCoder, + toTableRow, + kmsKey); + } + + StreamingInserts withIgnoreInsertIds(boolean ignoreInsertIds) { + return new StreamingInserts<>( + createDisposition, + dynamicDestinations, + bigQueryServices, + retryPolicy, + extendedErrorInfo, + skipInvalidRows, + ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow, kmsKey); @@ -153,6 +176,7 @@ StreamingInserts withKmsKey(String kmsKey) { extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow, kmsKey); @@ -167,6 +191,7 @@ StreamingInserts withTestServices(BigQueryServices bigQu extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow, kmsKey); @@ -188,6 +213,7 @@ public WriteResult expand(PCollection> input) { .withExtendedErrorInfo(extendedErrorInfo) .withSkipInvalidRows(skipInvalidRows) .withIgnoreUnknownValues(ignoreUnknownValues) + .withIgnoreInsertIds(ignoreInsertIds) .withElementCoder(elementCoder) .withToTableRow(toTableRow)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java index f56cf01c14ba..4e12018ede45 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java @@ -49,6 +49,7 @@ class StreamingWriteFn private final ErrorContainer errorContainer; private final boolean skipInvalidRows; private final boolean ignoreUnknownValues; + private final boolean ignoreInsertIds; private final SerializableFunction toTableRow; /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */ @@ -67,6 +68,7 @@ class StreamingWriteFn ErrorContainer errorContainer, boolean skipInvalidRows, boolean ignoreUnknownValues, + boolean ignoreInsertIds, SerializableFunction toTableRow) { this.bqServices = bqServices; this.retryPolicy = retryPolicy; @@ -74,6 +76,7 @@ class StreamingWriteFn this.errorContainer = errorContainer; this.skipInvalidRows = skipInvalidRows; this.ignoreUnknownValues = ignoreUnknownValues; + this.ignoreInsertIds = ignoreInsertIds; this.toTableRow = toTableRow; } @@ -145,7 +148,8 @@ private void flushRows( failedInserts, errorContainer, skipInvalidRows, - ignoreUnknownValues); + ignoreUnknownValues, + ignoreInsertIds); byteCounter.inc(totalBytes); } catch (IOException e) { throw new RuntimeException(e); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java index ea2c02033cee..81f097ae7c96 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java @@ -53,6 +53,7 @@ public class StreamingWriteTables private static final String FAILED_INSERTS_TAG_ID = "failedInserts"; private final boolean skipInvalidRows; private final boolean ignoreUnknownValues; + private final boolean ignoreInsertIds; private final Coder elementCoder; private final SerializableFunction toTableRow; @@ -63,6 +64,7 @@ public StreamingWriteTables() { false, // extendedErrorInfo false, // skipInvalidRows false, // ignoreUnknownValues + false, // ignoreInsertIds null, // elementCoder null); // toTableRow } @@ -73,6 +75,7 @@ private StreamingWriteTables( boolean extendedErrorInfo, boolean skipInvalidRows, boolean ignoreUnknownValues, + boolean ignoreInsertIds, Coder elementCoder, SerializableFunction toTableRow) { this.bigQueryServices = bigQueryServices; @@ -80,6 +83,7 @@ private StreamingWriteTables( this.extendedErrorInfo = extendedErrorInfo; this.skipInvalidRows = skipInvalidRows; this.ignoreUnknownValues = ignoreUnknownValues; + this.ignoreInsertIds = ignoreInsertIds; this.elementCoder = elementCoder; this.toTableRow = toTableRow; } @@ -91,6 +95,7 @@ StreamingWriteTables withTestServices(BigQueryServices bigQueryService extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -102,6 +107,7 @@ StreamingWriteTables withInsertRetryPolicy(InsertRetryPolicy retryPoli extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -113,6 +119,7 @@ StreamingWriteTables withExtendedErrorInfo(boolean extendedErrorInfo) extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -124,6 +131,7 @@ StreamingWriteTables withSkipInvalidRows(boolean skipInvalidRows) { extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -135,6 +143,19 @@ StreamingWriteTables withIgnoreUnknownValues(boolean ignoreUnknownValu extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, + elementCoder, + toTableRow); + } + + StreamingWriteTables withIgnoreInsertIds(boolean ignoreInsertIds) { + return new StreamingWriteTables<>( + bigQueryServices, + retryPolicy, + extendedErrorInfo, + skipInvalidRows, + ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -146,6 +167,7 @@ StreamingWriteTables withElementCoder(Coder elementCoder) { extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -158,6 +180,7 @@ StreamingWriteTables withToTableRow( extendedErrorInfo, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, elementCoder, toTableRow); } @@ -240,6 +263,7 @@ private PCollection writeAndGetErrors( errorContainer, skipInvalidRows, ignoreUnknownValues, + ignoreInsertIds, toTableRow)) .withOutputTags(mainOutputTag, TupleTagList.of(failedInsertsTag))); PCollection failedInserts = tuple.get(failedInsertsTag); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java index 7916513c1435..2ab1fdc0683a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java @@ -32,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; import java.util.regex.Pattern; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; @@ -90,6 +89,13 @@ public List getAllRows(String projectId, String datasetId, String tabl } } + public List getAllIds(String projectId, String datasetId, String tableId) + throws InterruptedException, IOException { + synchronized (tables) { + return getTableContainer(projectId, datasetId, tableId).getIds(); + } + } + private TableContainer getTableContainer(String projectId, String datasetId, String tableId) throws InterruptedException, IOException { synchronized (tables) { @@ -215,7 +221,15 @@ public long insertAll( PaneInfo.ON_TIME_AND_ONLY_FIRING)); } return insertAll( - ref, windowedRows, insertIdList, InsertRetryPolicy.alwaysRetry(), null, null, false, false); + ref, + windowedRows, + insertIdList, + InsertRetryPolicy.alwaysRetry(), + null, + null, + false, + false, + false); } @Override @@ -227,17 +241,17 @@ public long insertAll( List> failedInserts, ErrorContainer errorContainer, boolean skipInvalidRows, - boolean ignoreUnknownValues) + boolean ignoreUnknownValues, + boolean ignoreInsertIds) throws IOException, InterruptedException { Map> insertErrors = getInsertErrors(); synchronized (tables) { + if (ignoreInsertIds) { + insertIdList = null; + } + if (insertIdList != null) { assertEquals(rowList.size(), insertIdList.size()); - } else { - insertIdList = Lists.newArrayListWithExpectedSize(rowList.size()); - for (int i = 0; i < rowList.size(); ++i) { - insertIdList.add(Integer.toString(ThreadLocalRandom.current().nextInt())); - } } long dataSize = 0; @@ -258,7 +272,11 @@ public long insertAll( } } if (shouldInsert) { - dataSize += tableContainer.addRow(row, insertIdList.get(i)); + if (insertIdList == null) { + dataSize += tableContainer.addRow(row, null); + } else { + dataSize += tableContainer.addRow(row, insertIdList.get(i)); + } } else { errorContainer.add( failedInserts, allErrors.get(allErrors.size() - 1), ref, rowList.get(i)); 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 402da860823a..46c9f37048fe 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 @@ -40,7 +40,9 @@ class TableContainer { long addRow(TableRow row, String id) { rows.add(row); - ids.add(id); + if (id != null) { + ids.add(id); + } long tableSize = table.getNumBytes() == null ? 0L : table.getNumBytes(); try { long rowSize = TableRowJsonCoder.of().getEncodedElementByteSize(row); @@ -58,4 +60,8 @@ Table getTable() { List getRows() { return rows; } + + List getIds() { + return ids; + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index 99e3fbaec130..624a62d31ee5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -684,6 +684,35 @@ public void testWrite() throws Exception { p.run(); } + @Test + public void testWriteWithoutInsertId() throws Exception { + TableRow row1 = new TableRow().set("name", "a").set("number", 1); + TableRow row2 = new TableRow().set("name", "b").set("number", 2); + TableRow row3 = new TableRow().set("name", "c").set("number", 3); + p.apply(Create.of(row1, row2, row3).withCoder(TableRowJsonCoder.of())) + .apply( + BigQueryIO.writeTableRows() + .to("project-id:dataset-id.table-id") + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withMethod(BigQueryIO.Write.Method.STREAMING_INSERTS) + .withSchema( + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER")))) + .withTestServices(fakeBqServices) + .ignoreInsertIds() + .withoutValidation()); + p.run(); + assertThat( + fakeDatasetService.getAllRows("project-id", "dataset-id", "table-id"), + containsInAnyOrder(row1, row2, row3)); + // Verify no insert id is added. + assertThat( + fakeDatasetService.getAllIds("project-id", "dataset-id", "table-id"), containsInAnyOrder()); + } + @AutoValue abstract static class InputRecord implements Serializable { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index fb88bb474278..bac6c238d679 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -517,6 +517,7 @@ public void testInsertRateLimitRetry() throws Exception { null, null, false, + false, false); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); @@ -551,6 +552,7 @@ public void testInsertQuotaExceededRetry() throws Exception { null, null, false, + false, false); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); @@ -601,6 +603,7 @@ public void testInsertRetrySelectRows() throws Exception { null, null, false, + false, false); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); @@ -646,6 +649,7 @@ public void testInsertFailsGracefully() throws Exception { null, null, false, + false, false); fail(); } catch (IOException e) { @@ -693,6 +697,7 @@ public void testInsertOtherRetry() throws Throwable { null, null, false, + false, false); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); @@ -761,17 +766,18 @@ public void testInsertRetryPolicy() throws InterruptedException, IOException { failedInserts, ErrorContainer.TABLE_ROW_ERROR_CONTAINER, false, + false, false); assertEquals(1, failedInserts.size()); expectedLogs.verifyInfo("Retrying 1 failed inserts to BigQuery"); } /** - * Tests that {@link DatasetServiceImpl#insertAll} respects the skipInvalidRows and - * ignoreUnknownValues parameters. + * Tests that {@link DatasetServiceImpl#insertAll} respects the skipInvalidRows, + * ignoreUnknownValues and ignoreInsertIds parameters. */ @Test - public void testSkipInvalidRowsIgnoreUnknownValuesStreaming() + public void testSkipInvalidRowsIgnoreUnknownIgnoreInsertIdsValuesStreaming() throws InterruptedException, IOException { TableReference ref = new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); @@ -790,7 +796,7 @@ public void testSkipInvalidRowsIgnoreUnknownValuesStreaming() DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); - // First, test with both flags disabled + // First, test with all flags disabled dataService.insertAll( ref, rows, @@ -801,6 +807,7 @@ public void testSkipInvalidRowsIgnoreUnknownValuesStreaming() Lists.newArrayList(), ErrorContainer.TABLE_ROW_ERROR_CONTAINER, false, + false, false); TableDataInsertAllRequest parsedRequest = @@ -809,7 +816,7 @@ public void testSkipInvalidRowsIgnoreUnknownValuesStreaming() assertFalse(parsedRequest.getSkipInvalidRows()); assertFalse(parsedRequest.getIgnoreUnknownValues()); - // Then with both enabled + // Then with all enabled dataService.insertAll( ref, rows, @@ -820,12 +827,15 @@ public void testSkipInvalidRowsIgnoreUnknownValuesStreaming() Lists.newArrayList(), ErrorContainer.TABLE_ROW_ERROR_CONTAINER, true, + true, true); parsedRequest = fromString(request.getContentAsString(), TableDataInsertAllRequest.class); assertTrue(parsedRequest.getSkipInvalidRows()); assertTrue(parsedRequest.getIgnoreUnknownValues()); + assertNull(parsedRequest.getRows().get(0).getInsertId()); + assertNull(parsedRequest.getRows().get(1).getInsertId()); } /** A helper to convert a string response back to a {@link GenericJson} subclass. */ @@ -1002,6 +1012,7 @@ public void testSimpleErrorRetrieval() throws InterruptedException, IOException failedInserts, ErrorContainer.TABLE_ROW_ERROR_CONTAINER, false, + false, false); assertThat(failedInserts, is(rows)); @@ -1056,6 +1067,7 @@ public void testExtendedErrorRetrieval() throws InterruptedException, IOExceptio failedInserts, ErrorContainer.BIG_QUERY_INSERT_ERROR_ERROR_CONTAINER, false, + false, false); assertThat(failedInserts, is(expected)); 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 4dddf0dd7cee..98b8bc89218a 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 @@ -205,7 +205,7 @@ public void testInsertAll() throws Exception { try { totalBytes = datasetService.insertAll( - ref, rows, ids, InsertRetryPolicy.alwaysRetry(), null, null, false, false); + ref, rows, ids, InsertRetryPolicy.alwaysRetry(), null, null, false, false, false); } finally { verifyInsertAll(5); // Each of the 25 rows has 1 byte for length and 30 bytes: '{"f":[{"v":"foo"},{"v":1234}]}' From 784d18b7ac89f87dd7fbf2861ee877f5b6070276 Mon Sep 17 00:00:00 2001 From: dpcollins-google <40498610+dpcollins-google@users.noreply.github.com> Date: Tue, 7 Jan 2020 21:49:21 -0500 Subject: [PATCH 67/67] [BEAM-8932] [BEAM-9036] Revert reverted commit to use PubsubMessage as the canonical type in beam client (#10474) * Revert "Revert "[BEAM-8932] Modify PubsubClient to use the proto message throughout."" This reverts commit aeae2f417629c374cb025faa3a664a9646859a01. * Check for null attribute map in IncomingMessage before adding on PubsubMessage. --- .../beam/gradle/BeamModulePlugin.groovy | 4 +- .../beam/sdk/io/gcp/pubsub/PubsubClient.java | 139 +++++------------- .../sdk/io/gcp/pubsub/PubsubGrpcClient.java | 29 +--- .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 9 +- .../sdk/io/gcp/pubsub/PubsubJsonClient.java | 38 +++-- .../sdk/io/gcp/pubsub/PubsubTestClient.java | 11 +- .../io/gcp/pubsub/PubsubUnboundedSink.java | 23 ++- .../io/gcp/pubsub/PubsubUnboundedSource.java | 33 +++-- .../beam/sdk/io/gcp/pubsub/TestPubsub.java | 20 ++- .../sdk/io/gcp/pubsub/TestPubsubSignal.java | 6 +- .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 1 + .../io/gcp/pubsub/PubsubGrpcClientTest.java | 19 ++- .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java | 8 +- .../io/gcp/pubsub/PubsubJsonClientTest.java | 76 ++++++++-- .../io/gcp/pubsub/PubsubTestClientTest.java | 31 +++- .../gcp/pubsub/PubsubUnboundedSinkTest.java | 39 +++-- .../gcp/pubsub/PubsubUnboundedSourceTest.java | 28 +++- 17 files changed, 287 insertions(+), 227 deletions(-) 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 9a01a1f2fcb4..c4fb762f656d 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -365,7 +365,7 @@ class BeamModulePlugin implements Plugin { def cassandra_driver_version = "3.8.0" def classgraph_version = "4.8.56" def generated_grpc_beta_version = "0.44.0" - def generated_grpc_ga_version = "1.43.0" + def generated_grpc_ga_version = "1.83.0" def generated_grpc_dc_beta_version = "0.27.0-alpha" def google_auth_version = "0.12.0" def google_clients_version = "1.28.0" @@ -444,7 +444,7 @@ class BeamModulePlugin implements Plugin { google_api_services_clouddebugger : "com.google.apis:google-api-services-clouddebugger:v2-rev20181114-$google_clients_version", google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20181015-$google_clients_version", google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20190927-$google_clients_version", - google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20181213-$google_clients_version", + google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20191111-$google_clients_version", google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20181109-$google_clients_version", google_auth_library_credentials : "com.google.auth:google-auth-library-credentials:$google_auth_version", google_auth_library_oauth2_http : "com.google.auth:google-auth-library-oauth2-http:$google_auth_version", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java index 07d6da610bea..6f0f54d8b21e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java @@ -21,10 +21,12 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.api.client.util.DateTime; +import com.google.auto.value.AutoValue; +import com.google.protobuf.ByteString; +import com.google.pubsub.v1.PubsubMessage; import java.io.Closeable; import java.io.IOException; import java.io.Serializable; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -298,59 +300,37 @@ public static TopicPath topicPathFromName(String projectId, String topicName) { *

NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. Java * serialization is never used for non-test clients. */ - public static class OutgoingMessage implements Serializable { - /** Underlying (encoded) element. */ - public final byte[] elementBytes; + @AutoValue + public abstract static class OutgoingMessage implements Serializable { - public final Map attributes; + /** Underlying Message. May not have publish timestamp set. */ + public abstract PubsubMessage message(); /** Timestamp for element (ms since epoch). */ - public final long timestampMsSinceEpoch; + public abstract long timestampMsSinceEpoch(); /** * If using an id attribute, the record id to associate with this record's metadata so the * receiver can reject duplicates. Otherwise {@literal null}. */ - @Nullable public final String recordId; + @Nullable + public abstract String recordId(); - public OutgoingMessage( - byte[] elementBytes, - Map attributes, - long timestampMsSinceEpoch, - @Nullable String recordId) { - this.elementBytes = elementBytes; - this.attributes = attributes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - this.recordId = recordId; - } - - @Override - public String toString() { - return String.format( - "OutgoingMessage(%db, %dms)", elementBytes.length, timestampMsSinceEpoch); + public static OutgoingMessage of( + PubsubMessage message, long timestampMsSinceEpoch, @Nullable String recordId) { + return new AutoValue_PubsubClient_OutgoingMessage(message, timestampMsSinceEpoch, recordId); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; + public static OutgoingMessage of( + org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage message, + long timestampMsSinceEpoch, + @Nullable String recordId) { + PubsubMessage.Builder builder = + PubsubMessage.newBuilder().setData(ByteString.copyFrom(message.getPayload())); + if (message.getAttributeMap() != null) { + builder.putAllAttributes(message.getAttributeMap()); } - - OutgoingMessage that = (OutgoingMessage) o; - - return timestampMsSinceEpoch == that.timestampMsSinceEpoch - && Arrays.equals(elementBytes, that.elementBytes) - && Objects.equal(attributes, that.attributes) - && Objects.equal(recordId, that.recordId); - } - - @Override - public int hashCode() { - return Objects.hashCode( - Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch, recordId); + return of(builder.build(), timestampMsSinceEpoch, recordId); } } @@ -360,86 +340,35 @@ public int hashCode() { *

NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. Java * serialization is never used for non-test clients. */ - static class IncomingMessage implements Serializable { - /** Underlying (encoded) element. */ - public final byte[] elementBytes; + @AutoValue + abstract static class IncomingMessage implements Serializable { - public Map attributes; + /** Underlying Message. */ + public abstract PubsubMessage message(); /** * Timestamp for element (ms since epoch). Either Pubsub's processing time, or the custom * timestamp associated with the message. */ - public final long timestampMsSinceEpoch; + public abstract long timestampMsSinceEpoch(); /** Timestamp (in system time) at which we requested the message (ms since epoch). */ - public final long requestTimeMsSinceEpoch; + public abstract long requestTimeMsSinceEpoch(); /** Id to pass back to Pubsub to acknowledge receipt of this message. */ - public final String ackId; + public abstract String ackId(); /** Id to pass to the runner to distinguish this message from all others. */ - public final String recordId; + public abstract String recordId(); - public IncomingMessage( - byte[] elementBytes, - Map attributes, + public static IncomingMessage of( + PubsubMessage message, long timestampMsSinceEpoch, long requestTimeMsSinceEpoch, String ackId, String recordId) { - this.elementBytes = elementBytes; - this.attributes = attributes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; - this.ackId = ackId; - this.recordId = recordId; - } - - public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) { - return new IncomingMessage( - elementBytes, - attributes, - timestampMsSinceEpoch, - requestTimeMsSinceEpoch, - ackId, - recordId); - } - - @Override - public String toString() { - return String.format( - "IncomingMessage(%db, %dms)", elementBytes.length, timestampMsSinceEpoch); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - IncomingMessage that = (IncomingMessage) o; - - return timestampMsSinceEpoch == that.timestampMsSinceEpoch - && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch - && ackId.equals(that.ackId) - && recordId.equals(that.recordId) - && Arrays.equals(elementBytes, that.elementBytes) - && Objects.equal(attributes, that.attributes); - } - - @Override - public int hashCode() { - return Objects.hashCode( - Arrays.hashCode(elementBytes), - attributes, - timestampMsSinceEpoch, - requestTimeMsSinceEpoch, - ackId, - recordId); + return new AutoValue_PubsubClient_IncomingMessage( + message, timestampMsSinceEpoch, requestTimeMsSinceEpoch, ackId, recordId); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java index ae3fa0268aa1..a3b6b8d82f53 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java @@ -20,7 +20,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.auth.Credentials; -import com.google.protobuf.ByteString; import com.google.protobuf.Timestamp; import com.google.pubsub.v1.AcknowledgeRequest; import com.google.pubsub.v1.DeleteSubscriptionRequest; @@ -213,21 +212,15 @@ private SubscriberBlockingStub subscriberStub() throws IOException { public int publish(TopicPath topic, List outgoingMessages) throws IOException { PublishRequest.Builder request = PublishRequest.newBuilder().setTopic(topic.getPath()); for (OutgoingMessage outgoingMessage : outgoingMessages) { - PubsubMessage.Builder message = - PubsubMessage.newBuilder().setData(ByteString.copyFrom(outgoingMessage.elementBytes)); - - if (outgoingMessage.attributes != null) { - message.putAllAttributes(outgoingMessage.attributes); - } + PubsubMessage.Builder message = outgoingMessage.message().toBuilder(); if (timestampAttribute != null) { - message - .getMutableAttributes() - .put(timestampAttribute, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); + message.putAttributes( + timestampAttribute, String.valueOf(outgoingMessage.timestampMsSinceEpoch())); } - if (idAttribute != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) { - message.getMutableAttributes().put(idAttribute, outgoingMessage.recordId); + if (idAttribute != null && !Strings.isNullOrEmpty(outgoingMessage.recordId())) { + message.putAttributes(idAttribute, outgoingMessage.recordId()); } request.addMessages(message); @@ -259,9 +252,6 @@ public List pull( PubsubMessage pubsubMessage = message.getMessage(); @Nullable Map attributes = pubsubMessage.getAttributes(); - // Payload. - byte[] elementBytes = pubsubMessage.getData().toByteArray(); - // Timestamp. String pubsubTimestampString = null; Timestamp timestampProto = pubsubMessage.getPublishTime(); @@ -287,13 +277,8 @@ public List pull( } incomingMessages.add( - new IncomingMessage( - elementBytes, - attributes, - timestampMsSinceEpoch, - requestTimeMsSinceEpoch, - ackId, - recordId)); + IncomingMessage.of( + pubsubMessage, timestampMsSinceEpoch, requestTimeMsSinceEpoch, ackId, recordId)); } return incomingMessages; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index da5266fc8112..5f6d04469b2c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -1303,7 +1303,14 @@ public void processElement(ProcessContext c) throws IOException, SizeLimitExceed } // NOTE: The record id is always null. - output.add(new OutgoingMessage(payload, attributes, c.timestamp().getMillis(), null)); + output.add( + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(payload)) + .putAllAttributes(attributes) + .build(), + c.timestamp().getMillis(), + null)); currentOutputBytes += payload.length; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java index 136b1d2af9d0..1ae5a55673fd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java @@ -39,8 +39,10 @@ import com.google.auth.Credentials; import com.google.auth.http.HttpCredentialsAdapter; import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; +import com.google.protobuf.ByteString; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -123,8 +125,12 @@ public void close() { public int publish(TopicPath topic, List outgoingMessages) throws IOException { List pubsubMessages = new ArrayList<>(outgoingMessages.size()); for (OutgoingMessage outgoingMessage : outgoingMessages) { - PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes); + PubsubMessage pubsubMessage = + new PubsubMessage().encodeData(outgoingMessage.message().getData().toByteArray()); pubsubMessage.setAttributes(getMessageAttributes(outgoingMessage)); + if (!outgoingMessage.message().getOrderingKey().isEmpty()) { + pubsubMessage.put("orderingKey", outgoingMessage.message().getOrderingKey()); + } pubsubMessages.add(pubsubMessage); } PublishRequest request = new PublishRequest().setMessages(pubsubMessages); @@ -135,16 +141,16 @@ public int publish(TopicPath topic, List outgoingMessages) thro private Map getMessageAttributes(OutgoingMessage outgoingMessage) { Map attributes = null; - if (outgoingMessage.attributes == null) { + if (outgoingMessage.message().getAttributesMap() == null) { attributes = new TreeMap<>(); } else { - attributes = new TreeMap<>(outgoingMessage.attributes); + attributes = new TreeMap<>(outgoingMessage.message().getAttributesMap()); } if (timestampAttribute != null) { - attributes.put(timestampAttribute, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); + attributes.put(timestampAttribute, String.valueOf(outgoingMessage.timestampMsSinceEpoch())); } - if (idAttribute != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) { - attributes.put(idAttribute, outgoingMessage.recordId); + if (idAttribute != null && !Strings.isNullOrEmpty(outgoingMessage.recordId())) { + attributes.put(idAttribute, outgoingMessage.recordId()); } return attributes; } @@ -166,7 +172,12 @@ public List pull( List incomingMessages = new ArrayList<>(response.getReceivedMessages().size()); for (ReceivedMessage message : response.getReceivedMessages()) { PubsubMessage pubsubMessage = message.getMessage(); - @Nullable Map attributes = pubsubMessage.getAttributes(); + Map attributes; + if (pubsubMessage.getAttributes() != null) { + attributes = pubsubMessage.getAttributes(); + } else { + attributes = new HashMap<>(); + } // Payload. byte[] elementBytes = pubsubMessage.getData() == null ? null : pubsubMessage.decodeData(); @@ -184,7 +195,7 @@ public List pull( // Record id, if any. @Nullable String recordId = null; - if (idAttribute != null && attributes != null) { + if (idAttribute != null) { recordId = attributes.get(idAttribute); } if (Strings.isNullOrEmpty(recordId)) { @@ -192,10 +203,15 @@ public List pull( recordId = pubsubMessage.getMessageId(); } + com.google.pubsub.v1.PubsubMessage.Builder protoMessage = + com.google.pubsub.v1.PubsubMessage.newBuilder(); + protoMessage.setData(ByteString.copyFrom(elementBytes)); + protoMessage.putAllAttributes(attributes); + protoMessage.setOrderingKey( + (String) pubsubMessage.getUnknownKeys().getOrDefault("orderingKey", "")); incomingMessages.add( - new IncomingMessage( - elementBytes, - attributes, + IncomingMessage.of( + protoMessage.build(), timestampMsSinceEpoch, requestTimeMsSinceEpoch, ackId, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java index 6b20b56b0b3c..c3b915da9d16 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java @@ -309,12 +309,17 @@ public List pull( IncomingMessage incomingMessage = pendItr.next(); pendItr.remove(); IncomingMessage incomingMessageWithRequestTime = - incomingMessage.withRequestTime(requestTimeMsSinceEpoch); + IncomingMessage.of( + incomingMessage.message(), + incomingMessage.timestampMsSinceEpoch(), + requestTimeMsSinceEpoch, + incomingMessage.ackId(), + incomingMessage.recordId()); incomingMessages.add(incomingMessageWithRequestTime); STATE.pendingAckIncomingMessages.put( - incomingMessageWithRequestTime.ackId, incomingMessageWithRequestTime); + incomingMessageWithRequestTime.ackId(), incomingMessageWithRequestTime); STATE.ackDeadline.put( - incomingMessageWithRequestTime.ackId, + incomingMessageWithRequestTime.ackId(), requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000); if (incomingMessages.size() >= batchSize) { break; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index 1258d0b9743d..8be8c56c731f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -30,7 +30,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; -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.KvCoder; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.PubsubClientFactory; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; @@ -101,19 +101,18 @@ private static class OutgoingMessageCoder extends AtomicCoder { @Override public void encode(OutgoingMessage value, OutputStream outStream) throws CoderException, IOException { - ByteArrayCoder.of().encode(value.elementBytes, outStream); - ATTRIBUTES_CODER.encode(value.attributes, outStream); - BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch, outStream); - RECORD_ID_CODER.encode(value.recordId, outStream); + ProtoCoder.of(com.google.pubsub.v1.PubsubMessage.class).encode(value.message(), outStream); + BigEndianLongCoder.of().encode(value.timestampMsSinceEpoch(), outStream); + RECORD_ID_CODER.encode(value.recordId(), outStream); } @Override public OutgoingMessage decode(InputStream inStream) throws CoderException, IOException { - byte[] elementBytes = ByteArrayCoder.of().decode(inStream); - Map attributes = ATTRIBUTES_CODER.decode(inStream); + com.google.pubsub.v1.PubsubMessage message = + ProtoCoder.of(com.google.pubsub.v1.PubsubMessage.class).decode(inStream); long timestampMsSinceEpoch = BigEndianLongCoder.of().decode(inStream); @Nullable String recordId = RECORD_ID_CODER.decode(inStream); - return new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch, recordId); + return OutgoingMessage.of(message, timestampMsSinceEpoch, recordId); } } @@ -154,7 +153,6 @@ public void processElement(ProcessContext c) throws Exception { elementCounter.inc(); PubsubMessage message = c.element(); byte[] elementBytes = message.getPayload(); - Map attributes = message.getAttributeMap(); long timestampMsSinceEpoch = c.timestamp().getMillis(); @Nullable String recordId = null; @@ -175,7 +173,7 @@ public void processElement(ProcessContext c) throws Exception { c.output( KV.of( ThreadLocalRandom.current().nextInt(numShards), - new OutgoingMessage(elementBytes, attributes, timestampMsSinceEpoch, recordId))); + OutgoingMessage.of(message, timestampMsSinceEpoch, recordId))); } @Override @@ -246,7 +244,8 @@ public void processElement(ProcessContext c) throws Exception { List pubsubMessages = new ArrayList<>(publishBatchSize); int bytes = 0; for (OutgoingMessage message : c.element().getValue()) { - if (!pubsubMessages.isEmpty() && bytes + message.elementBytes.length > publishBatchBytes) { + if (!pubsubMessages.isEmpty() + && bytes + message.message().getData().size() > publishBatchBytes) { // Break large (in bytes) batches into smaller. // (We've already broken by batch size using the trigger below, though that may // run slightly over the actual PUBLISH_BATCH_SIZE. We'll consider that ok since @@ -257,7 +256,7 @@ public void processElement(ProcessContext c) throws Exception { bytes = 0; } pubsubMessages.add(message); - bytes += message.elementBytes.length; + bytes += message.message().getData().size(); } if (!pubsubMessages.isEmpty()) { // BLOCKS until published. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index d8abfe17c407..230161ca1433 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -727,18 +727,18 @@ private void pull() throws IOException { // Capture the received messages. for (PubsubClient.IncomingMessage incomingMessage : receivedMessages) { notYetRead.add(incomingMessage); - notYetReadBytes += incomingMessage.elementBytes.length; + notYetReadBytes += incomingMessage.message().getData().size(); inFlight.put( - incomingMessage.ackId, + incomingMessage.ackId(), new InFlightState(requestTimeMsSinceEpoch, deadlineMsSinceEpoch)); numReceived++; numReceivedRecently.add(requestTimeMsSinceEpoch, 1L); minReceivedTimestampMsSinceEpoch.add( - requestTimeMsSinceEpoch, incomingMessage.timestampMsSinceEpoch); + requestTimeMsSinceEpoch, incomingMessage.timestampMsSinceEpoch()); maxReceivedTimestampMsSinceEpoch.add( - requestTimeMsSinceEpoch, incomingMessage.timestampMsSinceEpoch); + requestTimeMsSinceEpoch, incomingMessage.timestampMsSinceEpoch()); minUnreadTimestampMsSinceEpoch.add( - requestTimeMsSinceEpoch, incomingMessage.timestampMsSinceEpoch); + requestTimeMsSinceEpoch, incomingMessage.timestampMsSinceEpoch()); } } @@ -837,7 +837,7 @@ public boolean advance() throws IOException { if (current != null) { // Current is consumed. It can no longer contribute to holding back the watermark. - minUnreadTimestampMsSinceEpoch.remove(current.requestTimeMsSinceEpoch); + minUnreadTimestampMsSinceEpoch.remove(current.requestTimeMsSinceEpoch()); current = null; } @@ -864,18 +864,18 @@ public boolean advance() throws IOException { // Try again later. return false; } - notYetReadBytes -= current.elementBytes.length; + notYetReadBytes -= current.message().getData().size(); checkState(notYetReadBytes >= 0); long nowMsSinceEpoch = now(); - numReadBytes.add(nowMsSinceEpoch, current.elementBytes.length); - minReadTimestampMsSinceEpoch.add(nowMsSinceEpoch, current.timestampMsSinceEpoch); - if (current.timestampMsSinceEpoch < lastWatermarkMsSinceEpoch) { + numReadBytes.add(nowMsSinceEpoch, current.message().getData().size()); + minReadTimestampMsSinceEpoch.add(nowMsSinceEpoch, current.timestampMsSinceEpoch()); + if (current.timestampMsSinceEpoch() < lastWatermarkMsSinceEpoch) { numLateMessages.add(nowMsSinceEpoch, 1L); } // Current message can be considered 'read' and will be persisted by the next // checkpoint. So it is now safe to ACK back to Pubsub. - safeToAckIds.add(current.ackId); + safeToAckIds.add(current.ackId()); return true; } @@ -884,7 +884,10 @@ public PubsubMessage getCurrent() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } - return new PubsubMessage(current.elementBytes, current.attributes, current.recordId); + return new PubsubMessage( + current.message().getData().toByteArray(), + current.message().getAttributesMap(), + current.recordId()); } @Override @@ -892,7 +895,7 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } - return new Instant(current.timestampMsSinceEpoch); + return new Instant(current.timestampMsSinceEpoch()); } @Override @@ -900,7 +903,7 @@ public byte[] getCurrentRecordId() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } - return current.recordId.getBytes(StandardCharsets.UTF_8); + return current.recordId().getBytes(StandardCharsets.UTF_8); } /** @@ -984,7 +987,7 @@ public PubsubCheckpoint getCheckpointMark() { List snapshotSafeToAckIds = Lists.newArrayList(safeToAckIds); List snapshotNotYetReadIds = new ArrayList<>(notYetRead.size()); for (PubsubClient.IncomingMessage incomingMessage : notYetRead) { - snapshotNotYetReadIds.add(incomingMessage.ackId); + snapshotNotYetReadIds.add(incomingMessage.ackId()); } if (outer.subscriptionPath == null) { // need to include the subscription in case we resume, as it's not stored in the source. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java index 1e75d4377abf..e1e8711587a6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java @@ -22,12 +22,14 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import com.google.protobuf.ByteString; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeoutException; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.ProjectPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; @@ -205,11 +207,16 @@ public List pull(int maxBatchSize) throws IOException { if (!messages.isEmpty()) { pubsub.acknowledge( subscriptionPath, - messages.stream().map(msg -> msg.ackId).collect(ImmutableList.toImmutableList())); + messages.stream().map(IncomingMessage::ackId).collect(ImmutableList.toImmutableList())); } return messages.stream() - .map(msg -> new PubsubMessage(msg.elementBytes, msg.attributes, msg.recordId)) + .map( + msg -> + new PubsubMessage( + msg.message().getData().toByteArray(), + msg.message().getAttributesMap(), + msg.recordId())) .collect(ImmutableList.toImmutableList()); } @@ -292,7 +299,12 @@ public void checkIfAnySubscriptionExists(String project, Duration timeoutDuratio } private PubsubClient.OutgoingMessage toOutgoingMessage(PubsubMessage message) { - return new PubsubClient.OutgoingMessage( - message.getPayload(), message.getAttributeMap(), DateTime.now().getMillis(), null); + return PubsubClient.OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(message.getPayload())) + .putAllAttributes(message.getAttributeMap()) + .build(), + DateTime.now().getMillis(), + null); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java index f4f8b18ffc54..de4a71557dd1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.pubsub; -import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.stream.Collectors.toList; import static org.apache.beam.sdk.io.gcp.pubsub.TestPubsub.createTopicName; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; @@ -30,6 +29,7 @@ import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.state.BagState; @@ -251,7 +251,7 @@ private String pollForResultForDuration( try { signal = pubsub.pull(DateTime.now().getMillis(), signalSubscriptionPath, 1, false); pubsub.acknowledge( - signalSubscriptionPath, signal.stream().map(m -> m.ackId).collect(toList())); + signalSubscriptionPath, signal.stream().map(IncomingMessage::ackId).collect(toList())); break; } catch (StatusRuntimeException e) { if (!Status.DEADLINE_EXCEEDED.equals(e.getStatus())) { @@ -271,7 +271,7 @@ private String pollForResultForDuration( signalSubscriptionPath, duration.getStandardSeconds())); } - return new String(signal.get(0).elementBytes, UTF_8); + return signal.get(0).message().getData().toStringUtf8(); } private void sleep(long t) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java index 50f6548059aa..bc146ce63c23 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java @@ -67,6 +67,7 @@ public void testGcpApiSurface() throws Exception { classesInPackage("com.google.cloud.bigtable.config"), classesInPackage("com.google.cloud.bigtable.data"), classesInPackage("com.google.spanner.v1"), + classesInPackage("com.google.pubsub.v1"), Matchers.equalTo(com.google.api.gax.rpc.ApiException.class), Matchers.>equalTo(com.google.api.gax.longrunning.OperationFuture.class), Matchers.>equalTo(com.google.api.gax.longrunning.OperationSnapshot.class), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java index 7c53170ce2d6..4dd719b5de1d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java @@ -142,11 +142,11 @@ public void pull(PullRequest request, StreamObserver responseObser List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); assertEquals(1, acutalMessages.size()); IncomingMessage actualMessage = acutalMessages.get(0); - assertEquals(ACK_ID, actualMessage.ackId); - assertEquals(DATA, new String(actualMessage.elementBytes, StandardCharsets.UTF_8)); - assertEquals(RECORD_ID, actualMessage.recordId); - assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch); - assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch); + assertEquals(ACK_ID, actualMessage.ackId()); + assertEquals(DATA, actualMessage.message().getData().toStringUtf8()); + assertEquals(RECORD_ID, actualMessage.recordId()); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch()); + assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch()); assertEquals(expectedRequest, Iterables.getOnlyElement(requestsReceived)); } finally { server.shutdownNow(); @@ -187,8 +187,13 @@ public void publish( InProcessServerBuilder.forName(channelName).addService(publisherImplBase).build().start(); try { OutgoingMessage actualMessage = - new OutgoingMessage( - DATA.getBytes(StandardCharsets.UTF_8), ATTRIBUTES, MESSAGE_TIME, RECORD_ID); + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .putAllAttributes(ATTRIBUTES) + .build(), + MESSAGE_TIME, + RECORD_ID); int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); assertEquals(1, n); assertEquals(expectedRequest, Iterables.getOnlyElement(requestsReceived)); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index 65b89a79fbe3..0dc910f2c701 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -29,6 +29,7 @@ import static org.junit.Assert.assertThat; import com.google.api.client.util.Clock; +import com.google.protobuf.ByteString; import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; @@ -391,9 +392,10 @@ private void setupTestClient(List inputs, Coder coder) { }) .map( ba -> - new IncomingMessage( - ba, - null, + IncomingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(ba)) + .build(), 1234L, 0, UUID.randomUUID().toString(), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java index f7fc0f339fd8..aad9729699a8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java @@ -34,6 +34,7 @@ import com.google.api.services.pubsub.model.ReceivedMessage; import com.google.api.services.pubsub.model.Subscription; import com.google.api.services.pubsub.model.Topic; +import com.google.protobuf.ByteString; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Collections; @@ -73,6 +74,7 @@ public class PubsubJsonClientTest { private static final String DATA = "testData"; private static final String RECORD_ID = "testRecordId"; private static final String ACK_ID = "testAckId"; + private static final String ORDERING_KEY = "testOrderingKey"; @Before public void setup() { @@ -98,7 +100,40 @@ public void pullOneMessage() throws IOException { .setPublishTime(String.valueOf(PUB_TIME)) .setAttributes( ImmutableMap.of( - TIMESTAMP_ATTRIBUTE, String.valueOf(MESSAGE_TIME), ID_ATTRIBUTE, RECORD_ID)); + TIMESTAMP_ATTRIBUTE, String.valueOf(MESSAGE_TIME), ID_ATTRIBUTE, RECORD_ID)) + .set("orderingKey", ORDERING_KEY); + ReceivedMessage expectedReceivedMessage = + new ReceivedMessage().setMessage(expectedPubsubMessage).setAckId(ACK_ID); + PullResponse expectedResponse = + new PullResponse().setReceivedMessages(ImmutableList.of(expectedReceivedMessage)); + when((Object) + (mockPubsub + .projects() + .subscriptions() + .pull(expectedSubscription, expectedRequest) + .execute())) + .thenReturn(expectedResponse); + List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); + assertEquals(1, acutalMessages.size()); + IncomingMessage actualMessage = acutalMessages.get(0); + assertEquals(ACK_ID, actualMessage.ackId()); + assertEquals(DATA, actualMessage.message().getData().toStringUtf8()); + assertEquals(RECORD_ID, actualMessage.recordId()); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch()); + assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch()); + assertEquals(ORDERING_KEY, actualMessage.message().getOrderingKey()); + } + + @Test + public void pullOneMessageEmptyAttributes() throws IOException { + client = new PubsubJsonClient(null, null, mockPubsub); + String expectedSubscription = SUBSCRIPTION.getPath(); + PullRequest expectedRequest = new PullRequest().setReturnImmediately(true).setMaxMessages(10); + PubsubMessage expectedPubsubMessage = + new PubsubMessage() + .setMessageId(MESSAGE_ID) + .encodeData(DATA.getBytes(StandardCharsets.UTF_8)) + .setPublishTime(String.valueOf(PUB_TIME)); ReceivedMessage expectedReceivedMessage = new ReceivedMessage().setMessage(expectedPubsubMessage).setAckId(ACK_ID); PullResponse expectedResponse = @@ -113,11 +148,10 @@ public void pullOneMessage() throws IOException { List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); assertEquals(1, acutalMessages.size()); IncomingMessage actualMessage = acutalMessages.get(0); - assertEquals(ACK_ID, actualMessage.ackId); - assertEquals(DATA, new String(actualMessage.elementBytes, StandardCharsets.UTF_8)); - assertEquals(RECORD_ID, actualMessage.recordId); - assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch); - assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch); + assertEquals(ACK_ID, actualMessage.ackId()); + assertEquals(DATA, actualMessage.message().getData().toStringUtf8()); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch()); + assertEquals(PUB_TIME, actualMessage.timestampMsSinceEpoch()); } @Test @@ -146,7 +180,7 @@ public void pullOneMessageWithNoData() throws IOException { List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); assertEquals(1, acutalMessages.size()); IncomingMessage actualMessage = acutalMessages.get(0); - assertArrayEquals(new byte[0], actualMessage.elementBytes); + assertArrayEquals(new byte[0], actualMessage.message().getData().toByteArray()); } @Test @@ -160,7 +194,8 @@ public void publishOneMessage() throws IOException { .put(TIMESTAMP_ATTRIBUTE, String.valueOf(MESSAGE_TIME)) .put(ID_ATTRIBUTE, RECORD_ID) .put("k", "v") - .build()); + .build()) + .set("orderingKey", ORDERING_KEY); PublishRequest expectedRequest = new PublishRequest().setMessages(ImmutableList.of(expectedPubsubMessage)); PublishResponse expectedResponse = @@ -171,7 +206,14 @@ public void publishOneMessage() throws IOException { Map attrs = new HashMap<>(); attrs.put("k", "v"); OutgoingMessage actualMessage = - new OutgoingMessage(DATA.getBytes(StandardCharsets.UTF_8), attrs, MESSAGE_TIME, RECORD_ID); + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .putAllAttributes(attrs) + .setOrderingKey(ORDERING_KEY) + .build(), + MESSAGE_TIME, + RECORD_ID); int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); assertEquals(1, n); } @@ -195,8 +237,12 @@ public void publishOneMessageWithOnlyTimestampAndIdAttributes() throws IOExcepti (mockPubsub.projects().topics().publish(expectedTopic, expectedRequest).execute())) .thenReturn(expectedResponse); OutgoingMessage actualMessage = - new OutgoingMessage( - DATA.getBytes(StandardCharsets.UTF_8), ImmutableMap.of(), MESSAGE_TIME, RECORD_ID); + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .build(), + MESSAGE_TIME, + RECORD_ID); int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); assertEquals(1, n); } @@ -222,7 +268,13 @@ public void publishOneMessageWithNoTimestampOrIdAttribute() throws IOException { Map attrs = new HashMap<>(); attrs.put("k", "v"); OutgoingMessage actualMessage = - new OutgoingMessage(DATA.getBytes(StandardCharsets.UTF_8), attrs, MESSAGE_TIME, RECORD_ID); + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .putAllAttributes(attrs) + .build(), + MESSAGE_TIME, + RECORD_ID); int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); assertEquals(1, n); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java index 2b698f0708cd..6b920e839c59 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java @@ -20,8 +20,9 @@ import static org.junit.Assert.assertEquals; import com.google.api.client.util.Clock; +import com.google.protobuf.ByteString; +import com.google.pubsub.v1.PubsubMessage; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.List; import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.IncomingMessage; @@ -54,9 +55,8 @@ public void pullOneMessage() throws IOException { final AtomicLong now = new AtomicLong(); Clock clock = now::get; IncomingMessage expectedIncomingMessage = - new IncomingMessage( - DATA.getBytes(StandardCharsets.UTF_8), - null, + IncomingMessage.of( + PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(DATA)).build(), MESSAGE_TIME, REQ_TIME, ACK_ID, @@ -75,7 +75,14 @@ public void pullOneMessage() throws IOException { client.advance(); incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true); assertEquals(1, incomingMessages.size()); - assertEquals(expectedIncomingMessage.withRequestTime(now.get()), incomingMessages.get(0)); + assertEquals( + IncomingMessage.of( + expectedIncomingMessage.message(), + expectedIncomingMessage.timestampMsSinceEpoch(), + now.get(), + expectedIncomingMessage.ackId(), + expectedIncomingMessage.recordId()), + incomingMessages.get(0)); now.addAndGet(10 * 1000); client.advance(); // Extend ack @@ -85,7 +92,14 @@ public void pullOneMessage() throws IOException { client.advance(); incomingMessages = client.pull(now.get(), SUBSCRIPTION, 1, true); assertEquals(1, incomingMessages.size()); - assertEquals(expectedIncomingMessage.withRequestTime(now.get()), incomingMessages.get(0)); + assertEquals( + IncomingMessage.of( + expectedIncomingMessage.message(), + expectedIncomingMessage.timestampMsSinceEpoch(), + now.get(), + expectedIncomingMessage.ackId(), + expectedIncomingMessage.recordId()), + incomingMessages.get(0)); // Extend ack client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20); // Ack @@ -99,7 +113,10 @@ public void pullOneMessage() throws IOException { @Test public void publishOneMessage() throws IOException { OutgoingMessage expectedOutgoingMessage = - new OutgoingMessage(DATA.getBytes(StandardCharsets.UTF_8), null, MESSAGE_TIME, MESSAGE_ID); + OutgoingMessage.of( + PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(DATA)).build(), + MESSAGE_TIME, + MESSAGE_ID); try (PubsubTestClientFactory factory = PubsubTestClient.createFactoryForPublish( TOPIC, Sets.newHashSet(expectedOutgoingMessage), ImmutableList.of())) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java index f588e05b1ee1..f8cd86ee463c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.pubsub; +import com.google.protobuf.ByteString; import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; @@ -83,8 +84,12 @@ private String getRecordId(String data) { @Test public void saneCoder() throws Exception { OutgoingMessage message = - new OutgoingMessage( - DATA.getBytes(StandardCharsets.UTF_8), ImmutableMap.of(), TIMESTAMP, getRecordId(DATA)); + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .build(), + TIMESTAMP, + getRecordId(DATA)); CoderProperties.coderDecodeEncodeEqual(PubsubUnboundedSink.CODER, message); CoderProperties.coderSerializable(PubsubUnboundedSink.CODER); } @@ -93,8 +98,13 @@ public void saneCoder() throws Exception { public void sendOneMessage() throws IOException { List outgoing = ImmutableList.of( - new OutgoingMessage( - DATA.getBytes(StandardCharsets.UTF_8), ATTRIBUTES, TIMESTAMP, getRecordId(DATA))); + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .putAllAttributes(ATTRIBUTES) + .build(), + TIMESTAMP, + getRecordId(DATA))); int batchSize = 1; int batchBytes = 1; try (PubsubTestClientFactory factory = @@ -121,9 +131,10 @@ public void sendOneMessage() throws IOException { public void sendOneMessageWithoutAttributes() throws IOException { List outgoing = ImmutableList.of( - new OutgoingMessage( - DATA.getBytes(StandardCharsets.UTF_8), - null /* attributes */, + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .build(), TIMESTAMP, getRecordId(DATA))); try (PubsubTestClientFactory factory = @@ -157,9 +168,10 @@ public void sendMoreThanOneBatchByNumMessages() throws IOException { for (int i = 0; i < batchSize * 10; i++) { String str = String.valueOf(i); outgoing.add( - new OutgoingMessage( - str.getBytes(StandardCharsets.UTF_8), - ImmutableMap.of(), + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(str)) + .build(), TIMESTAMP, getRecordId(str))); data.add(str); @@ -198,9 +210,10 @@ public void sendMoreThanOneBatchByByteSize() throws IOException { } String str = sb.toString(); outgoing.add( - new OutgoingMessage( - str.getBytes(StandardCharsets.UTF_8), - ImmutableMap.of(), + OutgoingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(str)) + .build(), TIMESTAMP, getRecordId(str))); data.add(str); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java index b2dacf07942d..43ecbdc5821e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java @@ -31,6 +31,7 @@ import static org.junit.Assert.assertTrue; import com.google.api.client.util.Clock; +import com.google.protobuf.ByteString; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -100,8 +101,14 @@ private void setupOneMessage(Iterable incoming) { private void setupOneMessage() { setupOneMessage( ImmutableList.of( - new IncomingMessage( - DATA.getBytes(StandardCharsets.UTF_8), null, TIMESTAMP, 0, ACK_ID, RECORD_ID))); + IncomingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(DATA)) + .build(), + TIMESTAMP, + 0, + ACK_ID, + RECORD_ID))); } @After @@ -219,8 +226,14 @@ public void multipleReaders() throws IOException { String data = String.format("data_%d", i); String ackid = String.format("ackid_%d", i); incoming.add( - new IncomingMessage( - data.getBytes(StandardCharsets.UTF_8), null, TIMESTAMP, 0, ackid, RECORD_ID)); + IncomingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(data)) + .build(), + TIMESTAMP, + 0, + ackid, + RECORD_ID)); } setupOneMessage(incoming); PubsubReader reader = primSource.createReader(p.getOptions(), null); @@ -279,9 +292,10 @@ public void readManyMessages() throws IOException { String recid = String.format("recordid_%d", messageNum); String ackId = String.format("ackid_%d", messageNum); incoming.add( - new IncomingMessage( - data.getBytes(StandardCharsets.UTF_8), - null, + IncomingMessage.of( + com.google.pubsub.v1.PubsubMessage.newBuilder() + .setData(ByteString.copyFromUtf8(data)) + .build(), messageNumToTimestamp(messageNum), 0, ackId,