Merge pull request #10171: [BEAM-8470] Enable custom window tests in new spark runner validates runner tests.
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 07ee590..1c9c13b 100644
--- a/model/fn-execution/src/main/proto/beam_fn_api.proto
+++ b/model/fn-execution/src/main/proto/beam_fn_api.proto
@@ -637,6 +637,27 @@
bytes key = 1;
}
+ // Represents a request for the values associated with a specified window
+ // in a PCollection. See
+ // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ // details.
+ //
+ // Can only be used to perform StateGetRequests on side inputs of the URN
+ // beam:side_input:iterable:v1.
+ //
+ // For a PCollection<V>, the response data stream will be a concatenation
+ // of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
+ // for further details.
+ message IterableSideInput {
+ // (Required) The id of the PTransform containing a side input.
+ string transform_id = 1;
+ // (Required) The id of the side input.
+ string side_input_id = 2;
+ // (Required) The window (after mapping the currently executing elements
+ // window into the side input windows domain) encoded in a nested context.
+ bytes window = 3;
+ }
+
// Represents a request for the values associated with a specified user key
// and window in a PCollection. See
// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
@@ -661,6 +682,27 @@
bytes key = 4;
}
+ // Represents a request for the keys associated with a specified window in a PCollection. See
+ // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+ // details.
+ //
+ // Can only be used to perform StateGetRequests on side inputs of the URN
+ // beam:side_input:multimap:v1.
+ //
+ // For a PCollection<KV<K, V>>, the response data stream will be a
+ // concatenation of all K's associated with the specified window. See
+ // https://s.apache.org/beam-fn-api-send-and-receive-data for further
+ // details.
+ message MultimapKeysSideInput {
+ // (Required) The id of the PTransform containing a side input.
+ string transform_id = 1;
+ // (Required) The id of the side input.
+ string side_input_id = 2;
+ // (Required) The window (after mapping the currently executing elements
+ // window into the side input windows domain) encoded in a nested context.
+ bytes window = 3;
+ }
+
message BagUserState {
// (Required) The id of the PTransform containing user state.
string transform_id = 1;
@@ -673,33 +715,13 @@
bytes key = 4;
}
- // Represents a request for the values associated with a specified window
- // in a PCollection. See
- // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
- // details.
- //
- // Can only be used to perform StateGetRequests on side inputs of the URN
- // beam:side_input:iterable:v1 and beam:side_input:multimap:v1.
- //
- // For a PCollection<V>, the response data stream will be a concatenation
- // of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
- // for further details.
- message IterableSideInput {
- // (Required) The id of the PTransform containing a side input.
- string transform_id = 1;
- // (Required) The id of the side input.
- string side_input_id = 2;
- // (Required) The window (after mapping the currently executing elements
- // window into the side input windows domain) encoded in a nested context.
- bytes window = 3;
- }
-
// (Required) One of the following state keys must be set.
oneof type {
Runner runner = 1;
MultimapSideInput multimap_side_input = 2;
BagUserState bag_user_state = 3;
IterableSideInput iterable_side_input = 4;
+ MultimapKeysSideInput multimap_keys_side_input = 5;
// TODO: represent a state key for user map state
}
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java
index e00f330..4ed3c06 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java
@@ -17,16 +17,34 @@
*/
package org.apache.beam.runners.core;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.Materializations;
import org.apache.beam.sdk.transforms.Materializations.MultimapView;
import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ArrayListMultimap;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimaps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
/** An in-memory representation of {@link MultimapView}. */
public class InMemoryMultimapSideInputView<K, V> implements Materializations.MultimapView<K, V> {
+ /** An empty {@link MultimapView}. */
+ private static final MultimapView EMPTY =
+ new MultimapView() {
+ @Override
+ public Iterable get() {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public Iterable get(@Nullable Object k) {
+ return Collections.emptyList();
+ }
+ };
/**
* Creates a {@link MultimapView} from the provided values. The provided {@link Coder} is used to
@@ -34,27 +52,45 @@
*/
public static <K, V> MultimapView<K, V> fromIterable(
Coder<K> keyCoder, Iterable<KV<K, V>> values) {
- // We specifically use an array list multimap to allow for:
- // * null keys
- // * null values
- // * duplicate values
- Multimap<Object, Object> multimap = ArrayListMultimap.create();
+ // We specifically use a hash map to allow for null keys
+ Map<Object, KV<K, List<V>>> data = new HashMap<>();
+
for (KV<K, V> value : values) {
- multimap.put(keyCoder.structuralValue(value.getKey()), value.getValue());
+ KV<K, List<V>> keyedValues =
+ data.computeIfAbsent(
+ keyCoder.structuralValue(value.getKey()),
+ o -> KV.of(value.getKey(), new ArrayList<>()));
+ keyedValues.getValue().add(value.getValue());
}
- return new InMemoryMultimapSideInputView(keyCoder, Multimaps.unmodifiableMultimap(multimap));
+ return new InMemoryMultimapSideInputView(keyCoder, data);
+ }
+
+ /** Returns an empty {@link MultimapView}. */
+ public static <K, V> MultimapView<K, V> empty() {
+ return EMPTY;
}
private final Coder<K> keyCoder;
- private final Multimap<Object, V> structuralKeyToValuesMap;
+ private final Map<Object, KV<K, List<V>>> structuralKeyToValuesMap;
- private InMemoryMultimapSideInputView(Coder<K> keyCoder, Multimap<Object, V> data) {
+ private InMemoryMultimapSideInputView(Coder<K> keyCoder, Map<Object, KV<K, List<V>>> data) {
this.keyCoder = keyCoder;
this.structuralKeyToValuesMap = data;
}
@Override
+ public Iterable<K> get() {
+ return Iterables.unmodifiableIterable(
+ FluentIterable.from(structuralKeyToValuesMap.values())
+ .transform(kListKV -> kListKV.getKey()));
+ }
+
+ @Override
public Iterable<V> get(K k) {
- return structuralKeyToValuesMap.get(keyCoder.structuralValue(k));
+ KV<K, List<V>> records = structuralKeyToValuesMap.get(keyCoder.structuralValue(k));
+ if (records == null) {
+ return Collections.emptyList();
+ }
+ return records.getValue();
}
}
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryMultimapSideInputViewTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryMultimapSideInputViewTest.java
index 4de5c7f..313b90b 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryMultimapSideInputViewTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryMultimapSideInputViewTest.java
@@ -17,13 +17,19 @@
*/
package org.apache.beam.runners.core;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.Assert.assertEquals;
+import java.util.Arrays;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.transforms.Materializations.MultimapView;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.hamcrest.Description;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeMatcher;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
@@ -40,6 +46,29 @@
assertEquals(view.get(new byte[] {0x00}), ImmutableList.of(0));
assertEquals(view.get(new byte[] {0x01}), ImmutableList.of(1));
assertEquals(view.get(new byte[] {0x02}), ImmutableList.of());
+ assertThat(
+ view.get(),
+ Matchers.containsInAnyOrder(
+ new ByteArrayMatcher(new byte[] {0x00}), new ByteArrayMatcher(new byte[] {0x01})));
+ }
+
+ /** A matcher for byte[]s since primitive arrays only support reference equality by default. */
+ private static class ByteArrayMatcher extends TypeSafeMatcher<byte[]> {
+ private final byte[] expected;
+
+ public ByteArrayMatcher(byte[] expected) {
+ this.expected = expected;
+ }
+
+ @Override
+ protected boolean matchesSafely(byte[] item) {
+ return Arrays.equals(expected, item);
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendValue(expected);
+ }
}
@Test
@@ -51,5 +80,6 @@
assertEquals(view.get("A"), ImmutableList.of("a1", "a2"));
assertEquals(view.get("B"), ImmutableList.of("b1"));
assertEquals(view.get("C"), ImmutableList.of());
+ assertThat(view.get(), containsInAnyOrder("A", "B"));
}
}
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
index 0b77a69..f53dc61 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
@@ -181,7 +181,9 @@
EnumMap<StateKey.TypeCase, StateRequestHandler> handlerMap =
new EnumMap<>(StateKey.TypeCase.class);
+ handlerMap.put(StateKey.TypeCase.ITERABLE_SIDE_INPUT, sideInputHandler);
handlerMap.put(StateKey.TypeCase.MULTIMAP_SIDE_INPUT, sideInputHandler);
+ handlerMap.put(StateKey.TypeCase.MULTIMAP_KEYS_SIDE_INPUT, sideInputHandler);
handlerMap.put(StateKey.TypeCase.BAG_USER_STATE, userStateHandler);
return StateRequestHandlers.delegateBasedUponType(handlerMap);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
index 65d833d..34530c3 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
@@ -20,10 +20,10 @@
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.util.Collections;
import java.util.HashMap;
import java.util.Map;
import javax.annotation.Nullable;
+import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
import org.apache.beam.runners.core.SideInputReader;
import org.apache.beam.sdk.transforms.Materializations;
import org.apache.beam.sdk.transforms.Materializations.MultimapView;
@@ -36,9 +36,6 @@
/** A {@link SideInputReader} for the Flink Batch Runner. */
public class FlinkSideInputReader implements SideInputReader {
- /** A {@link MultimapView} which always returns an empty iterable. */
- private static final MultimapView EMPTY_MULTMAP_VIEW = o -> Collections.EMPTY_LIST;
-
private final Map<TupleTag<?>, WindowingStrategy<?, ?>> sideInputs;
private RuntimeContext runtimeContext;
@@ -75,7 +72,7 @@
T result = sideInputs.get(window);
if (result == null) {
ViewFn<MultimapView, T> viewFn = (ViewFn<MultimapView, T>) view.getViewFn();
- result = viewFn.apply(EMPTY_MULTMAP_VIEW);
+ result = viewFn.apply(InMemoryMultimapSideInputView.empty());
}
return result;
}
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java
index a4016e2..5a7f1d9 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java
@@ -20,18 +20,16 @@
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.ByteArrayOutputStream;
-import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.Map;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.graph.ExecutableStage;
import org.apache.beam.runners.core.construction.graph.SideInputReference;
import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler;
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandlerFactory;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
@@ -87,77 +85,75 @@
}
@Override
- public <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput(
- String transformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder) {
+ public <V, W extends BoundedWindow> IterableSideInputHandler<V, W> forIterableSideInput(
+ String transformId, String sideInputId, Coder<V> elementCoder, Coder<W> windowCoder) {
PCollectionView collectionNode =
sideInputToCollection.get(
SideInputId.newBuilder().setTransformId(transformId).setLocalName(sideInputId).build());
checkArgument(collectionNode != null, "No side input for %s/%s", transformId, sideInputId);
- if (PTransformTranslation.ITERABLE_SIDE_INPUT.equals(accessPattern.getUrn())) {
- @SuppressWarnings("unchecked") // T == V
- Coder<V> outputCoder = (Coder<V>) elementCoder;
- return forIterableSideInput(collectionNode, outputCoder);
- } else if (PTransformTranslation.MULTIMAP_SIDE_INPUT.equals(accessPattern.getUrn())) {
- @SuppressWarnings("unchecked") // T == KV<?, V>
- KvCoder<?, V> kvCoder = (KvCoder<?, V>) elementCoder;
- return forMultimapSideInput(collectionNode, kvCoder.getKeyCoder(), kvCoder.getValueCoder());
- } else {
- throw new IllegalArgumentException(
- String.format("Unknown side input access pattern: %s", accessPattern));
- }
- }
-
- private <T, W extends BoundedWindow> SideInputHandler<T, W> forIterableSideInput(
- PCollectionView<?> collection, Coder<T> elementCoder) {
-
- return new SideInputHandler<T, W>() {
+ return new IterableSideInputHandler<V, W>() {
@Override
- public Iterable<T> get(byte[] key, W window) {
+ public Iterable<V> get(W window) {
return checkNotNull(
- (Iterable<T>) runnerHandler.getIterable(collection, window),
+ (Iterable<V>) runnerHandler.getIterable(collectionNode, window),
"Element processed by SDK before side input is ready");
}
@Override
- public Coder<T> resultCoder() {
+ public Coder<V> elementCoder() {
return elementCoder;
}
};
}
- private <K, V, W extends BoundedWindow> SideInputHandler<V, W> forMultimapSideInput(
- PCollectionView<?> collection, Coder<K> keyCoder, Coder<V> valueCoder) {
+ @Override
+ public <K, V, W extends BoundedWindow> MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String transformId, String sideInputId, KvCoder<K, V> elementCoder, Coder<W> windowCoder) {
- return new SideInputHandler<V, W>() {
+ PCollectionView collectionNode =
+ sideInputToCollection.get(
+ SideInputId.newBuilder().setTransformId(transformId).setLocalName(sideInputId).build());
+ checkArgument(collectionNode != null, "No side input for %s/%s", transformId, sideInputId);
+
+ return new MultimapSideInputHandler<K, V, W>() {
@Override
- public Iterable<V> get(byte[] key, W window) {
+ public Iterable<V> get(K key, W window) {
Iterable<KV<K, V>> values =
- (Iterable<KV<K, V>>) runnerHandler.getIterable(collection, window);
+ (Iterable<KV<K, V>>) runnerHandler.getIterable(collectionNode, window);
+ Object structuralK = keyCoder().structuralValue(key);
ArrayList<V> result = new ArrayList<>();
// find values for the given key
for (KV<K, V> kv : values) {
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- try {
- keyCoder.encode(kv.getKey(), bos);
- if (Arrays.equals(key, bos.toByteArray())) {
- result.add(kv.getValue());
- }
- } catch (IOException ex) {
- throw new RuntimeException(ex);
+ if (structuralK.equals(keyCoder().structuralValue(kv.getKey()))) {
+ result.add(kv.getValue());
}
}
- return result;
+ return Collections.unmodifiableList(result);
}
@Override
- public Coder<V> resultCoder() {
- return valueCoder;
+ public Iterable<K> get(W window) {
+ Iterable<KV<K, V>> values =
+ (Iterable<KV<K, V>>) runnerHandler.getIterable(collectionNode, window);
+
+ Map<Object, K> result = new HashMap<>();
+ // find all keys
+ for (KV<K, V> kv : values) {
+ result.putIfAbsent(keyCoder().structuralValue(kv.getKey()), kv.getKey());
+ }
+ return Collections.unmodifiableCollection(result.values());
+ }
+
+ @Override
+ public Coder<K> keyCoder() {
+ return elementCoder.getKeyCoder();
+ }
+
+ @Override
+ public Coder<V> valueCoder() {
+ return elementCoder.getValueCoder();
}
};
}
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 11b2af2..ba951b1 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
@@ -255,7 +255,9 @@
}
EnumMap<TypeCase, StateRequestHandler> handlerMap = new EnumMap<>(TypeCase.class);
+ handlerMap.put(TypeCase.ITERABLE_SIDE_INPUT, sideInputStateHandler);
handlerMap.put(TypeCase.MULTIMAP_SIDE_INPUT, sideInputStateHandler);
+ handlerMap.put(TypeCase.MULTIMAP_KEYS_SIDE_INPUT, sideInputStateHandler);
handlerMap.put(TypeCase.BAG_USER_STATE, userStateRequestHandler);
return StateRequestHandlers.delegateBasedUponType(handlerMap);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java
index b1ea344..f101d85 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java
@@ -46,6 +46,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import javax.annotation.Nonnull;
+import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
import org.apache.beam.runners.core.SideInputReader;
import org.apache.beam.runners.dataflow.internal.IsmFormat;
import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecord;
@@ -585,7 +586,7 @@
List<IsmReader<V>> readers = (List) tagToIsmReaderMap.get(tag);
if (readers.isEmpty()) {
- return k -> Collections.emptyList();
+ return InMemoryMultimapSideInputView.empty();
}
return new IsmMultimapView<>(window, readers);
@@ -606,6 +607,11 @@
}
@Override
+ public Iterable<K> get() {
+ throw new UnsupportedOperationException("TODO: Support enumerating the keys.");
+ }
+
+ @Override
public Iterable<V> get(K k) {
try {
return new ListOverReaderIterators<>(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
index 34dc4b1..d0d79f3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
@@ -20,12 +20,12 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
import java.util.Map;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.SideInputReader;
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.Materializations;
@@ -60,12 +60,18 @@
}
@Override
- public <T, V, W extends BoundedWindow> StateRequestHandlers.SideInputHandler<V, W> forSideInput(
- String pTransformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder) {
+ public <V, W extends BoundedWindow> IterableSideInputHandler<V, W> forIterableSideInput(
+ String pTransformId, String sideInputId, Coder<V> elementCoder, Coder<W> windowCoder) {
+ throw new UnsupportedOperationException(
+ String.format(
+ "The %s does not support handling sides inputs for PTransform %s with side "
+ + "input id %s.",
+ DataflowSideInputHandlerFactory.class.getSimpleName(), pTransformId, sideInputId));
+ }
+
+ @Override
+ public <K, V, W extends BoundedWindow> MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String pTransformId, String sideInputId, KvCoder<K, V> elementCoder, Coder<W> windowCoder) {
checkArgument(
pTransformId != null && pTransformId.length() > 0, "Expect a valid PTransform ID.");
@@ -99,14 +105,14 @@
KvCoder.class.getSimpleName(),
view.getCoderInternal().getClass().getSimpleName()));
- KvCoder<?, V> kvCoder = (KvCoder<?, V>) elementCoder;
+ KvCoder<K, V> kvCoder = elementCoder;
- return new DataflowSideInputHandler<>(
+ return new DataflowMultimapSideInputHandler<>(
sideInputReader, view, kvCoder.getKeyCoder(), kvCoder.getValueCoder(), windowCoder);
}
- private static class DataflowSideInputHandler<K, V, W extends BoundedWindow>
- implements StateRequestHandlers.SideInputHandler<V, W> {
+ private static class DataflowMultimapSideInputHandler<K, V, W extends BoundedWindow>
+ implements MultimapSideInputHandler<K, V, W> {
private final SideInputReader sideInputReader;
PCollectionView<Materializations.MultimapView<Object, Object>> view;
@@ -114,7 +120,7 @@
private final Coder<V> valueCoder;
private final Coder<W> windowCoder;
- private DataflowSideInputHandler(
+ private DataflowMultimapSideInputHandler(
SideInputReader sideInputReader,
PCollectionView<Materializations.MultimapView<Object, Object>> view,
Coder<K> keyCoder,
@@ -128,14 +134,15 @@
}
@Override
- public Iterable<V> get(byte[] keyBytes, W window) {
- K key;
- try {
- key = keyCoder.decode(new ByteArrayInputStream(keyBytes));
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
+ public Iterable<K> get(W window) {
+ Materializations.MultimapView<K, V> sideInput =
+ (Materializations.MultimapView<K, V>)
+ sideInputReader.get(view, (BoundedWindow) windowCoder.structuralValue(window));
+ return sideInput.get();
+ }
+ @Override
+ public Iterable<V> get(K key, W window) {
Materializations.MultimapView<K, V> sideInput =
(Materializations.MultimapView<K, V>)
sideInputReader.get(view, (BoundedWindow) windowCoder.structuralValue(window));
@@ -144,7 +151,12 @@
}
@Override
- public Coder<V> resultCoder() {
+ public Coder<K> keyCoder() {
+ return keyCoder;
+ }
+
+ @Override
+ public Coder<V> valueCoder() {
return valueCoder;
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java
index f9ff270..da427df 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java
@@ -103,7 +103,9 @@
EnumMap<BeamFnApi.StateKey.TypeCase, StateRequestHandler> handlerMap =
new EnumMap<BeamFnApi.StateKey.TypeCase, StateRequestHandler>(
BeamFnApi.StateKey.TypeCase.class);
+ handlerMap.put(BeamFnApi.StateKey.TypeCase.ITERABLE_SIDE_INPUT, sideInputHandler);
handlerMap.put(BeamFnApi.StateKey.TypeCase.MULTIMAP_SIDE_INPUT, sideInputHandler);
+ handlerMap.put(BeamFnApi.StateKey.TypeCase.MULTIMAP_KEYS_SIDE_INPUT, sideInputHandler);
return StateRequestHandlers.delegateBasedUponType(handlerMap);
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java
index 3e9bfe7..ec7b53d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java
@@ -23,17 +23,12 @@
import static org.hamcrest.Matchers.instanceOf;
import static org.junit.Assert.assertEquals;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.InMemoryMultimapSideInputView;
import org.apache.beam.runners.core.SideInputReader;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.dataflow.worker.DataflowPortabilityPCollectionView;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandlers;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandler;
-import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.coders.VarIntCoder;
@@ -59,10 +54,6 @@
private static final String TRANSFORM_ID = "transformId";
private static final String SIDE_INPUT_NAME = "testSideInputId";
- private static final RunnerApi.FunctionSpec MULTIMAP_ACCESS =
- RunnerApi.FunctionSpec.newBuilder().setUrn(PTransformTranslation.MULTIMAP_SIDE_INPUT).build();
- private static final byte[] ENCODED_FOO = encode("foo", StringUtf8Coder.of());
- private static final byte[] ENCODED_FOO2 = encode("foo2", StringUtf8Coder.of());
private static final PCollectionView view =
DataflowPortabilityPCollectionView.with(
@@ -119,10 +110,9 @@
DataflowSideInputHandlerFactory factory =
DataflowSideInputHandlerFactory.of(sideInputReadersMap, sideInputIdToPCollectionViewMap);
thrown.expect(instanceOf(IllegalStateException.class));
- factory.forSideInput(
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
KvCoder.of(VoidCoder.of(), VoidCoder.of()),
GlobalWindow.Coder.INSTANCE);
}
@@ -142,15 +132,14 @@
DataflowSideInputHandlerFactory factory =
DataflowSideInputHandlerFactory.of(sideInputReadersMap, sideInputIdToPCollectionViewMap);
- SideInputHandler<Integer, GlobalWindow> handler =
- factory.forSideInput(
+ MultimapSideInputHandler<String, Integer, GlobalWindow> handler =
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
- KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()),
+ KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()),
GlobalWindow.Coder.INSTANCE);
- Iterable<Integer> result = handler.get(ENCODED_FOO2, GlobalWindow.INSTANCE);
+ Iterable<Integer> result = handler.get("foo2", GlobalWindow.INSTANCE);
assertThat(result, emptyIterable());
}
@@ -170,24 +159,13 @@
DataflowSideInputHandlerFactory factory =
DataflowSideInputHandlerFactory.of(sideInputReadersMap, sideInputIdToPCollectionViewMap);
- StateRequestHandlers.SideInputHandler handler =
- factory.forSideInput(
+ MultimapSideInputHandler<String, Integer, GlobalWindow> handler =
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()),
GlobalWindow.Coder.INSTANCE);
- Iterable<String> result = handler.get(ENCODED_FOO, GlobalWindow.INSTANCE);
+ Iterable<Integer> result = handler.get("foo", GlobalWindow.INSTANCE);
assertThat(result, containsInAnyOrder(1, 4, 3));
}
-
- private static <T> byte[] encode(T value, Coder<T> coder) {
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- try {
- coder.encode(value, out);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return out.toByteArray();
- }
}
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
index f2d374a..d4932a9 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
@@ -449,11 +449,11 @@
}
/**
- * A container type storing references to the key, value, and window {@link Coder} used when
- * handling side input state requests.
+ * A container type storing references to the value, and window {@link Coder} used when handling
+ * side input state requests.
*/
@AutoValue
- public abstract static class SideInputSpec<K, T, W extends BoundedWindow> {
+ public abstract static class SideInputSpec<T, W extends BoundedWindow> {
public static <T, W extends BoundedWindow> SideInputSpec of(
String transformId,
String sideInputId,
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java
index 4d8f816..26dd6ac 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java
@@ -38,16 +38,19 @@
import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey.TypeCase;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest.RequestCase;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.BagUserStateSpec;
import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.ExecutableProcessBundleDescriptor;
import org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.SideInputSpec;
import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.fn.stream.DataStreams;
import org.apache.beam.sdk.fn.stream.DataStreams.ElementDelimitedOutputStream;
+import org.apache.beam.sdk.transforms.Materializations;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.common.Reiterable;
+import org.apache.beam.sdk.values.KV;
import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.sdk.v2.sdk.extensions.protobuf.ByteStringCoder;
@@ -60,28 +63,68 @@
public class StateRequestHandlers {
/**
- * A handler for side inputs.
+ * Marker interface that denotes some type of side input handler. The access pattern defines the
+ * underlying type.
+ *
+ * <ul>
+ * <li>access pattern: Java type
+ * <li>{@code beam:side_input:iterable:v1}: {@link IterableSideInputHandler}
+ * <li>{@code beam:side_input:multimap:v1}: {@link MultimapSideInputHandler}
+ * </ul>
+ */
+ @ThreadSafe
+ public interface SideInputHandler {}
+
+ /**
+ * A handler for iterable side inputs.
*
* <p>Note that this handler is expected to be thread safe as it will be invoked concurrently.
*/
@ThreadSafe
- public interface SideInputHandler<V, W extends BoundedWindow> {
+ public interface IterableSideInputHandler<V, W extends BoundedWindow> extends SideInputHandler {
+ /**
+ * Returns an {@link Iterable} of values representing the side input for the given window.
+ *
+ * <p>TODO: Add support for side input chunking and caching if a {@link Reiterable} is returned.
+ */
+ Iterable<V> get(W window);
+
+ /** Returns the {@link Coder} to use for the elements of the resulting values iterable. */
+ Coder<V> elementCoder();
+ }
+
+ /**
+ * A handler for multimap side inputs.
+ *
+ * <p>Note that this handler is expected to be thread safe as it will be invoked concurrently.
+ */
+ @ThreadSafe
+ public interface MultimapSideInputHandler<K, V, W extends BoundedWindow>
+ extends SideInputHandler {
+ /**
+ * Returns an {@link Iterable} of keys representing the side input for the given window.
+ *
+ * <p>TODO: Add support for side input chunking and caching if a {@link Reiterable} is returned.
+ */
+ Iterable<K> get(W window);
+
/**
* Returns an {@link Iterable} of values representing the side input for the given key and
* window.
*
- * <p>The key is interpreted according to the access pattern of side input.
- *
* <p>TODO: Add support for side input chunking and caching if a {@link Reiterable} is returned.
*/
- Iterable<V> get(byte[] key, W window);
+ Iterable<V> get(K key, W window);
+
+ /** Returns the {@link Coder} to use for the elements of the resulting keys iterable. */
+ Coder<K> keyCoder();
/** Returns the {@link Coder} to use for the elements of the resulting values iterable. */
- Coder<V> resultCoder();
+ Coder<V> valueCoder();
}
/**
- * A factory which constructs {@link SideInputHandler}s.
+ * A factory which constructs {@link MultimapSideInputHandler}s.
*
* <p>Note that this factory should be thread safe because it will be invoked concurrently.
*/
@@ -89,32 +132,46 @@
public interface SideInputHandlerFactory {
/**
- * Returns a {@link SideInputHandler} for the given {@code pTransformId}, {@code sideInputId},
- * and {@code accessPattern}. The supplied {@code elementCoder} and {@code windowCoder} should
- * be used to encode/decode their respective values.
+ * Returns an {@link IterableSideInputHandler} for the given {@code pTransformId}, {@code
+ * sideInputId}. The supplied {@code elementCoder} and {@code windowCoder} should be used to
+ * encode/decode their respective values.
*/
- <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput(
- String pTransformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder);
+ <V, W extends BoundedWindow> IterableSideInputHandler<V, W> forIterableSideInput(
+ String pTransformId, String sideInputId, Coder<V> elementCoder, Coder<W> windowCoder);
+
+ /**
+ * Returns a {@link MultimapSideInputHandler} for the given {@code pTransformId}, {@code
+ * sideInputId}. The supplied {@code elementCoder} and {@code windowCoder} should be used to
+ * encode/decode their respective values.
+ */
+ <K, V, W extends BoundedWindow> MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String pTransformId, String sideInputId, KvCoder<K, V> elementCoder, Coder<W> windowCoder);
/** Throws a {@link UnsupportedOperationException} on the first access. */
static SideInputHandlerFactory unsupported() {
return new SideInputHandlerFactory() {
@Override
- public <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput(
- String pTransformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder) {
+ public <V, W extends BoundedWindow> IterableSideInputHandler<V, W> forIterableSideInput(
+ String pTransformId, String sideInputId, Coder<V> elementCoder, Coder<W> windowCoder) {
throw new UnsupportedOperationException(
String.format(
"The %s does not support handling sides inputs for PTransform %s with side "
+ "input id %s.",
- SideInputHandler.class.getSimpleName(), pTransformId, sideInputId));
+ SideInputHandlerFactory.class.getSimpleName(), pTransformId, sideInputId));
+ }
+
+ @Override
+ public <K, V, W extends BoundedWindow>
+ MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String pTransformId,
+ String sideInputId,
+ KvCoder<K, V> elementCoder,
+ Coder<W> windowCoder) {
+ throw new UnsupportedOperationException(
+ String.format(
+ "The %s does not support handling sides inputs for PTransform %s with side "
+ + "input id %s.",
+ SideInputHandlerFactory.class.getSimpleName(), pTransformId, sideInputId));
}
};
}
@@ -235,8 +292,8 @@
* ExecutableProcessBundleDescriptor#getSideInputSpecs} for the set of side inputs that are
* contained.
*
- * <p>Instances of {@link SideInputHandler}s returned by the {@link SideInputHandlerFactory} are
- * cached.
+ * <p>Instances of {@link MultimapSideInputHandler}s returned by the {@link
+ * SideInputHandlerFactory} are cached.
*/
public static StateRequestHandler forSideInputHandlerFactory(
Map<String, Map<String, SideInputSpec>> sideInputSpecs,
@@ -262,29 +319,51 @@
@Override
public CompletionStage<StateResponse.Builder> handle(StateRequest request) throws Exception {
+ checkState(
+ RequestCase.GET.equals(request.getRequestCase()),
+ String.format("Unsupported request type %s for side input.", request.getRequestCase()));
+
try {
- checkState(
- TypeCase.MULTIMAP_SIDE_INPUT.equals(request.getStateKey().getTypeCase()),
- "Unsupported %s type %s, expected %s",
- StateRequest.class.getSimpleName(),
- request.getStateKey().getTypeCase(),
- TypeCase.MULTIMAP_SIDE_INPUT);
-
- StateKey.MultimapSideInput stateKey = request.getStateKey().getMultimapSideInput();
- SideInputSpec<?, ?, ?> referenceSpec =
- sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
- SideInputHandler<?, ?> handler =
- handlerCache.computeIfAbsent(referenceSpec, this::createHandler);
-
- switch (request.getRequestCase()) {
- case GET:
- return handleGetRequest(request, handler);
- case APPEND:
- case CLEAR:
+ switch (request.getStateKey().getTypeCase()) {
+ case MULTIMAP_SIDE_INPUT:
+ {
+ StateKey.MultimapSideInput stateKey = request.getStateKey().getMultimapSideInput();
+ SideInputSpec<?, ?> referenceSpec =
+ sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
+ MultimapSideInputHandler handler =
+ (MultimapSideInputHandler)
+ handlerCache.computeIfAbsent(referenceSpec, this::createHandler);
+ return handleGetMultimapValuesRequest(request, handler);
+ }
+ case MULTIMAP_KEYS_SIDE_INPUT:
+ {
+ StateKey.MultimapKeysSideInput stateKey =
+ request.getStateKey().getMultimapKeysSideInput();
+ SideInputSpec<?, ?> referenceSpec =
+ sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
+ MultimapSideInputHandler handler =
+ (MultimapSideInputHandler)
+ handlerCache.computeIfAbsent(referenceSpec, this::createHandler);
+ return handleGetMultimapKeysRequest(request, handler);
+ }
+ case ITERABLE_SIDE_INPUT:
+ {
+ StateKey.IterableSideInput stateKey = request.getStateKey().getIterableSideInput();
+ SideInputSpec<?, ?> referenceSpec =
+ sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
+ IterableSideInputHandler handler =
+ (IterableSideInputHandler)
+ handlerCache.computeIfAbsent(referenceSpec, this::createHandler);
+ return handleGetIterableValuesRequest(request, handler);
+ }
default:
- throw new Exception(
+ throw new IllegalStateException(
String.format(
- "Unsupported request type %s for side input.", request.getRequestCase()));
+ "Unsupported %s type %s, expected %s or %s",
+ StateRequest.class.getSimpleName(),
+ request.getStateKey().getTypeCase(),
+ TypeCase.MULTIMAP_SIDE_INPUT,
+ TypeCase.MULTIMAP_KEYS_SIDE_INPUT));
}
} catch (Exception e) {
CompletableFuture f = new CompletableFuture();
@@ -293,26 +372,27 @@
}
}
- private <K, V, W extends BoundedWindow> CompletionStage<StateResponse.Builder> handleGetRequest(
- StateRequest request, SideInputHandler<V, W> handler) throws Exception {
+ private <K, V, W extends BoundedWindow>
+ CompletionStage<StateResponse.Builder> handleGetMultimapKeysRequest(
+ StateRequest request, MultimapSideInputHandler<K, V, W> handler) throws Exception {
// TODO: Add support for continuation tokens when handling state if the handler
// returned a {@link Reiterable}.
checkState(
request.getGet().getContinuationToken().isEmpty(),
"Continuation tokens are unsupported.");
- StateKey.MultimapSideInput stateKey = request.getStateKey().getMultimapSideInput();
+ StateKey.MultimapKeysSideInput stateKey = request.getStateKey().getMultimapKeysSideInput();
- SideInputSpec<K, V, W> sideInputReferenceSpec =
+ SideInputSpec<KV<K, V>, W> sideInputReferenceSpec =
sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
W window = sideInputReferenceSpec.windowCoder().decode(stateKey.getWindow().newInput());
- Iterable<V> values = handler.get(stateKey.getKey().toByteArray(), window);
+ Iterable<K> keys = handler.get(window);
List<ByteString> encodedValues = new ArrayList<>();
ElementDelimitedOutputStream outputStream = DataStreams.outbound(encodedValues::add);
- for (V value : values) {
- handler.resultCoder().encode(value, outputStream);
+ for (K key : keys) {
+ handler.keyCoder().encode(key, outputStream);
outputStream.delimitElement();
}
outputStream.close();
@@ -324,14 +404,91 @@
return CompletableFuture.completedFuture(response);
}
- private <K, V, W extends BoundedWindow> SideInputHandler<V, W> createHandler(
- SideInputSpec cacheKey) {
- return sideInputHandlerFactory.forSideInput(
- cacheKey.transformId(),
- cacheKey.sideInputId(),
- cacheKey.accessPattern(),
- cacheKey.elementCoder(),
- cacheKey.windowCoder());
+ private <K, V, W extends BoundedWindow>
+ CompletionStage<StateResponse.Builder> handleGetMultimapValuesRequest(
+ StateRequest request, MultimapSideInputHandler<K, V, W> handler) throws Exception {
+ // TODO: Add support for continuation tokens when handling state if the handler
+ // returned a {@link Reiterable}.
+ checkState(
+ request.getGet().getContinuationToken().isEmpty(),
+ "Continuation tokens are unsupported.");
+
+ StateKey.MultimapSideInput stateKey = request.getStateKey().getMultimapSideInput();
+
+ SideInputSpec<KV<K, V>, W> sideInputReferenceSpec =
+ sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
+
+ W window = sideInputReferenceSpec.windowCoder().decode(stateKey.getWindow().newInput());
+
+ Iterable<V> values =
+ handler.get(handler.keyCoder().decode(stateKey.getKey().newInput()), window);
+ List<ByteString> encodedValues = new ArrayList<>();
+ ElementDelimitedOutputStream outputStream = DataStreams.outbound(encodedValues::add);
+ for (V value : values) {
+ handler.valueCoder().encode(value, outputStream);
+ outputStream.delimitElement();
+ }
+ outputStream.close();
+
+ StateResponse.Builder response = StateResponse.newBuilder();
+ response.setId(request.getId());
+ response.setGet(
+ StateGetResponse.newBuilder().setData(ByteString.copyFrom(encodedValues)).build());
+ return CompletableFuture.completedFuture(response);
+ }
+
+ private <V, W extends BoundedWindow>
+ CompletionStage<StateResponse.Builder> handleGetIterableValuesRequest(
+ StateRequest request, IterableSideInputHandler<V, W> handler) throws Exception {
+ // TODO: Add support for continuation tokens when handling state if the handler
+ // returned a {@link Reiterable}.
+ checkState(
+ request.getGet().getContinuationToken().isEmpty(),
+ "Continuation tokens are unsupported.");
+
+ StateKey.IterableSideInput stateKey = request.getStateKey().getIterableSideInput();
+
+ SideInputSpec<V, W> sideInputReferenceSpec =
+ sideInputSpecs.get(stateKey.getTransformId()).get(stateKey.getSideInputId());
+
+ W window = sideInputReferenceSpec.windowCoder().decode(stateKey.getWindow().newInput());
+
+ Iterable<V> values = handler.get(window);
+ List<ByteString> encodedValues = new ArrayList<>();
+ ElementDelimitedOutputStream outputStream = DataStreams.outbound(encodedValues::add);
+ for (V value : values) {
+ handler.elementCoder().encode(value, outputStream);
+ outputStream.delimitElement();
+ }
+ outputStream.close();
+
+ StateResponse.Builder response = StateResponse.newBuilder();
+ response.setId(request.getId());
+ response.setGet(
+ StateGetResponse.newBuilder().setData(ByteString.copyFrom(encodedValues)).build());
+ return CompletableFuture.completedFuture(response);
+ }
+
+ private SideInputHandler createHandler(SideInputSpec<?, ?> cacheKey) {
+ switch (cacheKey.accessPattern().getUrn()) {
+ case Materializations.ITERABLE_MATERIALIZATION_URN:
+ return sideInputHandlerFactory.forIterableSideInput(
+ cacheKey.transformId(),
+ cacheKey.sideInputId(),
+ cacheKey.elementCoder(),
+ cacheKey.windowCoder());
+
+ case Materializations.MULTIMAP_MATERIALIZATION_URN:
+ return sideInputHandlerFactory.forMultimapSideInput(
+ cacheKey.transformId(),
+ cacheKey.sideInputId(),
+ (KvCoder) cacheKey.elementCoder(),
+ cacheKey.windowCoder());
+
+ default:
+ throw new IllegalStateException(
+ String.format("Unsupported access pattern for side input %s", cacheKey));
+ }
}
}
@@ -344,8 +501,8 @@
* ExecutableProcessBundleDescriptor#getSideInputSpecs} for the set of multimap side inputs that
* are contained.
*
- * <p>Instances of {@link SideInputHandler}s returned by the {@link SideInputHandlerFactory} are
- * cached.
+ * <p>Instances of {@link MultimapSideInputHandler}s returned by the {@link
+ * SideInputHandlerFactory} are cached.
*/
public static StateRequestHandler forBagUserStateHandlerFactory(
ExecutableProcessBundleDescriptor processBundleDescriptor,
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java
index 1f5a6c65..810e459 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java
@@ -19,29 +19,28 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
-import com.google.auto.value.AutoValue;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.graph.ExecutableStage;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
import org.apache.beam.runners.core.construction.graph.SideInputReference;
import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler;
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandlerFactory;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.KV;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.FluentIterable;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMultimap;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
/** {@link StateRequestHandler} that uses a {@link SideInputGetter} to access side inputs. */
public class BatchSideInputHandlerFactory implements SideInputHandlerFactory {
@@ -80,68 +79,52 @@
}
@Override
- public <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput(
- String transformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder) {
+ public <V, W extends BoundedWindow> IterableSideInputHandler<V, W> forIterableSideInput(
+ String transformId, String sideInputId, Coder<V> elementCoder, Coder<W> windowCoder) {
PCollectionNode collectionNode =
sideInputToCollection.get(
SideInputId.newBuilder().setTransformId(transformId).setLocalName(sideInputId).build());
checkArgument(collectionNode != null, "No side input for %s/%s", transformId, sideInputId);
- if (PTransformTranslation.ITERABLE_SIDE_INPUT.equals(accessPattern.getUrn())) {
- @SuppressWarnings("unchecked") // T == V
- Coder<V> outputCoder = (Coder<V>) elementCoder;
- return forIterableSideInput(
- sideInputGetter.getSideInput(collectionNode.getId()), outputCoder, windowCoder);
- } else if (PTransformTranslation.MULTIMAP_SIDE_INPUT.equals(accessPattern.getUrn())) {
- @SuppressWarnings("unchecked") // T == KV<?, V>
- KvCoder<?, V> kvCoder = (KvCoder<?, V>) elementCoder;
- return forMultimapSideInput(
- sideInputGetter.getSideInput(collectionNode.getId()),
- kvCoder.getKeyCoder(),
- kvCoder.getValueCoder(),
- windowCoder);
- } else {
- throw new IllegalArgumentException(
- String.format("Unknown side input access pattern: %s", accessPattern));
- }
- }
-
- private <T, W extends BoundedWindow> SideInputHandler<T, W> forIterableSideInput(
- List<WindowedValue<T>> broadcastVariable, Coder<T> elementCoder, Coder<W> windowCoder) {
- ImmutableMultimap.Builder<Object, T> windowToValuesBuilder = ImmutableMultimap.builder();
- for (WindowedValue<T> windowedValue : broadcastVariable) {
+ ImmutableMultimap.Builder<Object, V> windowToValuesBuilder = ImmutableMultimap.builder();
+ List<WindowedValue<V>> broadcastVariable = sideInputGetter.getSideInput(collectionNode.getId());
+ for (WindowedValue<V> windowedValue : broadcastVariable) {
for (BoundedWindow boundedWindow : windowedValue.getWindows()) {
@SuppressWarnings("unchecked")
W window = (W) boundedWindow;
windowToValuesBuilder.put(windowCoder.structuralValue(window), windowedValue.getValue());
}
}
- ImmutableMultimap<Object, T> windowToValues = windowToValuesBuilder.build();
+ ImmutableMultimap<Object, V> windowToValues = windowToValuesBuilder.build();
- return new SideInputHandler<T, W>() {
+ return new IterableSideInputHandler<V, W>() {
@Override
- public Iterable<T> get(byte[] key, W window) {
+ public Iterable<V> get(W window) {
return windowToValues.get(windowCoder.structuralValue(window));
}
@Override
- public Coder<T> resultCoder() {
+ public Coder<V> elementCoder() {
return elementCoder;
}
};
}
- private <K, V, W extends BoundedWindow> SideInputHandler<V, W> forMultimapSideInput(
- List<WindowedValue<KV<K, V>>> broadcastVariable,
- Coder<K> keyCoder,
- Coder<V> valueCoder,
- Coder<W> windowCoder) {
- ImmutableMultimap.Builder<SideInputKey, V> multimap = ImmutableMultimap.builder();
+ @Override
+ public <K, V, W extends BoundedWindow> MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String transformId, String sideInputId, KvCoder<K, V> elementCoder, Coder<W> windowCoder) {
+
+ PCollectionNode collectionNode =
+ sideInputToCollection.get(
+ SideInputId.newBuilder().setTransformId(transformId).setLocalName(sideInputId).build());
+ checkArgument(collectionNode != null, "No side input for %s/%s", transformId, sideInputId);
+
+ Coder<K> keyCoder = elementCoder.getKeyCoder();
+ Map<Object /* structural window */, Map<Object /* structural key */, KV<K, List<V>>>> data =
+ new HashMap<>();
+ List<WindowedValue<KV<K, V>>> broadcastVariable =
+ sideInputGetter.getSideInput(collectionNode.getId());
for (WindowedValue<KV<K, V>> windowedValue : broadcastVariable) {
K key = windowedValue.getValue().getKey();
V value = windowedValue.getValue().getValue();
@@ -149,62 +132,44 @@
for (BoundedWindow boundedWindow : windowedValue.getWindows()) {
@SuppressWarnings("unchecked")
W window = (W) boundedWindow;
- multimap.put(
- SideInputKey.of(keyCoder.structuralValue(key), windowCoder.structuralValue(window)),
- value);
+ Object structuralW = windowCoder.structuralValue(window);
+ Object structuralK = keyCoder.structuralValue(key);
+ KV<K, List<V>> records =
+ data.computeIfAbsent(structuralW, o -> new HashMap<>())
+ .computeIfAbsent(structuralK, o -> KV.of(key, new ArrayList<>()));
+ records.getValue().add(value);
}
}
- return new MultimapSideInputHandler<>(multimap.build(), keyCoder, valueCoder, windowCoder);
- }
-
- private static class MultimapSideInputHandler<K, V, W extends BoundedWindow>
- implements SideInputHandler<V, W> {
-
- private final Multimap<SideInputKey, V> collection;
- private final Coder<K> keyCoder;
- private final Coder<V> valueCoder;
- private final Coder<W> windowCoder;
-
- private MultimapSideInputHandler(
- Multimap<SideInputKey, V> collection,
- Coder<K> keyCoder,
- Coder<V> valueCoder,
- Coder<W> windowCoder) {
- this.collection = collection;
- this.keyCoder = keyCoder;
- this.valueCoder = valueCoder;
- this.windowCoder = windowCoder;
- }
-
- @Override
- public Iterable<V> get(byte[] keyBytes, W window) {
- K key;
- try {
- // TODO: We could skip decoding and just compare encoded values for deterministic keyCoders.
- key = keyCoder.decode(new ByteArrayInputStream(keyBytes));
- } catch (IOException e) {
- throw new RuntimeException(e);
+ return new MultimapSideInputHandler<K, V, W>() {
+ @Override
+ public Iterable<V> get(K key, W window) {
+ KV<K, List<V>> records =
+ data.getOrDefault(windowCoder.structuralValue(window), Collections.emptyMap())
+ .get(keyCoder.structuralValue(key));
+ if (records == null) {
+ return Collections.emptyList();
+ }
+ return Collections.unmodifiableList(records.getValue());
}
- return collection.get(
- SideInputKey.of(keyCoder.structuralValue(key), windowCoder.structuralValue(window)));
- }
- @Override
- public Coder<V> resultCoder() {
- return valueCoder;
- }
- }
+ @Override
+ public Coder<V> valueCoder() {
+ return elementCoder.getValueCoder();
+ }
- @AutoValue
- abstract static class SideInputKey {
- static SideInputKey of(Object key, Object window) {
- return new AutoValue_BatchSideInputHandlerFactory_SideInputKey(key, window);
- }
+ @Override
+ public Iterable<K> get(W window) {
+ Map<Object, KV<K, List<V>>> records =
+ data.getOrDefault(windowCoder.structuralValue(window), Collections.emptyMap());
+ return Iterables.unmodifiableIterable(
+ FluentIterable.concat(records.values()).transform(kListKV -> kListKV.getKey()));
+ }
- @Nullable
- abstract Object key();
-
- abstract Object window();
+ @Override
+ public Coder<K> keyCoder() {
+ return elementCoder.getKeyCoder();
+ }
+ };
}
}
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 5d4c8f0..6582a3d 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
@@ -70,7 +70,8 @@
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers;
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.BagUserStateHandler;
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.BagUserStateHandlerFactory;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler;
import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandlerFactory;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.BigEndianLongCoder;
@@ -459,21 +460,41 @@
descriptor.getSideInputSpecs(),
new SideInputHandlerFactory() {
@Override
- public <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput(
- String pTransformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder) {
- return new SideInputHandler<V, W>() {
+ public <V, W extends BoundedWindow>
+ IterableSideInputHandler<V, W> forIterableSideInput(
+ String pTransformId,
+ String sideInputId,
+ Coder<V> elementCoder,
+ Coder<W> windowCoder) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public <K, V, W extends BoundedWindow>
+ MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String pTransformId,
+ String sideInputId,
+ KvCoder<K, V> elementCoder,
+ Coder<W> windowCoder) {
+ return new MultimapSideInputHandler<K, V, W>() {
@Override
- public Iterable<V> get(byte[] key, W window) {
+ public Iterable<K> get(W window) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Iterable<V> get(K key, W window) {
return (Iterable) sideInputData;
}
@Override
- public Coder<V> resultCoder() {
- return ((KvCoder) elementCoder).getValueCoder();
+ public Coder<K> keyCoder() {
+ return elementCoder.getKeyCoder();
+ }
+
+ @Override
+ public Coder<V> valueCoder() {
+ return elementCoder.getValueCoder();
}
};
}
@@ -610,21 +631,41 @@
descriptor.getSideInputSpecs(),
new SideInputHandlerFactory() {
@Override
- public <T, V, W extends BoundedWindow> SideInputHandler<V, W> forSideInput(
- String pTransformId,
- String sideInputId,
- RunnerApi.FunctionSpec accessPattern,
- Coder<T> elementCoder,
- Coder<W> windowCoder) {
- return new SideInputHandler<V, W>() {
+ public <V, W extends BoundedWindow>
+ IterableSideInputHandler<V, W> forIterableSideInput(
+ String pTransformId,
+ String sideInputId,
+ Coder<V> elementCoder,
+ Coder<W> windowCoder) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public <K, V, W extends BoundedWindow>
+ MultimapSideInputHandler<K, V, W> forMultimapSideInput(
+ String pTransformId,
+ String sideInputId,
+ KvCoder<K, V> elementCoder,
+ Coder<W> windowCoder) {
+ return new MultimapSideInputHandler<K, V, W>() {
@Override
- public Iterable<V> get(byte[] key, W window) {
- return (Iterable) sideInputData;
+ public Iterable<V> get(BoundedWindow window) {
+ return null;
}
@Override
- public Coder<V> resultCoder() {
- return ((KvCoder) elementCoder).getValueCoder();
+ public Coder<K> keyCoder() {
+ return elementCoder.getKeyCoder();
+ }
+
+ @Override
+ public Coder<V> valueCoder() {
+ return elementCoder.getValueCoder();
+ }
+
+ @Override
+ public Iterable<V> get(K key, W window) {
+ return (Iterable) sideInputData;
}
};
}
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
index f664aa9..1afb93f 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
@@ -24,22 +24,19 @@
import static org.hamcrest.Matchers.instanceOf;
import static org.mockito.Mockito.when;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.graph.ExecutableStage;
import org.apache.beam.runners.core.construction.graph.ImmutableExecutableStage;
import org.apache.beam.runners.core.construction.graph.PipelineNode;
import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
import org.apache.beam.runners.core.construction.graph.SideInputReference;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.SideInputHandler;
-import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.IterableSideInputHandler;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandlers.MultimapSideInputHandler;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.coders.VarIntCoder;
@@ -69,10 +66,6 @@
private static final String TRANSFORM_ID = "transform-id";
private static final String SIDE_INPUT_NAME = "side-input";
private static final String COLLECTION_ID = "collection";
- private static final RunnerApi.FunctionSpec MULTIMAP_ACCESS =
- RunnerApi.FunctionSpec.newBuilder().setUrn(PTransformTranslation.MULTIMAP_SIDE_INPUT).build();
- private static final RunnerApi.FunctionSpec ITERABLE_ACCESS =
- RunnerApi.FunctionSpec.newBuilder().setUrn(PTransformTranslation.ITERABLE_SIDE_INPUT).build();
private static final ExecutableStage EXECUTABLE_STAGE =
createExecutableStage(
Arrays.asList(
@@ -81,8 +74,6 @@
SIDE_INPUT_NAME,
PipelineNode.pCollection(
COLLECTION_ID, RunnerApi.PCollection.getDefaultInstance()))));
- private static final byte[] ENCODED_NULL = encode(null, VoidCoder.of());
- private static final byte[] ENCODED_FOO = encode("foo", StringUtf8Coder.of());
@Rule public ExpectedException thrown = ExpectedException.none();
@@ -98,10 +89,9 @@
ExecutableStage stage = createExecutableStage(Collections.emptyList());
BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(stage, context);
thrown.expect(instanceOf(IllegalArgumentException.class));
- factory.forSideInput(
+ factory.forMultimapSideInput(
"transform-id",
"side-input",
- MULTIMAP_ACCESS,
KvCoder.of(VoidCoder.of(), VoidCoder.of()),
GlobalWindow.Coder.INSTANCE);
}
@@ -110,17 +100,18 @@
public void emptyResultForEmptyCollection() {
BatchSideInputHandlerFactory factory =
BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context);
- SideInputHandler<Integer, GlobalWindow> handler =
- factory.forSideInput(
+ MultimapSideInputHandler<Void, Integer, GlobalWindow> handler =
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
KvCoder.of(VoidCoder.of(), VarIntCoder.of()),
GlobalWindow.Coder.INSTANCE);
// We never populated the broadcast variable for "side-input", so the mock will return an empty
// list.
- Iterable<Integer> result = handler.get(ENCODED_NULL, GlobalWindow.INSTANCE);
- assertThat(result, emptyIterable());
+ Iterable<Void> keys = handler.get(GlobalWindow.INSTANCE);
+ assertThat(keys, emptyIterable());
+ Iterable<Integer> values = handler.get(null, GlobalWindow.INSTANCE);
+ assertThat(values, emptyIterable());
}
@Test
@@ -131,15 +122,16 @@
BatchSideInputHandlerFactory factory =
BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context);
- SideInputHandler<Integer, GlobalWindow> handler =
- factory.forSideInput(
+ MultimapSideInputHandler<Void, Integer, GlobalWindow> handler =
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
KvCoder.of(VoidCoder.of(), VarIntCoder.of()),
GlobalWindow.Coder.INSTANCE);
- Iterable<Integer> result = handler.get(ENCODED_NULL, GlobalWindow.INSTANCE);
- assertThat(result, contains(3));
+ Iterable<Void> keys = handler.get(GlobalWindow.INSTANCE);
+ assertThat(keys, contains((Void) null));
+ Iterable<Integer> values = handler.get(null, GlobalWindow.INSTANCE);
+ assertThat(values, contains(3));
}
@Test
@@ -153,15 +145,16 @@
BatchSideInputHandlerFactory factory =
BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context);
- SideInputHandler<Integer, GlobalWindow> handler =
- factory.forSideInput(
+ MultimapSideInputHandler<String, Integer, GlobalWindow> handler =
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()),
GlobalWindow.Coder.INSTANCE);
- Iterable<Integer> result = handler.get(ENCODED_FOO, GlobalWindow.INSTANCE);
- assertThat(result, containsInAnyOrder(2, 5));
+ Iterable<String> keys = handler.get(GlobalWindow.INSTANCE);
+ assertThat(keys, containsInAnyOrder("foo", "bar"));
+ Iterable<Integer> values = handler.get("foo", GlobalWindow.INSTANCE);
+ assertThat(values, containsInAnyOrder(2, 5));
}
@Test
@@ -175,7 +168,7 @@
.thenReturn(
Arrays.asList(
WindowedValue.of(KV.of("foo", 1), instantA, windowA, PaneInfo.NO_FIRING),
- WindowedValue.of(KV.of("bar", 2), instantA, windowA, PaneInfo.NO_FIRING),
+ WindowedValue.of(KV.of("baz", 2), instantA, windowA, PaneInfo.NO_FIRING),
WindowedValue.of(KV.of("foo", 3), instantA, windowA, PaneInfo.NO_FIRING),
WindowedValue.of(KV.of("foo", 4), instantB, windowB, PaneInfo.NO_FIRING),
WindowedValue.of(KV.of("bar", 5), instantB, windowB, PaneInfo.NO_FIRING),
@@ -183,17 +176,20 @@
BatchSideInputHandlerFactory factory =
BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context);
- SideInputHandler<Integer, IntervalWindow> handler =
- factory.forSideInput(
+ MultimapSideInputHandler<String, Integer, IntervalWindow> handler =
+ factory.forMultimapSideInput(
TRANSFORM_ID,
SIDE_INPUT_NAME,
- MULTIMAP_ACCESS,
KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()),
IntervalWindowCoder.of());
- Iterable<Integer> resultA = handler.get(ENCODED_FOO, windowA);
- Iterable<Integer> resultB = handler.get(ENCODED_FOO, windowB);
- assertThat(resultA, containsInAnyOrder(1, 3));
- assertThat(resultB, containsInAnyOrder(4, 6));
+ Iterable<String> keysA = handler.get(windowA);
+ Iterable<String> keysB = handler.get(windowB);
+ assertThat(keysA, containsInAnyOrder("foo", "baz"));
+ assertThat(keysB, containsInAnyOrder("foo", "bar"));
+ Iterable<Integer> valuesA = handler.get("foo", windowA);
+ Iterable<Integer> valuesB = handler.get("foo", windowB);
+ assertThat(valuesA, containsInAnyOrder(1, 3));
+ assertThat(valuesB, containsInAnyOrder(4, 6));
}
@Test
@@ -203,6 +199,7 @@
Instant instantC = new DateTime(2018, 1, 1, 1, 3, DateTimeZone.UTC).toInstant();
IntervalWindow windowA = new IntervalWindow(instantA, instantB);
IntervalWindow windowB = new IntervalWindow(instantB, instantC);
+ IntervalWindow windowC = new IntervalWindow(instantA, instantC);
when(context.getSideInput(COLLECTION_ID))
.thenReturn(
Arrays.asList(
@@ -213,17 +210,15 @@
BatchSideInputHandlerFactory factory =
BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context);
- SideInputHandler<Integer, IntervalWindow> handler =
- factory.forSideInput(
- TRANSFORM_ID,
- SIDE_INPUT_NAME,
- ITERABLE_ACCESS,
- VarIntCoder.of(),
- IntervalWindowCoder.of());
- Iterable<Integer> resultA = handler.get(null, windowA);
- Iterable<Integer> resultB = handler.get(null, windowB);
+ IterableSideInputHandler<Integer, IntervalWindow> handler =
+ factory.forIterableSideInput(
+ TRANSFORM_ID, SIDE_INPUT_NAME, VarIntCoder.of(), IntervalWindowCoder.of());
+ Iterable<Integer> resultA = handler.get(windowA);
+ Iterable<Integer> resultB = handler.get(windowB);
+ Iterable<Integer> resultC = handler.get(windowC);
assertThat(resultA, containsInAnyOrder(1, 2));
assertThat(resultB, containsInAnyOrder(3, 4));
+ assertThat(resultC, containsInAnyOrder());
}
private static ExecutableStage createExecutableStage(Collection<SideInputReference> sideInputs) {
@@ -241,14 +236,4 @@
Collections.emptyList(),
Collections.emptyList());
}
-
- private static <T> byte[] encode(T value, Coder<T> coder) {
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- try {
- coder.encode(value, out);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return out.toByteArray();
- }
}
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java
index 5d49767..1099949 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java
@@ -34,6 +34,7 @@
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.Materializations;
+import org.apache.beam.sdk.transforms.Materializations.MultimapView;
import org.apache.beam.sdk.transforms.ViewFn;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
@@ -46,7 +47,17 @@
public class SparkSideInputReader implements SideInputReader {
/** A {@link Materializations.MultimapView} which always returns an empty iterable. */
private static final Materializations.MultimapView EMPTY_MULTIMAP_VIEW =
- o -> Collections.EMPTY_LIST;
+ new MultimapView() {
+ @Override
+ public Iterable get() {
+ return Collections.EMPTY_LIST;
+ }
+
+ @Override
+ public Iterable get(@Nullable Object o) {
+ return Collections.EMPTY_LIST;
+ }
+ };
private final Map<TupleTag<?>, WindowingStrategy<?, ?>> sideInputs;
private final SideInputBroadcast broadcastStateData;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
index 2d1585b..a37657d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
@@ -271,7 +271,9 @@
userStateHandler = StateRequestHandler.unsupported();
}
+ handlerMap.put(StateKey.TypeCase.ITERABLE_SIDE_INPUT, sideInputHandler);
handlerMap.put(StateKey.TypeCase.MULTIMAP_SIDE_INPUT, sideInputHandler);
+ handlerMap.put(StateKey.TypeCase.MULTIMAP_KEYS_SIDE_INPUT, sideInputHandler);
handlerMap.put(StateKey.TypeCase.BAG_USER_STATE, userStateHandler);
return StateRequestHandlers.delegateBasedUponType(handlerMap);
}
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 32abbe6..236b518 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
@@ -75,7 +75,7 @@
return proto.EnumName(LogEntry_Severity_Enum_name, int32(x))
}
func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27, 1, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{27, 1, 0}
}
// A descriptor for connecting to a remote port using the Beam Fn Data API.
@@ -98,7 +98,7 @@
func (m *RemoteGrpcPort) String() string { return proto.CompactTextString(m) }
func (*RemoteGrpcPort) ProtoMessage() {}
func (*RemoteGrpcPort) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{0}
}
func (m *RemoteGrpcPort) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RemoteGrpcPort.Unmarshal(m, b)
@@ -158,7 +158,7 @@
func (m *InstructionRequest) String() string { return proto.CompactTextString(m) }
func (*InstructionRequest) ProtoMessage() {}
func (*InstructionRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{1}
}
func (m *InstructionRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InstructionRequest.Unmarshal(m, b)
@@ -414,7 +414,7 @@
func (m *InstructionResponse) String() string { return proto.CompactTextString(m) }
func (*InstructionResponse) ProtoMessage() {}
func (*InstructionResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{2}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{2}
}
func (m *InstructionResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_InstructionResponse.Unmarshal(m, b)
@@ -662,7 +662,7 @@
func (m *RegisterRequest) String() string { return proto.CompactTextString(m) }
func (*RegisterRequest) ProtoMessage() {}
func (*RegisterRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{3}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{3}
}
func (m *RegisterRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RegisterRequest.Unmarshal(m, b)
@@ -700,7 +700,7 @@
func (m *RegisterResponse) String() string { return proto.CompactTextString(m) }
func (*RegisterResponse) ProtoMessage() {}
func (*RegisterResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{4}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{4}
}
func (m *RegisterResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RegisterResponse.Unmarshal(m, b)
@@ -748,7 +748,7 @@
func (m *ProcessBundleDescriptor) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleDescriptor) ProtoMessage() {}
func (*ProcessBundleDescriptor) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{5}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{5}
}
func (m *ProcessBundleDescriptor) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleDescriptor.Unmarshal(m, b)
@@ -856,7 +856,7 @@
func (m *BundleApplication) String() string { return proto.CompactTextString(m) }
func (*BundleApplication) ProtoMessage() {}
func (*BundleApplication) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{6}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{6}
}
func (m *BundleApplication) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BundleApplication.Unmarshal(m, b)
@@ -941,7 +941,7 @@
func (m *DelayedBundleApplication) String() string { return proto.CompactTextString(m) }
func (*DelayedBundleApplication) ProtoMessage() {}
func (*DelayedBundleApplication) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{7}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{7}
}
func (m *DelayedBundleApplication) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DelayedBundleApplication.Unmarshal(m, b)
@@ -1000,7 +1000,7 @@
func (m *ProcessBundleRequest) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleRequest) ProtoMessage() {}
func (*ProcessBundleRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{8}
}
func (m *ProcessBundleRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleRequest.Unmarshal(m, b)
@@ -1054,7 +1054,7 @@
func (m *ProcessBundleRequest_CacheToken) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleRequest_CacheToken) ProtoMessage() {}
func (*ProcessBundleRequest_CacheToken) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{8, 0}
}
func (m *ProcessBundleRequest_CacheToken) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleRequest_CacheToken.Unmarshal(m, b)
@@ -1203,7 +1203,7 @@
func (m *ProcessBundleRequest_CacheToken_UserState) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleRequest_CacheToken_UserState) ProtoMessage() {}
func (*ProcessBundleRequest_CacheToken_UserState) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8, 0, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{8, 0, 0}
}
func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Unmarshal(m, b)
@@ -1238,7 +1238,7 @@
func (m *ProcessBundleRequest_CacheToken_SideInput) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleRequest_CacheToken_SideInput) ProtoMessage() {}
func (*ProcessBundleRequest_CacheToken_SideInput) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8, 0, 1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{8, 0, 1}
}
func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Unmarshal(m, b)
@@ -1292,7 +1292,7 @@
func (m *ProcessBundleResponse) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleResponse) ProtoMessage() {}
func (*ProcessBundleResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{9}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{9}
}
func (m *ProcessBundleResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleResponse.Unmarshal(m, b)
@@ -1356,7 +1356,7 @@
func (m *ProcessBundleProgressRequest) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleProgressRequest) ProtoMessage() {}
func (*ProcessBundleProgressRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{10}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{10}
}
func (m *ProcessBundleProgressRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleProgressRequest.Unmarshal(m, b)
@@ -1395,7 +1395,7 @@
func (m *Metrics) String() string { return proto.CompactTextString(m) }
func (*Metrics) ProtoMessage() {}
func (*Metrics) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11}
}
func (m *Metrics) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics.Unmarshal(m, b)
@@ -1448,7 +1448,7 @@
func (m *Metrics_PTransform) String() string { return proto.CompactTextString(m) }
func (*Metrics_PTransform) ProtoMessage() {}
func (*Metrics_PTransform) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 0}
}
func (m *Metrics_PTransform) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_PTransform.Unmarshal(m, b)
@@ -1518,7 +1518,7 @@
func (m *Metrics_PTransform_Measured) String() string { return proto.CompactTextString(m) }
func (*Metrics_PTransform_Measured) ProtoMessage() {}
func (*Metrics_PTransform_Measured) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 0, 0}
}
func (m *Metrics_PTransform_Measured) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_PTransform_Measured.Unmarshal(m, b)
@@ -1572,7 +1572,7 @@
func (m *Metrics_PTransform_ProcessedElements) String() string { return proto.CompactTextString(m) }
func (*Metrics_PTransform_ProcessedElements) ProtoMessage() {}
func (*Metrics_PTransform_ProcessedElements) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0, 1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 0, 1}
}
func (m *Metrics_PTransform_ProcessedElements) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_PTransform_ProcessedElements.Unmarshal(m, b)
@@ -1626,7 +1626,7 @@
func (m *Metrics_PTransform_ActiveElements) String() string { return proto.CompactTextString(m) }
func (*Metrics_PTransform_ActiveElements) ProtoMessage() {}
func (*Metrics_PTransform_ActiveElements) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0, 2}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 0, 2}
}
func (m *Metrics_PTransform_ActiveElements) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_PTransform_ActiveElements.Unmarshal(m, b)
@@ -1687,7 +1687,7 @@
func (m *Metrics_User) String() string { return proto.CompactTextString(m) }
func (*Metrics_User) ProtoMessage() {}
func (*Metrics_User) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 1}
}
func (m *Metrics_User) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_User.Unmarshal(m, b)
@@ -1868,7 +1868,7 @@
func (m *Metrics_User_MetricName) String() string { return proto.CompactTextString(m) }
func (*Metrics_User_MetricName) ProtoMessage() {}
func (*Metrics_User_MetricName) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 1, 0}
}
func (m *Metrics_User_MetricName) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_User_MetricName.Unmarshal(m, b)
@@ -1914,7 +1914,7 @@
func (m *Metrics_User_CounterData) String() string { return proto.CompactTextString(m) }
func (*Metrics_User_CounterData) ProtoMessage() {}
func (*Metrics_User_CounterData) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 1, 1}
}
func (m *Metrics_User_CounterData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_User_CounterData.Unmarshal(m, b)
@@ -1956,7 +1956,7 @@
func (m *Metrics_User_DistributionData) String() string { return proto.CompactTextString(m) }
func (*Metrics_User_DistributionData) ProtoMessage() {}
func (*Metrics_User_DistributionData) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 2}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 1, 2}
}
func (m *Metrics_User_DistributionData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_User_DistributionData.Unmarshal(m, b)
@@ -2017,7 +2017,7 @@
func (m *Metrics_User_GaugeData) String() string { return proto.CompactTextString(m) }
func (*Metrics_User_GaugeData) ProtoMessage() {}
func (*Metrics_User_GaugeData) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 3}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{11, 1, 3}
}
func (m *Metrics_User_GaugeData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Metrics_User_GaugeData.Unmarshal(m, b)
@@ -2069,7 +2069,7 @@
func (m *ProcessBundleProgressResponse) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleProgressResponse) ProtoMessage() {}
func (*ProcessBundleProgressResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{12}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{12}
}
func (m *ProcessBundleProgressResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleProgressResponse.Unmarshal(m, b)
@@ -2130,7 +2130,7 @@
func (m *ProcessBundleSplitRequest) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleSplitRequest) ProtoMessage() {}
func (*ProcessBundleSplitRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{13}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{13}
}
func (m *ProcessBundleSplitRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleSplitRequest.Unmarshal(m, b)
@@ -2190,7 +2190,7 @@
func (m *ProcessBundleSplitRequest_DesiredSplit) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleSplitRequest_DesiredSplit) ProtoMessage() {}
func (*ProcessBundleSplitRequest_DesiredSplit) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{13, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{13, 0}
}
func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Unmarshal(m, b)
@@ -2260,7 +2260,7 @@
func (m *ProcessBundleSplitResponse) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleSplitResponse) ProtoMessage() {}
func (*ProcessBundleSplitResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{14}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{14}
}
func (m *ProcessBundleSplitResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleSplitResponse.Unmarshal(m, b)
@@ -2331,7 +2331,7 @@
func (m *ProcessBundleSplitResponse_ChannelSplit) String() string { return proto.CompactTextString(m) }
func (*ProcessBundleSplitResponse_ChannelSplit) ProtoMessage() {}
func (*ProcessBundleSplitResponse_ChannelSplit) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{14, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{14, 0}
}
func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Unmarshal(m, b)
@@ -2385,7 +2385,7 @@
func (m *FinalizeBundleRequest) String() string { return proto.CompactTextString(m) }
func (*FinalizeBundleRequest) ProtoMessage() {}
func (*FinalizeBundleRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{15}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{15}
}
func (m *FinalizeBundleRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FinalizeBundleRequest.Unmarshal(m, b)
@@ -2422,7 +2422,7 @@
func (m *FinalizeBundleResponse) String() string { return proto.CompactTextString(m) }
func (*FinalizeBundleResponse) ProtoMessage() {}
func (*FinalizeBundleResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{16}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{16}
}
func (m *FinalizeBundleResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FinalizeBundleResponse.Unmarshal(m, b)
@@ -2456,7 +2456,7 @@
func (m *Elements) String() string { return proto.CompactTextString(m) }
func (*Elements) ProtoMessage() {}
func (*Elements) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{17}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{17}
}
func (m *Elements) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Elements.Unmarshal(m, b)
@@ -2514,7 +2514,7 @@
func (m *Elements_Data) String() string { return proto.CompactTextString(m) }
func (*Elements_Data) ProtoMessage() {}
func (*Elements_Data) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{17, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{17, 0}
}
func (m *Elements_Data) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Elements_Data.Unmarshal(m, b)
@@ -2582,7 +2582,7 @@
func (m *StateRequest) String() string { return proto.CompactTextString(m) }
func (*StateRequest) ProtoMessage() {}
func (*StateRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{18}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{18}
}
func (m *StateRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateRequest.Unmarshal(m, b)
@@ -2787,7 +2787,7 @@
func (m *StateResponse) String() string { return proto.CompactTextString(m) }
func (*StateResponse) ProtoMessage() {}
func (*StateResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{19}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{19}
}
func (m *StateResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateResponse.Unmarshal(m, b)
@@ -2968,6 +2968,7 @@
// *StateKey_MultimapSideInput_
// *StateKey_BagUserState_
// *StateKey_IterableSideInput_
+ // *StateKey_MultimapKeysSideInput_
Type isStateKey_Type `protobuf_oneof:"type"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@@ -2978,7 +2979,7 @@
func (m *StateKey) String() string { return proto.CompactTextString(m) }
func (*StateKey) ProtoMessage() {}
func (*StateKey) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20}
}
func (m *StateKey) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateKey.Unmarshal(m, b)
@@ -3014,11 +3015,15 @@
type StateKey_IterableSideInput_ struct {
IterableSideInput *StateKey_IterableSideInput `protobuf:"bytes,4,opt,name=iterable_side_input,json=iterableSideInput,proto3,oneof"`
}
+type StateKey_MultimapKeysSideInput_ struct {
+ MultimapKeysSideInput *StateKey_MultimapKeysSideInput `protobuf:"bytes,5,opt,name=multimap_keys_side_input,json=multimapKeysSideInput,proto3,oneof"`
+}
-func (*StateKey_Runner_) isStateKey_Type() {}
-func (*StateKey_MultimapSideInput_) isStateKey_Type() {}
-func (*StateKey_BagUserState_) isStateKey_Type() {}
-func (*StateKey_IterableSideInput_) isStateKey_Type() {}
+func (*StateKey_Runner_) isStateKey_Type() {}
+func (*StateKey_MultimapSideInput_) isStateKey_Type() {}
+func (*StateKey_BagUserState_) isStateKey_Type() {}
+func (*StateKey_IterableSideInput_) isStateKey_Type() {}
+func (*StateKey_MultimapKeysSideInput_) isStateKey_Type() {}
func (m *StateKey) GetType() isStateKey_Type {
if m != nil {
@@ -3055,6 +3060,13 @@
return nil
}
+func (m *StateKey) GetMultimapKeysSideInput() *StateKey_MultimapKeysSideInput {
+ if x, ok := m.GetType().(*StateKey_MultimapKeysSideInput_); ok {
+ return x.MultimapKeysSideInput
+ }
+ return nil
+}
+
// XXX_OneofFuncs is for the internal use of the proto package.
func (*StateKey) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
return _StateKey_OneofMarshaler, _StateKey_OneofUnmarshaler, _StateKey_OneofSizer, []interface{}{
@@ -3062,6 +3074,7 @@
(*StateKey_MultimapSideInput_)(nil),
(*StateKey_BagUserState_)(nil),
(*StateKey_IterableSideInput_)(nil),
+ (*StateKey_MultimapKeysSideInput_)(nil),
}
}
@@ -3089,6 +3102,11 @@
if err := b.EncodeMessage(x.IterableSideInput); err != nil {
return err
}
+ case *StateKey_MultimapKeysSideInput_:
+ b.EncodeVarint(5<<3 | proto.WireBytes)
+ if err := b.EncodeMessage(x.MultimapKeysSideInput); err != nil {
+ return err
+ }
case nil:
default:
return fmt.Errorf("StateKey.Type has unexpected type %T", x)
@@ -3131,6 +3149,14 @@
err := b.DecodeMessage(msg)
m.Type = &StateKey_IterableSideInput_{msg}
return true, err
+ case 5: // type.multimap_keys_side_input
+ if wire != proto.WireBytes {
+ return true, proto.ErrInternalBadWireType
+ }
+ msg := new(StateKey_MultimapKeysSideInput)
+ err := b.DecodeMessage(msg)
+ m.Type = &StateKey_MultimapKeysSideInput_{msg}
+ return true, err
default:
return false, nil
}
@@ -3160,6 +3186,11 @@
n += 1 // tag and wire
n += proto.SizeVarint(uint64(s))
n += s
+ case *StateKey_MultimapKeysSideInput_:
+ s := proto.Size(x.MultimapKeysSideInput)
+ n += 1 // tag and wire
+ n += proto.SizeVarint(uint64(s))
+ n += s
case nil:
default:
panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
@@ -3185,7 +3216,7 @@
func (m *StateKey_Runner) String() string { return proto.CompactTextString(m) }
func (*StateKey_Runner) ProtoMessage() {}
func (*StateKey_Runner) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20, 0}
}
func (m *StateKey_Runner) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateKey_Runner.Unmarshal(m, b)
@@ -3212,13 +3243,87 @@
return nil
}
-// Represents a request for the values associated with a specified user key and window
-// in a PCollection.
+// Represents a request for the values associated with a specified window
+// in a PCollection. See
+// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+// details.
//
-// Can only perform StateGetRequests on side inputs with the URN beam:side_input:multimap:v1.
+// Can only be used to perform StateGetRequests on side inputs of the URN
+// beam:side_input:iterable:v1.
//
-// For a PCollection<KV<K, V>>, the response data stream will be a concatenation of all V's
-// associated with the specified key K.
+// For a PCollection<V>, the response data stream will be a concatenation
+// of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
+// for further details.
+type StateKey_IterableSideInput struct {
+ // (Required) The id of the PTransform containing a side input.
+ TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+ // (Required) The id of the side input.
+ SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+ // (Required) The window (after mapping the currently executing elements
+ // window into the side input windows domain) encoded in a nested context.
+ Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
+}
+
+func (m *StateKey_IterableSideInput) Reset() { *m = StateKey_IterableSideInput{} }
+func (m *StateKey_IterableSideInput) String() string { return proto.CompactTextString(m) }
+func (*StateKey_IterableSideInput) ProtoMessage() {}
+func (*StateKey_IterableSideInput) Descriptor() ([]byte, []int) {
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20, 1}
+}
+func (m *StateKey_IterableSideInput) XXX_Unmarshal(b []byte) error {
+ return xxx_messageInfo_StateKey_IterableSideInput.Unmarshal(m, b)
+}
+func (m *StateKey_IterableSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ return xxx_messageInfo_StateKey_IterableSideInput.Marshal(b, m, deterministic)
+}
+func (dst *StateKey_IterableSideInput) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_StateKey_IterableSideInput.Merge(dst, src)
+}
+func (m *StateKey_IterableSideInput) XXX_Size() int {
+ return xxx_messageInfo_StateKey_IterableSideInput.Size(m)
+}
+func (m *StateKey_IterableSideInput) XXX_DiscardUnknown() {
+ xxx_messageInfo_StateKey_IterableSideInput.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StateKey_IterableSideInput proto.InternalMessageInfo
+
+func (m *StateKey_IterableSideInput) GetTransformId() string {
+ if m != nil {
+ return m.TransformId
+ }
+ return ""
+}
+
+func (m *StateKey_IterableSideInput) GetSideInputId() string {
+ if m != nil {
+ return m.SideInputId
+ }
+ return ""
+}
+
+func (m *StateKey_IterableSideInput) GetWindow() []byte {
+ if m != nil {
+ return m.Window
+ }
+ return nil
+}
+
+// Represents a request for the values associated with a specified user key
+// and window in a PCollection. See
+// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+// details.
+//
+// Can only be used to perform StateGetRequests on side inputs of the URN
+// beam:side_input:multimap:v1.
+//
+// For a PCollection<KV<K, V>>, the response data stream will be a
+// concatenation of all V's associated with the specified key K. See
+// https://s.apache.org/beam-fn-api-send-and-receive-data for further
+// details.
type StateKey_MultimapSideInput struct {
// (Required) The id of the PTransform containing a side input.
TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
@@ -3238,7 +3343,7 @@
func (m *StateKey_MultimapSideInput) String() string { return proto.CompactTextString(m) }
func (*StateKey_MultimapSideInput) ProtoMessage() {}
func (*StateKey_MultimapSideInput) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20, 2}
}
func (m *StateKey_MultimapSideInput) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateKey_MultimapSideInput.Unmarshal(m, b)
@@ -3286,6 +3391,75 @@
return nil
}
+// Represents a request for the keys associated with a specified window in a PCollection. See
+// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+// details.
+//
+// Can only be used to perform StateGetRequests on side inputs of the URN
+// beam:side_input:multimap:v1.
+//
+// For a PCollection<KV<K, V>>, the response data stream will be a
+// concatenation of all K's associated with the specified window. See
+// https://s.apache.org/beam-fn-api-send-and-receive-data for further
+// details.
+type StateKey_MultimapKeysSideInput struct {
+ // (Required) The id of the PTransform containing a side input.
+ TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+ // (Required) The id of the side input.
+ SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+ // (Required) The window (after mapping the currently executing elements
+ // window into the side input windows domain) encoded in a nested context.
+ Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+ XXX_NoUnkeyedLiteral struct{} `json:"-"`
+ XXX_unrecognized []byte `json:"-"`
+ XXX_sizecache int32 `json:"-"`
+}
+
+func (m *StateKey_MultimapKeysSideInput) Reset() { *m = StateKey_MultimapKeysSideInput{} }
+func (m *StateKey_MultimapKeysSideInput) String() string { return proto.CompactTextString(m) }
+func (*StateKey_MultimapKeysSideInput) ProtoMessage() {}
+func (*StateKey_MultimapKeysSideInput) Descriptor() ([]byte, []int) {
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20, 3}
+}
+func (m *StateKey_MultimapKeysSideInput) XXX_Unmarshal(b []byte) error {
+ return xxx_messageInfo_StateKey_MultimapKeysSideInput.Unmarshal(m, b)
+}
+func (m *StateKey_MultimapKeysSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ return xxx_messageInfo_StateKey_MultimapKeysSideInput.Marshal(b, m, deterministic)
+}
+func (dst *StateKey_MultimapKeysSideInput) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_StateKey_MultimapKeysSideInput.Merge(dst, src)
+}
+func (m *StateKey_MultimapKeysSideInput) XXX_Size() int {
+ return xxx_messageInfo_StateKey_MultimapKeysSideInput.Size(m)
+}
+func (m *StateKey_MultimapKeysSideInput) XXX_DiscardUnknown() {
+ xxx_messageInfo_StateKey_MultimapKeysSideInput.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StateKey_MultimapKeysSideInput proto.InternalMessageInfo
+
+func (m *StateKey_MultimapKeysSideInput) GetTransformId() string {
+ if m != nil {
+ return m.TransformId
+ }
+ return ""
+}
+
+func (m *StateKey_MultimapKeysSideInput) GetSideInputId() string {
+ if m != nil {
+ return m.SideInputId
+ }
+ return ""
+}
+
+func (m *StateKey_MultimapKeysSideInput) GetWindow() []byte {
+ if m != nil {
+ return m.Window
+ }
+ return nil
+}
+
type StateKey_BagUserState struct {
// (Required) The id of the PTransform containing user state.
TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
@@ -3305,7 +3479,7 @@
func (m *StateKey_BagUserState) String() string { return proto.CompactTextString(m) }
func (*StateKey_BagUserState) ProtoMessage() {}
func (*StateKey_BagUserState) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 2}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20, 4}
}
func (m *StateKey_BagUserState) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateKey_BagUserState.Unmarshal(m, b)
@@ -3353,70 +3527,6 @@
return nil
}
-// Represents a request for the values associated with a specified window in a PCollection.
-//
-// Can only perform StateGetRequests on side inputs with the URN beam:side_input:iterable:v1 and
-// beam:side_input:multimap:v1.
-//
-// For a PCollection<V>, the response data stream will be a concatenation of all V's.
-type StateKey_IterableSideInput struct {
- // (Required) The id of the PTransform containing a side input.
- TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
- // (Required) The id of the side input.
- SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
- // (Required) The window (after mapping the currently executing elements
- // window into the side input windows domain) encoded in a nested context.
- Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
- XXX_NoUnkeyedLiteral struct{} `json:"-"`
- XXX_unrecognized []byte `json:"-"`
- XXX_sizecache int32 `json:"-"`
-}
-
-func (m *StateKey_IterableSideInput) Reset() { *m = StateKey_IterableSideInput{} }
-func (m *StateKey_IterableSideInput) String() string { return proto.CompactTextString(m) }
-func (*StateKey_IterableSideInput) ProtoMessage() {}
-func (*StateKey_IterableSideInput) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 3}
-}
-func (m *StateKey_IterableSideInput) XXX_Unmarshal(b []byte) error {
- return xxx_messageInfo_StateKey_IterableSideInput.Unmarshal(m, b)
-}
-func (m *StateKey_IterableSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
- return xxx_messageInfo_StateKey_IterableSideInput.Marshal(b, m, deterministic)
-}
-func (dst *StateKey_IterableSideInput) XXX_Merge(src proto.Message) {
- xxx_messageInfo_StateKey_IterableSideInput.Merge(dst, src)
-}
-func (m *StateKey_IterableSideInput) XXX_Size() int {
- return xxx_messageInfo_StateKey_IterableSideInput.Size(m)
-}
-func (m *StateKey_IterableSideInput) XXX_DiscardUnknown() {
- xxx_messageInfo_StateKey_IterableSideInput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateKey_IterableSideInput proto.InternalMessageInfo
-
-func (m *StateKey_IterableSideInput) GetTransformId() string {
- if m != nil {
- return m.TransformId
- }
- return ""
-}
-
-func (m *StateKey_IterableSideInput) GetSideInputId() string {
- if m != nil {
- return m.SideInputId
- }
- return ""
-}
-
-func (m *StateKey_IterableSideInput) GetWindow() []byte {
- if m != nil {
- return m.Window
- }
- return nil
-}
-
// A request to get state.
type StateGetRequest struct {
// (Optional) If specified, signals to the runner that the response
@@ -3434,7 +3544,7 @@
func (m *StateGetRequest) String() string { return proto.CompactTextString(m) }
func (*StateGetRequest) ProtoMessage() {}
func (*StateGetRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{21}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{21}
}
func (m *StateGetRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateGetRequest.Unmarshal(m, b)
@@ -3481,7 +3591,7 @@
func (m *StateGetResponse) String() string { return proto.CompactTextString(m) }
func (*StateGetResponse) ProtoMessage() {}
func (*StateGetResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{22}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{22}
}
func (m *StateGetResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateGetResponse.Unmarshal(m, b)
@@ -3530,7 +3640,7 @@
func (m *StateAppendRequest) String() string { return proto.CompactTextString(m) }
func (*StateAppendRequest) ProtoMessage() {}
func (*StateAppendRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{23}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{23}
}
func (m *StateAppendRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateAppendRequest.Unmarshal(m, b)
@@ -3568,7 +3678,7 @@
func (m *StateAppendResponse) String() string { return proto.CompactTextString(m) }
func (*StateAppendResponse) ProtoMessage() {}
func (*StateAppendResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{24}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{24}
}
func (m *StateAppendResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateAppendResponse.Unmarshal(m, b)
@@ -3599,7 +3709,7 @@
func (m *StateClearRequest) String() string { return proto.CompactTextString(m) }
func (*StateClearRequest) ProtoMessage() {}
func (*StateClearRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{25}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{25}
}
func (m *StateClearRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateClearRequest.Unmarshal(m, b)
@@ -3630,7 +3740,7 @@
func (m *StateClearResponse) String() string { return proto.CompactTextString(m) }
func (*StateClearResponse) ProtoMessage() {}
func (*StateClearResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{26}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{26}
}
func (m *StateClearResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateClearResponse.Unmarshal(m, b)
@@ -3686,7 +3796,7 @@
func (m *LogEntry) String() string { return proto.CompactTextString(m) }
func (*LogEntry) ProtoMessage() {}
func (*LogEntry) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{27}
}
func (m *LogEntry) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LogEntry.Unmarshal(m, b)
@@ -3776,7 +3886,7 @@
func (m *LogEntry_List) String() string { return proto.CompactTextString(m) }
func (*LogEntry_List) ProtoMessage() {}
func (*LogEntry_List) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27, 0}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{27, 0}
}
func (m *LogEntry_List) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LogEntry_List.Unmarshal(m, b)
@@ -3826,7 +3936,7 @@
func (m *LogEntry_Severity) String() string { return proto.CompactTextString(m) }
func (*LogEntry_Severity) ProtoMessage() {}
func (*LogEntry_Severity) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27, 1}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{27, 1}
}
func (m *LogEntry_Severity) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LogEntry_Severity.Unmarshal(m, b)
@@ -3856,7 +3966,7 @@
func (m *LogControl) String() string { return proto.CompactTextString(m) }
func (*LogControl) ProtoMessage() {}
func (*LogControl) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{28}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{28}
}
func (m *LogControl) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_LogControl.Unmarshal(m, b)
@@ -3892,7 +4002,7 @@
func (m *StartWorkerRequest) String() string { return proto.CompactTextString(m) }
func (*StartWorkerRequest) ProtoMessage() {}
func (*StartWorkerRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{29}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{29}
}
func (m *StartWorkerRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StartWorkerRequest.Unmarshal(m, b)
@@ -3965,7 +4075,7 @@
func (m *StartWorkerResponse) String() string { return proto.CompactTextString(m) }
func (*StartWorkerResponse) ProtoMessage() {}
func (*StartWorkerResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{30}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{30}
}
func (m *StartWorkerResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StartWorkerResponse.Unmarshal(m, b)
@@ -4003,7 +4113,7 @@
func (m *StopWorkerRequest) String() string { return proto.CompactTextString(m) }
func (*StopWorkerRequest) ProtoMessage() {}
func (*StopWorkerRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{31}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{31}
}
func (m *StopWorkerRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StopWorkerRequest.Unmarshal(m, b)
@@ -4041,7 +4151,7 @@
func (m *StopWorkerResponse) String() string { return proto.CompactTextString(m) }
func (*StopWorkerResponse) ProtoMessage() {}
func (*StopWorkerResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{32}
+ return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{32}
}
func (m *StopWorkerResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StopWorkerResponse.Unmarshal(m, b)
@@ -4118,9 +4228,10 @@
proto.RegisterType((*StateResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateResponse")
proto.RegisterType((*StateKey)(nil), "org.apache.beam.model.fn_execution.v1.StateKey")
proto.RegisterType((*StateKey_Runner)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.Runner")
- proto.RegisterType((*StateKey_MultimapSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput")
- proto.RegisterType((*StateKey_BagUserState)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.BagUserState")
proto.RegisterType((*StateKey_IterableSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput")
+ proto.RegisterType((*StateKey_MultimapSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput")
+ proto.RegisterType((*StateKey_MultimapKeysSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysSideInput")
+ proto.RegisterType((*StateKey_BagUserState)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.BagUserState")
proto.RegisterType((*StateGetRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateGetRequest")
proto.RegisterType((*StateGetResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateGetResponse")
proto.RegisterType((*StateAppendRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateAppendRequest")
@@ -4644,209 +4755,211 @@
Metadata: "beam_fn_api.proto",
}
-func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_95f219ade4a36a20) }
+func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_232aec3131d70c18) }
-var fileDescriptor_beam_fn_api_95f219ade4a36a20 = []byte{
- // 3204 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0x5f, 0x6f, 0x1b, 0xc7,
- 0xb5, 0xf7, 0x92, 0x94, 0x44, 0x1e, 0x52, 0x12, 0x39, 0x92, 0x6c, 0x7a, 0xaf, 0x73, 0xaf, 0xc3,
- 0x7b, 0x03, 0x08, 0xb9, 0x08, 0x6d, 0xcb, 0x46, 0x62, 0xa7, 0x89, 0x13, 0x89, 0xa2, 0x6d, 0xc6,
- 0xb2, 0xcd, 0xae, 0xe4, 0xba, 0x4d, 0x9a, 0x2c, 0x56, 0xdc, 0x21, 0xbd, 0xf0, 0x72, 0x77, 0x33,
- 0xb3, 0x94, 0x2d, 0x37, 0x68, 0xd0, 0x3f, 0x68, 0xd1, 0xa2, 0x6d, 0x5e, 0xfa, 0x90, 0xf4, 0xad,
- 0x2d, 0x50, 0xa0, 0x2f, 0xfd, 0x00, 0xf9, 0x06, 0x05, 0x0a, 0xf4, 0x0b, 0xe4, 0xa5, 0x40, 0x0b,
- 0xb4, 0x4d, 0x9f, 0x0b, 0xf4, 0xad, 0x98, 0x3f, 0xfb, 0x87, 0x4b, 0xd2, 0x59, 0x92, 0x4a, 0xdf,
- 0x76, 0x66, 0xf6, 0xfc, 0x7e, 0x67, 0xce, 0x9e, 0x39, 0x73, 0xce, 0xcc, 0x42, 0xe5, 0x10, 0x1b,
- 0x7d, 0xbd, 0xeb, 0xe8, 0x86, 0x67, 0xd5, 0x3d, 0xe2, 0xfa, 0x2e, 0x7a, 0xc1, 0x25, 0xbd, 0xba,
- 0xe1, 0x19, 0x9d, 0x87, 0xb8, 0xce, 0x46, 0xeb, 0x7d, 0xd7, 0xc4, 0x76, 0xbd, 0xeb, 0xe8, 0xf8,
- 0x09, 0xee, 0x0c, 0x7c, 0xcb, 0x75, 0xea, 0x47, 0x97, 0xd4, 0x0d, 0x2e, 0x49, 0x06, 0x8e, 0x83,
- 0x49, 0x24, 0xad, 0xae, 0x62, 0xc7, 0xf4, 0x5c, 0xcb, 0xf1, 0xa9, 0xec, 0x38, 0xdf, 0x73, 0xdd,
- 0x9e, 0x8d, 0x2f, 0xf0, 0xd6, 0xe1, 0xa0, 0x7b, 0xc1, 0xc4, 0xb4, 0x43, 0x2c, 0xcf, 0x77, 0x89,
- 0x7c, 0xe3, 0x7f, 0x92, 0x6f, 0xf8, 0x56, 0x1f, 0x53, 0xdf, 0xe8, 0x7b, 0xf2, 0x85, 0xff, 0x1e,
- 0x81, 0x18, 0x10, 0x83, 0xeb, 0x31, 0x61, 0xfc, 0x31, 0x31, 0x3c, 0x0f, 0x93, 0x40, 0x85, 0xe5,
- 0x3e, 0xf6, 0x89, 0xd5, 0x91, 0xcd, 0xda, 0x2f, 0x14, 0x58, 0xd1, 0x70, 0xdf, 0xf5, 0xf1, 0x4d,
- 0xe2, 0x75, 0xda, 0x2e, 0xf1, 0x51, 0x1f, 0x4e, 0x1b, 0x9e, 0xa5, 0x53, 0x4c, 0x8e, 0xac, 0x0e,
- 0xd6, 0x23, 0x15, 0xab, 0xca, 0x79, 0x65, 0xb3, 0xb8, 0xf5, 0x4a, 0x7d, 0xbc, 0x51, 0x3c, 0xcb,
- 0xc3, 0xb6, 0xe5, 0xe0, 0xfa, 0xd1, 0xa5, 0xfa, 0xb6, 0x67, 0xed, 0x0b, 0xf9, 0xdd, 0x50, 0x5c,
- 0x5b, 0x37, 0xc6, 0xf4, 0xa2, 0xb3, 0x90, 0xef, 0xb8, 0x26, 0x26, 0xba, 0x65, 0x56, 0x33, 0xe7,
- 0x95, 0xcd, 0x82, 0xb6, 0xc4, 0xdb, 0x2d, 0xb3, 0xf6, 0x97, 0x1c, 0xa0, 0x96, 0x43, 0x7d, 0x32,
- 0xe8, 0xb0, 0x19, 0x6a, 0xf8, 0xfd, 0x01, 0xa6, 0x3e, 0x7a, 0x01, 0x56, 0xac, 0xa8, 0x97, 0xc9,
- 0x29, 0x5c, 0x6e, 0x39, 0xd6, 0xdb, 0x32, 0xd1, 0x7d, 0xc8, 0x13, 0xdc, 0xb3, 0xa8, 0x8f, 0x49,
- 0xf5, 0xaf, 0x4b, 0x5c, 0xf5, 0x97, 0xeb, 0xa9, 0xbe, 0x67, 0x5d, 0x93, 0x72, 0x92, 0xf1, 0xd6,
- 0x29, 0x2d, 0x84, 0x42, 0x18, 0x56, 0x3c, 0xe2, 0x76, 0x30, 0xa5, 0xfa, 0xe1, 0xc0, 0x31, 0x6d,
- 0x5c, 0xfd, 0x9b, 0x00, 0xff, 0x4a, 0x4a, 0xf0, 0xb6, 0x90, 0xde, 0xe1, 0xc2, 0x11, 0xc3, 0xb2,
- 0x17, 0xef, 0x47, 0xdf, 0x86, 0x33, 0xc3, 0x34, 0xba, 0x47, 0xdc, 0x1e, 0xc1, 0x94, 0x56, 0xff,
- 0x2e, 0xf8, 0x1a, 0xb3, 0xf0, 0xb5, 0x25, 0x48, 0xc4, 0xbb, 0xe1, 0x8d, 0x1b, 0x47, 0x03, 0x58,
- 0x4f, 0xf0, 0x53, 0xcf, 0xb6, 0xfc, 0xea, 0xe7, 0x82, 0xfc, 0xcd, 0x59, 0xc8, 0xf7, 0x19, 0x42,
- 0xc4, 0x8c, 0xbc, 0x91, 0x41, 0xf4, 0x10, 0x56, 0xbb, 0x96, 0x63, 0xd8, 0xd6, 0x53, 0x1c, 0x98,
- 0xf7, 0x1f, 0x82, 0xf1, 0xb5, 0x94, 0x8c, 0x37, 0xa4, 0x78, 0xd2, 0xbe, 0x2b, 0xdd, 0xa1, 0x81,
- 0x9d, 0x02, 0x2c, 0x11, 0x31, 0x58, 0xfb, 0xee, 0x02, 0xac, 0x0d, 0xf9, 0x19, 0xf5, 0x5c, 0x87,
- 0xe2, 0xb4, 0x8e, 0xb6, 0x0e, 0x0b, 0x98, 0x10, 0x97, 0x48, 0xf7, 0x15, 0x0d, 0xf4, 0xb5, 0x51,
- 0xf7, 0x7b, 0x65, 0x6a, 0xf7, 0x13, 0x8a, 0x0c, 0xf9, 0x5f, 0x77, 0x92, 0xff, 0xbd, 0x36, 0x9b,
- 0xff, 0x85, 0x14, 0x09, 0x07, 0xfc, 0xf0, 0x0b, 0x1d, 0x70, 0x77, 0x3e, 0x07, 0x0c, 0x89, 0x27,
- 0x78, 0xe0, 0xd1, 0xb3, 0x3d, 0x70, 0x7b, 0x0e, 0x0f, 0x0c, 0xa9, 0xc7, 0xb9, 0xa0, 0x35, 0xd1,
- 0x05, 0x5f, 0x9f, 0xd1, 0x05, 0x43, 0xba, 0xa4, 0x0f, 0x02, 0xf3, 0x11, 0x31, 0x5a, 0xfb, 0xa9,
- 0x02, 0xab, 0x89, 0xb8, 0x83, 0x9e, 0xc2, 0xd9, 0x84, 0x09, 0x86, 0xa2, 0x71, 0x76, 0xb3, 0xb8,
- 0x75, 0x7d, 0x16, 0x33, 0xc4, 0x82, 0xf2, 0x19, 0x6f, 0xfc, 0x40, 0x0d, 0x41, 0x39, 0xe9, 0x87,
- 0xb5, 0x5f, 0x03, 0x9c, 0x99, 0x00, 0x84, 0x56, 0x20, 0x13, 0x2e, 0x90, 0x8c, 0x65, 0x22, 0x07,
- 0xc0, 0x27, 0x86, 0x43, 0xbb, 0x2e, 0xe9, 0xd3, 0x6a, 0x86, 0x2b, 0x7b, 0x77, 0x3e, 0x65, 0xeb,
- 0x07, 0x21, 0x60, 0xd3, 0xf1, 0xc9, 0xb1, 0x16, 0x63, 0x40, 0x3e, 0x94, 0xbc, 0x8e, 0x6b, 0xdb,
- 0x98, 0x2f, 0x4b, 0x5a, 0xcd, 0x72, 0xc6, 0xf6, 0x9c, 0x8c, 0xed, 0x18, 0xa4, 0xe0, 0x1c, 0x62,
- 0x41, 0x3f, 0x56, 0x60, 0xfd, 0xb1, 0xe5, 0x98, 0xee, 0x63, 0xcb, 0xe9, 0xe9, 0xd4, 0x27, 0x86,
- 0x8f, 0x7b, 0x16, 0xa6, 0xd5, 0x1c, 0xa7, 0x7f, 0x30, 0x27, 0xfd, 0x83, 0x00, 0x7a, 0x3f, 0x44,
- 0x16, 0x5a, 0xac, 0x3d, 0x1e, 0x1d, 0x41, 0x87, 0xb0, 0xc8, 0xb7, 0x4e, 0x5a, 0x5d, 0xe0, 0xec,
- 0x6f, 0xcd, 0xc9, 0xde, 0xe0, 0x60, 0x82, 0x50, 0x22, 0x33, 0x33, 0x63, 0xe7, 0xc8, 0x22, 0xae,
- 0xd3, 0xc7, 0x8e, 0x4f, 0xab, 0x8b, 0x27, 0x62, 0xe6, 0x66, 0x0c, 0x52, 0x9a, 0x39, 0xce, 0x82,
- 0x9e, 0xc0, 0x39, 0xea, 0x1b, 0x3e, 0xd6, 0x27, 0x64, 0x26, 0x4b, 0xf3, 0x65, 0x26, 0x67, 0x39,
- 0xf8, 0xb8, 0x21, 0xd5, 0x86, 0xd5, 0x84, 0xd7, 0xa1, 0x32, 0x64, 0x1f, 0xe1, 0x63, 0xe9, 0xea,
- 0xec, 0x11, 0x35, 0x60, 0xe1, 0xc8, 0xb0, 0x07, 0x98, 0xef, 0x00, 0xc5, 0xad, 0x97, 0x52, 0xe8,
- 0xd1, 0x0e, 0x51, 0x35, 0x21, 0xfb, 0x6a, 0xe6, 0xaa, 0xa2, 0xba, 0x50, 0x19, 0xf1, 0xb8, 0x31,
- 0x7c, 0xbb, 0xc3, 0x7c, 0xf5, 0x34, 0x7c, 0x8d, 0x10, 0x36, 0x4e, 0xf8, 0x01, 0x54, 0x27, 0xf9,
- 0xd8, 0x18, 0xde, 0xb7, 0x86, 0x79, 0xaf, 0xa4, 0xe0, 0x4d, 0xa2, 0x1f, 0xc7, 0xd9, 0x3b, 0x50,
- 0x8c, 0xf9, 0xd8, 0x18, 0xc2, 0xeb, 0xc3, 0x84, 0x9b, 0x29, 0x08, 0x39, 0x60, 0xc2, 0xa6, 0x23,
- 0xee, 0x75, 0x32, 0x36, 0x8d, 0xc1, 0xc6, 0x08, 0x6b, 0xff, 0xca, 0x42, 0x45, 0x78, 0xf8, 0xb6,
- 0xe7, 0xd9, 0x56, 0x87, 0xa7, 0xe7, 0xe8, 0x79, 0x28, 0x85, 0xd1, 0x2a, 0x4a, 0x25, 0x8a, 0x61,
- 0x5f, 0xcb, 0x64, 0xa9, 0xb0, 0xe5, 0x78, 0x03, 0x3f, 0x96, 0x0a, 0xf3, 0x76, 0xcb, 0x44, 0x55,
- 0x58, 0xc2, 0x36, 0x66, 0x4c, 0xd5, 0xec, 0x79, 0x65, 0xb3, 0xa4, 0x05, 0x4d, 0xf4, 0x2d, 0xa8,
- 0xb8, 0x03, 0x9f, 0x49, 0x3d, 0x36, 0x7c, 0x4c, 0xfa, 0x06, 0x79, 0x14, 0x44, 0x9f, 0xb4, 0xe1,
- 0x76, 0x44, 0xd9, 0xfa, 0x3d, 0x8e, 0xf8, 0x20, 0x04, 0x14, 0x6b, 0xb2, 0xec, 0x26, 0xba, 0x51,
- 0x1b, 0xc0, 0xa2, 0xfa, 0xa1, 0x3b, 0x70, 0x4c, 0x6c, 0x56, 0x17, 0xce, 0x2b, 0x9b, 0x2b, 0x5b,
- 0x97, 0x52, 0x58, 0xae, 0x45, 0x77, 0x84, 0x4c, 0xbd, 0xe9, 0x0c, 0xfa, 0x5a, 0xc1, 0x0a, 0xda,
- 0xe8, 0x9b, 0x50, 0xee, 0xbb, 0x8e, 0xe5, 0xbb, 0x84, 0x05, 0x54, 0xcb, 0xe9, 0xba, 0x41, 0x8c,
- 0x49, 0x83, 0x7b, 0x27, 0x14, 0x6d, 0x39, 0x5d, 0x57, 0x5b, 0xed, 0x0f, 0xb5, 0xa9, 0xaa, 0xc3,
- 0xc6, 0xd8, 0xa9, 0x8d, 0xf1, 0x87, 0x8b, 0xc3, 0xfe, 0xa0, 0xd6, 0x45, 0x61, 0x55, 0x0f, 0x0a,
- 0xab, 0xfa, 0x41, 0x50, 0x99, 0xc5, 0xbf, 0xfd, 0x27, 0x19, 0xa8, 0xee, 0x62, 0xdb, 0x38, 0xc6,
- 0xe6, 0xa8, 0x0b, 0x1c, 0x40, 0x55, 0xa6, 0x9c, 0xd8, 0x8c, 0xbe, 0x80, 0xce, 0x4a, 0x3c, 0x59,
- 0x5b, 0x3d, 0x8b, 0xe5, 0x74, 0x28, 0xdb, 0x0c, 0x44, 0xd9, 0x20, 0x7a, 0x1b, 0x8a, 0x46, 0x44,
- 0x22, 0xd5, 0xbd, 0x3a, 0xeb, 0xa7, 0xd7, 0xe2, 0x60, 0xe8, 0x36, 0xac, 0x47, 0x1a, 0x33, 0x3d,
- 0x75, 0x93, 0x4d, 0x8e, 0xfb, 0x60, 0x71, 0xeb, 0xec, 0x88, 0xb6, 0xbb, 0xb2, 0x18, 0xd5, 0x50,
- 0x28, 0xc6, 0x74, 0xe4, 0x16, 0xa9, 0xfd, 0x2c, 0x07, 0xeb, 0xe3, 0x8a, 0x1f, 0xf4, 0x06, 0x9c,
- 0x9b, 0x98, 0xe6, 0x44, 0x4b, 0xe5, 0xec, 0x84, 0x4c, 0xa5, 0x65, 0x22, 0x0b, 0x4a, 0x1d, 0x36,
- 0x53, 0xdd, 0x77, 0x1f, 0x61, 0x27, 0xc8, 0x36, 0x6e, 0xcc, 0x51, 0x90, 0xd5, 0x1b, 0x4c, 0xea,
- 0x80, 0xc1, 0x69, 0xc5, 0x4e, 0xf8, 0x4c, 0xd5, 0xdf, 0x67, 0x00, 0xa2, 0x31, 0xf4, 0x3e, 0xc0,
- 0x80, 0x62, 0xa2, 0xf3, 0x0d, 0x44, 0x7e, 0xc4, 0xf6, 0xc9, 0xf0, 0xd6, 0xef, 0x53, 0x4c, 0xf6,
- 0x19, 0xee, 0xad, 0x53, 0x5a, 0x61, 0x10, 0x34, 0x18, 0x25, 0xb5, 0x4c, 0xac, 0xf3, 0xd0, 0x20,
- 0x3f, 0xf7, 0x49, 0x51, 0xee, 0x5b, 0x26, 0x6e, 0x31, 0x5c, 0x46, 0x49, 0x83, 0x06, 0xab, 0x70,
- 0xb8, 0x65, 0xab, 0xc0, 0x63, 0x8f, 0x68, 0xa8, 0x45, 0x28, 0x84, 0x2a, 0xaa, 0x2f, 0x42, 0x21,
- 0x14, 0x46, 0xcf, 0x0d, 0xa9, 0x28, 0x3e, 0x5f, 0x04, 0xb7, 0xb3, 0x08, 0x39, 0xff, 0xd8, 0xc3,
- 0xb5, 0xcf, 0x32, 0xb0, 0x31, 0xb6, 0x1a, 0x41, 0xb7, 0x60, 0x49, 0x9e, 0x53, 0x48, 0x9b, 0xd6,
- 0x53, 0x4e, 0xf0, 0x8e, 0x90, 0xd2, 0x02, 0x71, 0x56, 0x2e, 0x11, 0x4c, 0x2d, 0x73, 0x60, 0xd8,
- 0x3a, 0x71, 0x5d, 0x3f, 0x70, 0x8e, 0x37, 0x52, 0x02, 0x4e, 0x5a, 0xcc, 0xda, 0x72, 0x00, 0xab,
- 0x31, 0xd4, 0xb1, 0x71, 0x2b, 0x7b, 0x52, 0x71, 0x0b, 0x5d, 0x86, 0x0d, 0xb6, 0xa0, 0x2c, 0x82,
- 0xa9, 0x2e, 0x6b, 0x08, 0xb1, 0xda, 0x73, 0xe7, 0x95, 0xcd, 0xbc, 0xb6, 0x1e, 0x0c, 0xde, 0x88,
- 0x8d, 0xd5, 0x9a, 0x70, 0xee, 0x59, 0xb5, 0x7f, 0xca, 0xf2, 0xb6, 0xf6, 0xf1, 0x1a, 0x2c, 0x49,
- 0xb3, 0x22, 0x03, 0x8a, 0x5e, 0x2c, 0xab, 0x57, 0xa6, 0x32, 0xa5, 0x04, 0xa9, 0xb7, 0xfd, 0x44,
- 0x1a, 0x1f, 0xc7, 0x54, 0x3f, 0x2b, 0x02, 0x44, 0xc9, 0x11, 0x7a, 0x0a, 0x41, 0x8d, 0xc6, 0x62,
- 0xa6, 0xd8, 0xf3, 0x02, 0xa7, 0xb8, 0x3d, 0x2d, 0x71, 0x08, 0x1b, 0x2c, 0x04, 0x6c, 0x36, 0x25,
- 0xa4, 0x56, 0xf1, 0x92, 0x5d, 0xe8, 0x7d, 0x58, 0x35, 0x3a, 0xbe, 0x75, 0x84, 0x23, 0x62, 0xb1,
- 0xdc, 0x6e, 0xcd, 0x4e, 0xbc, 0xcd, 0x01, 0x43, 0xd6, 0x15, 0x63, 0xa8, 0x8d, 0x2c, 0x80, 0xd8,
- 0x36, 0x2e, 0x1c, 0xa8, 0x35, 0x3b, 0x5b, 0x72, 0x07, 0x8f, 0x81, 0xa3, 0x9b, 0x90, 0x63, 0x41,
- 0x45, 0xe6, 0x0a, 0x97, 0xa7, 0x24, 0x61, 0x2b, 0x5f, 0xe3, 0x00, 0xea, 0x9f, 0xb3, 0x90, 0xbf,
- 0x83, 0x0d, 0x3a, 0x20, 0xd8, 0x44, 0x3f, 0x51, 0x60, 0x5d, 0x24, 0x31, 0xd2, 0x66, 0x7a, 0xc7,
- 0x1d, 0x88, 0x4f, 0xc6, 0x68, 0xde, 0x9e, 0x7d, 0x2e, 0x01, 0x45, 0x9d, 0x07, 0x11, 0x69, 0xb1,
- 0x06, 0x07, 0x17, 0x93, 0x43, 0xd6, 0xc8, 0x00, 0xfa, 0x48, 0x81, 0x0d, 0x99, 0x1e, 0x25, 0xf4,
- 0x11, 0x61, 0xe0, 0x9d, 0x13, 0xd0, 0x47, 0x64, 0x14, 0x63, 0x14, 0x5a, 0x73, 0x47, 0x47, 0xd0,
- 0x26, 0x94, 0x7d, 0xd7, 0x37, 0x6c, 0xb1, 0x9d, 0x52, 0x2f, 0x48, 0xe9, 0x14, 0x6d, 0x85, 0xf7,
- 0xb3, 0xfd, 0x72, 0x9f, 0xf5, 0xaa, 0x4d, 0x38, 0x33, 0x61, 0xaa, 0x63, 0xd2, 0x95, 0xf5, 0x78,
- 0xba, 0x92, 0x8d, 0xe7, 0xbf, 0x37, 0xa0, 0x3a, 0x49, 0xc3, 0xa9, 0x70, 0x28, 0x54, 0x46, 0x56,
- 0x0d, 0x7a, 0x0f, 0xf2, 0x7d, 0x69, 0x07, 0xb9, 0x28, 0x77, 0xe6, 0xb7, 0xa8, 0x16, 0x62, 0xaa,
- 0x1f, 0x65, 0x61, 0x65, 0x78, 0xc9, 0x7c, 0xd9, 0x94, 0xe8, 0x25, 0x40, 0x5d, 0x62, 0x88, 0x98,
- 0x48, 0x70, 0xdf, 0xb0, 0x1c, 0xcb, 0xe9, 0x71, 0x73, 0x28, 0x5a, 0x25, 0x18, 0xd1, 0x82, 0x01,
- 0xf4, 0x4b, 0x05, 0xce, 0x0e, 0x7b, 0x18, 0x8d, 0x89, 0x89, 0x15, 0x8c, 0x4f, 0x2a, 0x5e, 0x0c,
- 0xfb, 0x1a, 0x0d, 0xb5, 0x10, 0xfe, 0x76, 0xc6, 0x1d, 0x3f, 0xaa, 0xbe, 0x05, 0xe7, 0x9e, 0x25,
- 0x38, 0x95, 0x1b, 0xbc, 0x0e, 0xab, 0x5f, 0x9c, 0x3c, 0x4f, 0x16, 0xff, 0xe3, 0x02, 0xe4, 0x58,
- 0xec, 0x40, 0x3a, 0x14, 0xc5, 0x1e, 0xad, 0x3b, 0x46, 0x98, 0xff, 0x5e, 0x9f, 0x21, 0x0a, 0xc9,
- 0xc6, 0x5d, 0xa3, 0x8f, 0x35, 0xe8, 0x87, 0xcf, 0x08, 0x43, 0x89, 0x2f, 0x75, 0x4c, 0x74, 0xd3,
- 0xf0, 0x8d, 0xe0, 0x98, 0xf4, 0x8d, 0x59, 0x28, 0x1a, 0x02, 0x68, 0xd7, 0xf0, 0x8d, 0x5b, 0xa7,
- 0xb4, 0x62, 0x27, 0x6a, 0x22, 0x1f, 0x2a, 0xa6, 0x45, 0x7d, 0x62, 0x1d, 0x8a, 0x6c, 0x9e, 0x73,
- 0x4d, 0x79, 0x42, 0x3a, 0xc4, 0xb5, 0x1b, 0x43, 0x93, 0x84, 0x65, 0x33, 0xd1, 0x87, 0x74, 0x80,
- 0x9e, 0x31, 0xe8, 0x61, 0x41, 0xf7, 0xf9, 0x74, 0xe7, 0x93, 0x43, 0x74, 0x37, 0x19, 0x8c, 0xe4,
- 0x29, 0xf4, 0x82, 0x86, 0x7a, 0x1d, 0x20, 0xb2, 0x2b, 0x3a, 0x07, 0x05, 0xf6, 0x95, 0xa8, 0x67,
- 0x74, 0xb0, 0x2c, 0x4d, 0xa3, 0x0e, 0x84, 0x20, 0xc7, 0xbf, 0x61, 0x96, 0x0f, 0xf0, 0x67, 0xf5,
- 0x7f, 0x59, 0x69, 0x1f, 0x59, 0x29, 0x74, 0x08, 0x25, 0xe6, 0x10, 0xea, 0x7b, 0x50, 0x4e, 0xce,
- 0x96, 0xbd, 0xc9, 0xcd, 0x1b, 0xbc, 0xc9, 0x1b, 0xcc, 0xc5, 0xe8, 0xa0, 0x2f, 0xdd, 0x89, 0x3d,
- 0xb2, 0x9e, 0xbe, 0xe5, 0x70, 0xce, 0xac, 0xc6, 0x1e, 0x79, 0x8f, 0xf1, 0x84, 0xa7, 0x44, 0xac,
- 0xc7, 0x78, 0xa2, 0xbe, 0x03, 0x85, 0x70, 0x7a, 0xe3, 0x55, 0x40, 0x57, 0xa1, 0x10, 0x5e, 0xb1,
- 0xa5, 0x28, 0xf5, 0xa2, 0x97, 0x59, 0x16, 0xcb, 0x8c, 0xaf, 0x1e, 0x43, 0x39, 0x99, 0xd1, 0x8c,
- 0x59, 0x11, 0xf7, 0x86, 0xcb, 0xc9, 0x6b, 0x33, 0x47, 0x84, 0x78, 0xb5, 0xf9, 0x9b, 0x0c, 0x3c,
- 0xf7, 0xcc, 0xd3, 0xf5, 0x13, 0x4c, 0xa4, 0xbf, 0xdc, 0x04, 0xf7, 0x5d, 0x58, 0xf6, 0x88, 0xd5,
- 0x37, 0xc8, 0xb1, 0xcc, 0xd2, 0x45, 0x56, 0x32, 0x7b, 0x19, 0x5b, 0x92, 0x70, 0x3c, 0x3b, 0xaf,
- 0x7d, 0x27, 0x07, 0x67, 0x27, 0x5e, 0x45, 0xa5, 0xbd, 0xe7, 0x79, 0x0a, 0x2b, 0x26, 0xa6, 0x16,
- 0xc1, 0xa6, 0xb8, 0x89, 0x08, 0xe6, 0xbf, 0x3f, 0xef, 0x5d, 0x58, 0x7d, 0x57, 0xc0, 0xf2, 0x3e,
- 0x99, 0x3b, 0x2c, 0x9b, 0xf1, 0x3e, 0xf5, 0x77, 0x0a, 0x94, 0xe2, 0x6f, 0xa1, 0x2d, 0xd8, 0x08,
- 0x77, 0x29, 0xb7, 0x2b, 0x77, 0x1c, 0x13, 0x8b, 0x4b, 0xda, 0x8c, 0xb6, 0x16, 0x0c, 0xde, 0xeb,
- 0x6a, 0xc1, 0x10, 0xba, 0x08, 0xeb, 0x86, 0x6d, 0xbb, 0x8f, 0x83, 0x09, 0xe8, 0xe2, 0x72, 0x9a,
- 0x4f, 0x23, 0xab, 0x21, 0x39, 0xc6, 0xf1, 0xdb, 0x7c, 0x04, 0x5d, 0x85, 0x2a, 0xa6, 0xbe, 0xd5,
- 0x37, 0x58, 0xfd, 0x3f, 0x94, 0xd6, 0x51, 0xb9, 0x16, 0x4f, 0x87, 0xe3, 0xf1, 0x5c, 0x85, 0xaa,
- 0x1f, 0x29, 0x80, 0x46, 0xa7, 0x35, 0x66, 0x61, 0x74, 0x86, 0x17, 0xc6, 0x9d, 0x13, 0x35, 0x66,
- 0x7c, 0xb1, 0xfc, 0x33, 0x0b, 0xea, 0xe4, 0xcb, 0xa0, 0x51, 0x0f, 0x54, 0x4e, 0xd2, 0x03, 0xff,
- 0x63, 0x75, 0xe8, 0x00, 0x56, 0x3a, 0x0f, 0x0d, 0xc7, 0xc1, 0xf6, 0xb0, 0x93, 0xde, 0x9d, 0xfb,
- 0xba, 0xac, 0xde, 0x10, 0xb8, 0xa2, 0x73, 0xb9, 0x13, 0x6b, 0x51, 0xf5, 0x13, 0x05, 0x4a, 0xf1,
- 0xf1, 0x34, 0xc7, 0x9d, 0x17, 0x61, 0xdd, 0x36, 0xa8, 0xaf, 0x07, 0x66, 0x0f, 0x0e, 0x38, 0x99,
- 0x23, 0x2c, 0x68, 0x88, 0x8d, 0xb5, 0xc5, 0x90, 0xf4, 0x2a, 0x74, 0x05, 0x4e, 0x77, 0x2d, 0x42,
- 0x7d, 0x3d, 0x34, 0x65, 0xfc, 0x50, 0x74, 0x41, 0x5b, 0xe7, 0xa3, 0x9a, 0x1c, 0x94, 0x52, 0xb5,
- 0xeb, 0xb0, 0x31, 0xf6, 0x52, 0x38, 0x6d, 0x01, 0x5c, 0x85, 0xd3, 0xe3, 0x6f, 0xf4, 0x6a, 0x9f,
- 0x2a, 0x90, 0x0f, 0xf3, 0xd2, 0x5b, 0x62, 0x3f, 0x90, 0x7e, 0x73, 0x25, 0xa5, 0xbd, 0xc3, 0xcc,
- 0x8e, 0xed, 0x51, 0x9a, 0xd8, 0x51, 0x4c, 0xc8, 0xf1, 0x1d, 0x2b, 0x65, 0x5c, 0x4a, 0x9a, 0x3a,
- 0x33, 0x6a, 0x6a, 0x24, 0x75, 0x13, 0x67, 0xc7, 0xfc, 0xb9, 0xf6, 0xf3, 0x2c, 0x94, 0xf8, 0xd9,
- 0x4d, 0x60, 0x8e, 0xe4, 0x0d, 0xde, 0x28, 0x7d, 0x66, 0x1c, 0xfd, 0x1e, 0x14, 0xc4, 0xdd, 0x0c,
- 0x5b, 0xd8, 0xe2, 0x60, 0xf0, 0x42, 0xca, 0xc9, 0x73, 0xfa, 0xdb, 0xf8, 0x58, 0xcb, 0x53, 0xf9,
- 0x84, 0x6e, 0x43, 0xb6, 0x87, 0xfd, 0x69, 0x7f, 0xd8, 0xe0, 0x40, 0x37, 0x71, 0xec, 0xe7, 0x02,
- 0x86, 0x82, 0x0e, 0x60, 0xd1, 0xf0, 0x3c, 0xec, 0x98, 0x41, 0xf2, 0x77, 0x6d, 0x1a, 0xbc, 0x6d,
- 0x2e, 0x1a, 0x41, 0x4a, 0x2c, 0xf4, 0x55, 0x58, 0xe8, 0xd8, 0xd8, 0x20, 0x41, 0x96, 0x77, 0x75,
- 0x1a, 0xd0, 0x06, 0x93, 0x8c, 0x30, 0x05, 0x52, 0xfc, 0x67, 0x84, 0x4f, 0x33, 0xb0, 0x2c, 0x3f,
- 0x8b, 0x8c, 0x4c, 0xc9, 0xef, 0x32, 0xfe, 0x7f, 0x83, 0xbd, 0x21, 0xc3, 0xbd, 0x32, 0xb5, 0xe1,
- 0xc2, 0x4b, 0x6a, 0x6e, 0xb9, 0xfb, 0x49, 0xcb, 0xbd, 0x3a, 0x8b, 0xe5, 0x42, 0xcc, 0xc0, 0x74,
- 0x5a, 0xc2, 0x74, 0xd7, 0x66, 0x30, 0x5d, 0x08, 0x2a, 0x6d, 0x17, 0xbf, 0x44, 0xff, 0xc3, 0x22,
- 0xe4, 0x03, 0xa7, 0x42, 0x6d, 0x58, 0x14, 0xbf, 0x64, 0xc9, 0xd4, 0xe7, 0xe5, 0x29, 0xbd, 0xb2,
- 0xae, 0x71, 0x69, 0xa6, 0xbe, 0xc0, 0x41, 0x14, 0xd6, 0xfa, 0x03, 0x9b, 0xed, 0x77, 0x9e, 0x3e,
- 0x72, 0x06, 0xbb, 0x3d, 0x2d, 0xfc, 0x1d, 0x09, 0x15, 0x3f, 0x74, 0xad, 0xf4, 0x93, 0x9d, 0xc8,
- 0x84, 0x95, 0x43, 0xa3, 0xa7, 0xc7, 0x8e, 0x99, 0xb3, 0x53, 0xfd, 0xef, 0x11, 0xf2, 0xed, 0x18,
- 0xbd, 0xf8, 0x91, 0x72, 0xe9, 0x30, 0xd6, 0x66, 0x53, 0xb3, 0x7c, 0x4c, 0x8c, 0x43, 0x1b, 0xc7,
- 0xa7, 0x96, 0x9b, 0x6d, 0x6a, 0x2d, 0x09, 0x35, 0x34, 0x35, 0x2b, 0xd9, 0xa9, 0xaa, 0xb0, 0x28,
- 0x6c, 0x1c, 0xcf, 0x0b, 0x4a, 0x3c, 0x2f, 0x50, 0xbf, 0xaf, 0x40, 0x65, 0xc4, 0x42, 0x69, 0xb6,
- 0x95, 0x1a, 0x2c, 0x47, 0x13, 0x88, 0xc5, 0xc3, 0xf0, 0xfc, 0xb9, 0x65, 0xa2, 0xd3, 0xb0, 0x28,
- 0x2e, 0xd0, 0x65, 0x44, 0x94, 0xad, 0x40, 0x8d, 0x5c, 0xa4, 0xc6, 0x87, 0x50, 0x8a, 0xdb, 0x2d,
- 0xa5, 0x02, 0xd1, 0xc7, 0x8a, 0x29, 0x10, 0x1e, 0xe1, 0x4f, 0xa5, 0x00, 0x81, 0xca, 0x88, 0x35,
- 0xbf, 0x64, 0x33, 0x84, 0x07, 0xf4, 0x6f, 0xc2, 0x6a, 0x22, 0xb2, 0xa2, 0x97, 0x00, 0x75, 0x5c,
- 0xc7, 0xb7, 0x9c, 0x81, 0x21, 0x2e, 0xaf, 0xf8, 0xbd, 0x80, 0xf8, 0x6e, 0x95, 0xf8, 0x08, 0xbf,
- 0x50, 0xa8, 0xdd, 0x87, 0x72, 0x32, 0xc4, 0x4c, 0x09, 0x11, 0xee, 0x5d, 0x99, 0xd8, 0xde, 0xb5,
- 0x09, 0x68, 0x34, 0x44, 0x87, 0x6f, 0x2a, 0xb1, 0x37, 0x37, 0x60, 0x6d, 0x4c, 0x48, 0xaa, 0xad,
- 0x41, 0x65, 0x24, 0x1c, 0xd7, 0xd6, 0x25, 0xea, 0x50, 0xa0, 0xa9, 0xfd, 0x2a, 0x07, 0xf9, 0x3d,
- 0x57, 0x9e, 0x94, 0x7c, 0x03, 0xf2, 0x14, 0x1f, 0x61, 0x62, 0xf9, 0xc2, 0x59, 0x57, 0x52, 0x17,
- 0xdd, 0x01, 0x44, 0x7d, 0x5f, 0xca, 0x8b, 0xab, 0xcf, 0x10, 0x6e, 0xf6, 0x4a, 0x14, 0x55, 0x59,
- 0x91, 0x47, 0xa9, 0xd1, 0x0b, 0x4a, 0xf0, 0xa0, 0xc9, 0x2f, 0x6e, 0x08, 0xab, 0xd9, 0x73, 0x62,
- 0xab, 0xe0, 0x8d, 0x31, 0x1b, 0xfb, 0x42, 0x9a, 0xbc, 0x62, 0x71, 0xd4, 0xc9, 0x9e, 0x87, 0x92,
- 0xed, 0xf6, 0x74, 0xdb, 0x95, 0x97, 0x8f, 0x4b, 0xe2, 0x15, 0xdb, 0xed, 0xed, 0xc9, 0x2e, 0xe6,
- 0x63, 0xfe, 0x43, 0x82, 0x0d, 0xb3, 0x9a, 0xe7, 0x83, 0xb2, 0xa5, 0x7e, 0x1d, 0x72, 0x7b, 0x16,
- 0xf5, 0x51, 0x1b, 0xd8, 0xeb, 0x3a, 0x76, 0x7c, 0x62, 0xe1, 0x20, 0xeb, 0xbe, 0x30, 0xa5, 0x51,
- 0x35, 0xb0, 0xc5, 0x93, 0x85, 0xa9, 0x4a, 0x20, 0x1f, 0xd8, 0xb8, 0xd6, 0x85, 0x1c, 0x33, 0x33,
- 0x5a, 0x85, 0xe2, 0xfd, 0xbb, 0xfb, 0xed, 0x66, 0xa3, 0x75, 0xa3, 0xd5, 0xdc, 0x2d, 0x9f, 0x42,
- 0x05, 0x58, 0x38, 0xd0, 0xb6, 0x1b, 0xcd, 0xb2, 0xc2, 0x1e, 0x77, 0x9b, 0x3b, 0xf7, 0x6f, 0x96,
- 0x33, 0x28, 0x0f, 0xb9, 0xd6, 0xdd, 0x1b, 0xf7, 0xca, 0x59, 0x04, 0xb0, 0x78, 0xf7, 0xde, 0x41,
- 0xab, 0xd1, 0x2c, 0xe7, 0x58, 0xef, 0x83, 0x6d, 0xed, 0x6e, 0x79, 0x81, 0xbd, 0xda, 0xd4, 0xb4,
- 0x7b, 0x5a, 0x79, 0x11, 0x95, 0x20, 0xdf, 0xd0, 0x5a, 0x07, 0xad, 0xc6, 0xf6, 0x5e, 0x79, 0xa9,
- 0x56, 0x02, 0xd8, 0x73, 0x7b, 0x0d, 0xd7, 0xf1, 0x89, 0x6b, 0xd7, 0xfe, 0x94, 0xe3, 0x9e, 0x44,
- 0xfc, 0x07, 0x2e, 0x79, 0x14, 0xfd, 0xce, 0xf5, 0x5f, 0x50, 0x78, 0xcc, 0x3b, 0xa2, 0x25, 0x9b,
- 0x17, 0x1d, 0x2d, 0x13, 0x1d, 0x42, 0xb9, 0x23, 0xc4, 0xf5, 0xe0, 0xb7, 0x61, 0xe9, 0x05, 0x33,
- 0xff, 0xd6, 0xb2, 0x2a, 0x01, 0x9b, 0x12, 0x8f, 0x71, 0xd8, 0x6e, 0xaf, 0xc7, 0x0a, 0xf8, 0x90,
- 0x23, 0x3b, 0x27, 0x87, 0x04, 0x0c, 0x39, 0x4c, 0xa8, 0x18, 0xc4, 0xb7, 0xba, 0x46, 0xc7, 0x8f,
- 0x48, 0x72, 0xf3, 0x91, 0x94, 0x03, 0xc4, 0x90, 0xa5, 0xcb, 0xaf, 0x85, 0x8e, 0x2c, 0xca, 0x1c,
- 0x38, 0xa4, 0x59, 0x98, 0x8f, 0xa6, 0x12, 0x42, 0x86, 0x3c, 0xef, 0xc2, 0xa2, 0x67, 0x10, 0xa3,
- 0x4f, 0xab, 0xc0, 0x1d, 0xb3, 0x99, 0x7e, 0x27, 0x4c, 0x7c, 0xfd, 0x7a, 0x9b, 0xe3, 0xc8, 0xbf,
- 0xa9, 0x04, 0xa8, 0x7a, 0x0d, 0x8a, 0xb1, 0xee, 0x2f, 0x3a, 0x48, 0x2d, 0xc4, 0xcb, 0xd9, 0xff,
- 0xe7, 0x81, 0x2d, 0x22, 0x91, 0xc1, 0x35, 0x4c, 0x0e, 0x95, 0x58, 0x72, 0x58, 0xbb, 0xc8, 0xc2,
- 0x9d, 0xeb, 0xa5, 0x77, 0xc7, 0xda, 0x8b, 0xcc, 0x83, 0x23, 0x89, 0x67, 0xa1, 0x6f, 0x7d, 0xac,
- 0xc0, 0xf2, 0x0e, 0x36, 0xfa, 0x37, 0x1c, 0xb9, 0x00, 0xd0, 0x0f, 0x14, 0x58, 0x0a, 0x9e, 0xd3,
- 0x66, 0x8e, 0x63, 0xfe, 0xc0, 0x55, 0xaf, 0xcd, 0x22, 0x2b, 0x82, 0xf9, 0xa9, 0x4d, 0xe5, 0xa2,
- 0xb2, 0xf5, 0x01, 0x80, 0xd0, 0x8c, 0x17, 0x54, 0x8e, 0x2c, 0xac, 0x2e, 0x4c, 0x59, 0x9c, 0xa9,
- 0xd3, 0x0a, 0x48, 0xf6, 0x1f, 0x2a, 0x50, 0x14, 0xf4, 0x22, 0x79, 0x78, 0x02, 0x0b, 0xe2, 0xe1,
- 0xf2, 0x34, 0x09, 0x95, 0x9c, 0x91, 0x7a, 0x65, 0x3a, 0x21, 0xb9, 0x7d, 0x09, 0x4d, 0x7e, 0x14,
- 0x7e, 0xa2, 0x3d, 0xb1, 0x5e, 0xd1, 0x13, 0x58, 0x0a, 0x1e, 0xaf, 0x4c, 0xbb, 0x85, 0xb1, 0xc0,
- 0xad, 0x5e, 0x4a, 0x2f, 0x15, 0xc4, 0x45, 0xa1, 0xcb, 0x6f, 0x33, 0x50, 0x15, 0xba, 0x34, 0x9f,
- 0xf8, 0x98, 0x38, 0x86, 0x2d, 0xbc, 0xac, 0xed, 0x0a, 0xcf, 0x29, 0xc6, 0xfc, 0x1a, 0x5d, 0x9b,
- 0x79, 0xc1, 0xa9, 0xaf, 0xce, 0x22, 0x1a, 0x58, 0x0d, 0x7d, 0x4f, 0x01, 0x88, 0x56, 0x00, 0x4a,
- 0x5f, 0xe4, 0x25, 0x96, 0x99, 0x7a, 0x6d, 0x06, 0xc9, 0x40, 0x8b, 0x9d, 0x6d, 0xf8, 0xbf, 0x49,
- 0xd2, 0x71, 0xe1, 0x9d, 0x82, 0x30, 0xe8, 0xb6, 0x67, 0xbd, 0xbd, 0x12, 0x1b, 0xd2, 0x8f, 0x2e,
- 0x1d, 0x2e, 0xf2, 0xe4, 0xe1, 0xf2, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0xd0, 0x49, 0x45, 0xe8,
- 0xdf, 0x32, 0x00, 0x00,
+var fileDescriptor_beam_fn_api_232aec3131d70c18 = []byte{
+ // 3241 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xdd, 0x6f, 0x1b, 0xc7,
+ 0xb5, 0xf7, 0xf2, 0x43, 0x22, 0x0f, 0x29, 0x89, 0x1c, 0x49, 0x36, 0xbd, 0xd7, 0xb9, 0xd7, 0xe1,
+ 0xbd, 0x01, 0x84, 0x5c, 0x84, 0xb6, 0x65, 0x23, 0xb1, 0x73, 0x13, 0x27, 0x12, 0x45, 0xdb, 0x8c,
+ 0x65, 0x9b, 0x77, 0x25, 0x5f, 0xdf, 0x9b, 0xdc, 0x64, 0xb1, 0xe2, 0x0e, 0xe9, 0x85, 0xc9, 0xdd,
+ 0xcd, 0xcc, 0x52, 0xb6, 0xdc, 0xa0, 0xe9, 0x17, 0x5a, 0xb4, 0x68, 0x9b, 0x97, 0x3e, 0x24, 0x7d,
+ 0x6b, 0x0b, 0x14, 0xe8, 0x4b, 0xff, 0x80, 0xfc, 0x03, 0x45, 0x9f, 0xfa, 0x0f, 0xe4, 0xa5, 0x40,
+ 0x0b, 0xb4, 0x4d, 0x9f, 0x0b, 0xf4, 0xad, 0x98, 0x8f, 0xfd, 0xe0, 0x72, 0xe9, 0x2c, 0x29, 0xb9,
+ 0x6f, 0x3b, 0x73, 0xf6, 0xfc, 0x7e, 0x33, 0x67, 0xcf, 0x9c, 0x39, 0x67, 0x66, 0xa1, 0x7a, 0x80,
+ 0x8d, 0xa1, 0xde, 0xb3, 0x75, 0xc3, 0xb5, 0x1a, 0x2e, 0x71, 0x3c, 0x07, 0xbd, 0xe4, 0x90, 0x7e,
+ 0xc3, 0x70, 0x8d, 0xee, 0x43, 0xdc, 0x60, 0xd2, 0xc6, 0xd0, 0x31, 0xf1, 0xa0, 0xd1, 0xb3, 0x75,
+ 0xfc, 0x04, 0x77, 0x47, 0x9e, 0xe5, 0xd8, 0x8d, 0xc3, 0x4b, 0xea, 0x3a, 0xd7, 0x24, 0x23, 0xdb,
+ 0xc6, 0x24, 0xd4, 0x56, 0x57, 0xb0, 0x6d, 0xba, 0x8e, 0x65, 0x7b, 0x54, 0x76, 0x9c, 0xef, 0x3b,
+ 0x4e, 0x7f, 0x80, 0x2f, 0xf0, 0xd6, 0xc1, 0xa8, 0x77, 0xc1, 0xc4, 0xb4, 0x4b, 0x2c, 0xd7, 0x73,
+ 0x88, 0x7c, 0xe3, 0xdf, 0xe2, 0x6f, 0x78, 0xd6, 0x10, 0x53, 0xcf, 0x18, 0xba, 0xf2, 0x85, 0x7f,
+ 0x9d, 0x80, 0x18, 0x11, 0x83, 0x8f, 0x63, 0x8a, 0xfc, 0x31, 0x31, 0x5c, 0x17, 0x13, 0x7f, 0x08,
+ 0x4b, 0x43, 0xec, 0x11, 0xab, 0x2b, 0x9b, 0xf5, 0x9f, 0x2a, 0xb0, 0xac, 0xe1, 0xa1, 0xe3, 0xe1,
+ 0x9b, 0xc4, 0xed, 0x76, 0x1c, 0xe2, 0xa1, 0x21, 0x9c, 0x36, 0x5c, 0x4b, 0xa7, 0x98, 0x1c, 0x5a,
+ 0x5d, 0xac, 0x87, 0x43, 0xac, 0x29, 0xe7, 0x95, 0x8d, 0xd2, 0xe6, 0x6b, 0x8d, 0x64, 0xa3, 0xb8,
+ 0x96, 0x8b, 0x07, 0x96, 0x8d, 0x1b, 0x87, 0x97, 0x1a, 0x5b, 0xae, 0xb5, 0x27, 0xf4, 0x77, 0x02,
+ 0x75, 0x6d, 0xcd, 0x48, 0xe8, 0x45, 0x67, 0xa1, 0xd0, 0x75, 0x4c, 0x4c, 0x74, 0xcb, 0xac, 0x65,
+ 0xce, 0x2b, 0x1b, 0x45, 0x6d, 0x91, 0xb7, 0xdb, 0x66, 0xfd, 0x8f, 0x39, 0x40, 0x6d, 0x9b, 0x7a,
+ 0x64, 0xd4, 0x65, 0x33, 0xd4, 0xf0, 0x87, 0x23, 0x4c, 0x3d, 0xf4, 0x12, 0x2c, 0x5b, 0x61, 0x2f,
+ 0xd3, 0x53, 0xb8, 0xde, 0x52, 0xa4, 0xb7, 0x6d, 0xa2, 0xfb, 0x50, 0x20, 0xb8, 0x6f, 0x51, 0x0f,
+ 0x93, 0xda, 0x9f, 0x16, 0xf9, 0xd0, 0x5f, 0x6d, 0xa4, 0xfa, 0x9e, 0x0d, 0x4d, 0xea, 0x49, 0xc6,
+ 0x5b, 0xa7, 0xb4, 0x00, 0x0a, 0x61, 0x58, 0x76, 0x89, 0xd3, 0xc5, 0x94, 0xea, 0x07, 0x23, 0xdb,
+ 0x1c, 0xe0, 0xda, 0x9f, 0x05, 0xf8, 0x7f, 0xa5, 0x04, 0xef, 0x08, 0xed, 0x6d, 0xae, 0x1c, 0x32,
+ 0x2c, 0xb9, 0xd1, 0x7e, 0xf4, 0x75, 0x38, 0x33, 0x4e, 0xa3, 0xbb, 0xc4, 0xe9, 0x13, 0x4c, 0x69,
+ 0xed, 0x2f, 0x82, 0xaf, 0x39, 0x0f, 0x5f, 0x47, 0x82, 0x84, 0xbc, 0xeb, 0x6e, 0x92, 0x1c, 0x8d,
+ 0x60, 0x2d, 0xc6, 0x4f, 0xdd, 0x81, 0xe5, 0xd5, 0xbe, 0x14, 0xe4, 0x6f, 0xcf, 0x43, 0xbe, 0xc7,
+ 0x10, 0x42, 0x66, 0xe4, 0x4e, 0x08, 0xd1, 0x43, 0x58, 0xe9, 0x59, 0xb6, 0x31, 0xb0, 0x9e, 0x62,
+ 0xdf, 0xbc, 0x7f, 0x15, 0x8c, 0x6f, 0xa4, 0x64, 0xbc, 0x21, 0xd5, 0xe3, 0xf6, 0x5d, 0xee, 0x8d,
+ 0x09, 0xb6, 0x8b, 0xb0, 0x48, 0x84, 0xb0, 0xfe, 0xad, 0x3c, 0xac, 0x8e, 0xf9, 0x19, 0x75, 0x1d,
+ 0x9b, 0xe2, 0xb4, 0x8e, 0xb6, 0x06, 0x79, 0x4c, 0x88, 0x43, 0xa4, 0xfb, 0x8a, 0x06, 0xfa, 0x9f,
+ 0x49, 0xf7, 0x7b, 0x6d, 0x66, 0xf7, 0x13, 0x03, 0x19, 0xf3, 0xbf, 0xde, 0x34, 0xff, 0x7b, 0x63,
+ 0x3e, 0xff, 0x0b, 0x28, 0x62, 0x0e, 0xf8, 0xf1, 0x57, 0x3a, 0xe0, 0xce, 0xf1, 0x1c, 0x30, 0x20,
+ 0x9e, 0xe2, 0x81, 0x87, 0xcf, 0xf6, 0xc0, 0xad, 0x63, 0x78, 0x60, 0x40, 0x9d, 0xe4, 0x82, 0xd6,
+ 0x54, 0x17, 0x7c, 0x73, 0x4e, 0x17, 0x0c, 0xe8, 0xe2, 0x3e, 0x08, 0xcc, 0x47, 0x84, 0xb4, 0xfe,
+ 0x23, 0x05, 0x56, 0x62, 0x71, 0x07, 0x3d, 0x85, 0xb3, 0x31, 0x13, 0x8c, 0x45, 0xe3, 0xec, 0x46,
+ 0x69, 0xf3, 0xfa, 0x3c, 0x66, 0x88, 0x04, 0xe5, 0x33, 0x6e, 0xb2, 0xa0, 0x8e, 0xa0, 0x12, 0xf7,
+ 0xc3, 0xfa, 0x2f, 0x00, 0xce, 0x4c, 0x01, 0x42, 0xcb, 0x90, 0x09, 0x16, 0x48, 0xc6, 0x32, 0x91,
+ 0x0d, 0xe0, 0x11, 0xc3, 0xa6, 0x3d, 0x87, 0x0c, 0x69, 0x2d, 0xc3, 0x07, 0x7b, 0xf7, 0x78, 0x83,
+ 0x6d, 0xec, 0x07, 0x80, 0x2d, 0xdb, 0x23, 0x47, 0x5a, 0x84, 0x01, 0x79, 0x50, 0x76, 0xbb, 0xce,
+ 0x60, 0x80, 0xf9, 0xb2, 0xa4, 0xb5, 0x2c, 0x67, 0xec, 0x1c, 0x93, 0xb1, 0x13, 0x81, 0x14, 0x9c,
+ 0x63, 0x2c, 0xe8, 0x07, 0x0a, 0xac, 0x3d, 0xb6, 0x6c, 0xd3, 0x79, 0x6c, 0xd9, 0x7d, 0x9d, 0x7a,
+ 0xc4, 0xf0, 0x70, 0xdf, 0xc2, 0xb4, 0x96, 0xe3, 0xf4, 0x0f, 0x8e, 0x49, 0xff, 0xc0, 0x87, 0xde,
+ 0x0b, 0x90, 0xc5, 0x28, 0x56, 0x1f, 0x4f, 0x4a, 0xd0, 0x01, 0x2c, 0xf0, 0xad, 0x93, 0xd6, 0xf2,
+ 0x9c, 0xfd, 0x9d, 0x63, 0xb2, 0x37, 0x39, 0x98, 0x20, 0x94, 0xc8, 0xcc, 0xcc, 0xd8, 0x3e, 0xb4,
+ 0x88, 0x63, 0x0f, 0xb1, 0xed, 0xd1, 0xda, 0xc2, 0x89, 0x98, 0xb9, 0x15, 0x81, 0x94, 0x66, 0x8e,
+ 0xb2, 0xa0, 0x27, 0x70, 0x8e, 0x7a, 0x86, 0x87, 0xf5, 0x29, 0x99, 0xc9, 0xe2, 0xf1, 0x32, 0x93,
+ 0xb3, 0x1c, 0x3c, 0x49, 0xa4, 0x0e, 0x60, 0x25, 0xe6, 0x75, 0xa8, 0x02, 0xd9, 0x47, 0xf8, 0x48,
+ 0xba, 0x3a, 0x7b, 0x44, 0x4d, 0xc8, 0x1f, 0x1a, 0x83, 0x11, 0xe6, 0x3b, 0x40, 0x69, 0xf3, 0x95,
+ 0x14, 0xe3, 0xe8, 0x04, 0xa8, 0x9a, 0xd0, 0x7d, 0x3d, 0x73, 0x55, 0x51, 0x1d, 0xa8, 0x4e, 0x78,
+ 0x5c, 0x02, 0xdf, 0xce, 0x38, 0x5f, 0x23, 0x0d, 0x5f, 0x33, 0x80, 0x8d, 0x12, 0x7e, 0x04, 0xb5,
+ 0x69, 0x3e, 0x96, 0xc0, 0xfb, 0xce, 0x38, 0xef, 0x95, 0x14, 0xbc, 0x71, 0xf4, 0xa3, 0x28, 0x7b,
+ 0x17, 0x4a, 0x11, 0x1f, 0x4b, 0x20, 0xbc, 0x3e, 0x4e, 0xb8, 0x91, 0x82, 0x90, 0x03, 0xc6, 0x6c,
+ 0x3a, 0xe1, 0x5e, 0x27, 0x63, 0xd3, 0x08, 0x6c, 0x84, 0xb0, 0xfe, 0xf7, 0x2c, 0x54, 0x85, 0x87,
+ 0x6f, 0xb9, 0xee, 0xc0, 0xea, 0xf2, 0xf4, 0x1c, 0xbd, 0x08, 0xe5, 0x20, 0x5a, 0x85, 0xa9, 0x44,
+ 0x29, 0xe8, 0x6b, 0x9b, 0x2c, 0x15, 0xb6, 0x6c, 0x77, 0xe4, 0x45, 0x52, 0x61, 0xde, 0x6e, 0x9b,
+ 0xa8, 0x06, 0x8b, 0x78, 0x80, 0x19, 0x53, 0x2d, 0x7b, 0x5e, 0xd9, 0x28, 0x6b, 0x7e, 0x13, 0x7d,
+ 0x0d, 0xaa, 0xce, 0xc8, 0x63, 0x5a, 0x8f, 0x0d, 0x0f, 0x93, 0xa1, 0x41, 0x1e, 0xf9, 0xd1, 0x27,
+ 0x6d, 0xb8, 0x9d, 0x18, 0x6c, 0xe3, 0x1e, 0x47, 0x7c, 0x10, 0x00, 0x8a, 0x35, 0x59, 0x71, 0x62,
+ 0xdd, 0xa8, 0x03, 0x60, 0x51, 0xfd, 0xc0, 0x19, 0xd9, 0x26, 0x36, 0x6b, 0xf9, 0xf3, 0xca, 0xc6,
+ 0xf2, 0xe6, 0xa5, 0x14, 0x96, 0x6b, 0xd3, 0x6d, 0xa1, 0xd3, 0x68, 0xd9, 0xa3, 0xa1, 0x56, 0xb4,
+ 0xfc, 0x36, 0xfa, 0x7f, 0xa8, 0x0c, 0x1d, 0xdb, 0xf2, 0x1c, 0xc2, 0x02, 0xaa, 0x65, 0xf7, 0x1c,
+ 0x3f, 0xc6, 0xa4, 0xc1, 0xbd, 0x13, 0xa8, 0xb6, 0xed, 0x9e, 0xa3, 0xad, 0x0c, 0xc7, 0xda, 0x54,
+ 0xd5, 0x61, 0x3d, 0x71, 0x6a, 0x09, 0xfe, 0x70, 0x71, 0xdc, 0x1f, 0xd4, 0x86, 0x28, 0xac, 0x1a,
+ 0x7e, 0x61, 0xd5, 0xd8, 0xf7, 0x2b, 0xb3, 0xe8, 0xb7, 0xff, 0x2c, 0x03, 0xb5, 0x1d, 0x3c, 0x30,
+ 0x8e, 0xb0, 0x39, 0xe9, 0x02, 0xfb, 0x50, 0x93, 0x29, 0x27, 0x36, 0xc3, 0x2f, 0xa0, 0xb3, 0x12,
+ 0x4f, 0xd6, 0x56, 0xcf, 0x62, 0x39, 0x1d, 0xe8, 0xb6, 0x7c, 0x55, 0x26, 0x44, 0xef, 0x42, 0xc9,
+ 0x08, 0x49, 0xe4, 0x70, 0xaf, 0xce, 0xfb, 0xe9, 0xb5, 0x28, 0x18, 0xba, 0x0d, 0x6b, 0xe1, 0x88,
+ 0xd9, 0x38, 0x75, 0x93, 0x4d, 0x8e, 0xfb, 0x60, 0x69, 0xf3, 0xec, 0xc4, 0x68, 0x77, 0x64, 0x31,
+ 0xaa, 0xa1, 0x40, 0x8d, 0x8d, 0x91, 0x5b, 0xa4, 0xfe, 0xe3, 0x1c, 0xac, 0x25, 0x15, 0x3f, 0xe8,
+ 0x2d, 0x38, 0x37, 0x35, 0xcd, 0x09, 0x97, 0xca, 0xd9, 0x29, 0x99, 0x4a, 0xdb, 0x44, 0x16, 0x94,
+ 0xbb, 0x6c, 0xa6, 0xba, 0xe7, 0x3c, 0xc2, 0xb6, 0x9f, 0x6d, 0xdc, 0x38, 0x46, 0x41, 0xd6, 0x68,
+ 0x32, 0xad, 0x7d, 0x06, 0xa7, 0x95, 0xba, 0xc1, 0x33, 0x55, 0x7f, 0x9b, 0x01, 0x08, 0x65, 0xe8,
+ 0x43, 0x80, 0x11, 0xc5, 0x44, 0xe7, 0x1b, 0x88, 0xfc, 0x88, 0x9d, 0x93, 0xe1, 0x6d, 0xdc, 0xa7,
+ 0x98, 0xec, 0x31, 0xdc, 0x5b, 0xa7, 0xb4, 0xe2, 0xc8, 0x6f, 0x30, 0x4a, 0x6a, 0x99, 0x58, 0xe7,
+ 0xa1, 0x41, 0x7e, 0xee, 0x93, 0xa2, 0xdc, 0xb3, 0x4c, 0xdc, 0x66, 0xb8, 0x8c, 0x92, 0xfa, 0x0d,
+ 0x56, 0xe1, 0x70, 0xcb, 0xd6, 0x80, 0xc7, 0x1e, 0xd1, 0x50, 0x4b, 0x50, 0x0c, 0x86, 0xa8, 0xbe,
+ 0x0c, 0xc5, 0x40, 0x19, 0xbd, 0x30, 0x36, 0x44, 0xf1, 0xf9, 0x42, 0xb8, 0xed, 0x05, 0xc8, 0x79,
+ 0x47, 0x2e, 0xae, 0x7f, 0x91, 0x81, 0xf5, 0xc4, 0x6a, 0x04, 0xdd, 0x82, 0x45, 0x79, 0x4e, 0x21,
+ 0x6d, 0xda, 0x48, 0x39, 0xc1, 0x3b, 0x42, 0x4b, 0xf3, 0xd5, 0x59, 0xb9, 0x44, 0x30, 0xb5, 0xcc,
+ 0x91, 0x31, 0xd0, 0x89, 0xe3, 0x78, 0xbe, 0x73, 0xbc, 0x95, 0x12, 0x70, 0xda, 0x62, 0xd6, 0x96,
+ 0x7c, 0x58, 0x8d, 0xa1, 0x26, 0xc6, 0xad, 0xec, 0x49, 0xc5, 0x2d, 0x74, 0x19, 0xd6, 0xd9, 0x82,
+ 0xb2, 0x08, 0xa6, 0xba, 0xac, 0x21, 0xc4, 0x6a, 0xcf, 0x9d, 0x57, 0x36, 0x0a, 0xda, 0x9a, 0x2f,
+ 0xbc, 0x11, 0x91, 0xd5, 0x5b, 0x70, 0xee, 0x59, 0xb5, 0x7f, 0xca, 0xf2, 0xb6, 0xfe, 0xe9, 0x2a,
+ 0x2c, 0x4a, 0xb3, 0x22, 0x03, 0x4a, 0x6e, 0x24, 0xab, 0x57, 0x66, 0x32, 0xa5, 0x04, 0x69, 0x74,
+ 0xbc, 0x58, 0x1a, 0x1f, 0xc5, 0x54, 0xbf, 0x28, 0x01, 0x84, 0xc9, 0x11, 0x7a, 0x0a, 0x7e, 0x8d,
+ 0xc6, 0x62, 0xa6, 0xd8, 0xf3, 0x7c, 0xa7, 0xb8, 0x3d, 0x2b, 0x71, 0x00, 0xeb, 0x2f, 0x04, 0x6c,
+ 0xb6, 0x24, 0xa4, 0x56, 0x75, 0xe3, 0x5d, 0xe8, 0x43, 0x58, 0x31, 0xba, 0x9e, 0x75, 0x88, 0x43,
+ 0x62, 0xb1, 0xdc, 0x6e, 0xcd, 0x4f, 0xbc, 0xc5, 0x01, 0x03, 0xd6, 0x65, 0x63, 0xac, 0x8d, 0x2c,
+ 0x80, 0xc8, 0x36, 0x2e, 0x1c, 0xa8, 0x3d, 0x3f, 0x5b, 0x7c, 0x07, 0x8f, 0x80, 0xa3, 0x9b, 0x90,
+ 0x63, 0x41, 0x45, 0xe6, 0x0a, 0x97, 0x67, 0x24, 0x61, 0x2b, 0x5f, 0xe3, 0x00, 0xea, 0x1f, 0xb2,
+ 0x50, 0xb8, 0x83, 0x0d, 0x3a, 0x22, 0xd8, 0x44, 0x3f, 0x54, 0x60, 0x4d, 0x24, 0x31, 0xd2, 0x66,
+ 0x7a, 0xd7, 0x19, 0x89, 0x4f, 0xc6, 0x68, 0xde, 0x9d, 0x7f, 0x2e, 0x3e, 0x45, 0x83, 0x07, 0x11,
+ 0x69, 0xb1, 0x26, 0x07, 0x17, 0x93, 0x43, 0xd6, 0x84, 0x00, 0x7d, 0xa2, 0xc0, 0xba, 0x4c, 0x8f,
+ 0x62, 0xe3, 0x11, 0x61, 0xe0, 0xbd, 0x13, 0x18, 0x8f, 0xc8, 0x28, 0x12, 0x06, 0xb4, 0xea, 0x4c,
+ 0x4a, 0xd0, 0x06, 0x54, 0x3c, 0xc7, 0x33, 0x06, 0x62, 0x3b, 0xa5, 0xae, 0x9f, 0xd2, 0x29, 0xda,
+ 0x32, 0xef, 0x67, 0xfb, 0xe5, 0x1e, 0xeb, 0x55, 0x5b, 0x70, 0x66, 0xca, 0x54, 0x13, 0xd2, 0x95,
+ 0xb5, 0x68, 0xba, 0x92, 0x8d, 0xe6, 0xbf, 0x37, 0xa0, 0x36, 0x6d, 0x84, 0x33, 0xe1, 0x50, 0xa8,
+ 0x4e, 0xac, 0x1a, 0xf4, 0x01, 0x14, 0x86, 0xd2, 0x0e, 0x72, 0x51, 0x6e, 0x1f, 0xdf, 0xa2, 0x5a,
+ 0x80, 0xa9, 0x7e, 0x92, 0x85, 0xe5, 0xf1, 0x25, 0xf3, 0xbc, 0x29, 0xd1, 0x2b, 0x80, 0x7a, 0xc4,
+ 0x10, 0x31, 0x91, 0xe0, 0xa1, 0x61, 0xd9, 0x96, 0xdd, 0xe7, 0xe6, 0x50, 0xb4, 0xaa, 0x2f, 0xd1,
+ 0x7c, 0x01, 0xfa, 0x99, 0x02, 0x67, 0xc7, 0x3d, 0x8c, 0x46, 0xd4, 0xc4, 0x0a, 0xc6, 0x27, 0x15,
+ 0x2f, 0xc6, 0x7d, 0x8d, 0x06, 0xa3, 0x10, 0xfe, 0x76, 0xc6, 0x49, 0x96, 0xaa, 0xef, 0xc0, 0xb9,
+ 0x67, 0x29, 0xce, 0xe4, 0x06, 0x6f, 0xc2, 0xca, 0x57, 0x27, 0xcf, 0xd3, 0xd5, 0x7f, 0x97, 0x87,
+ 0x1c, 0x8b, 0x1d, 0x48, 0x87, 0x92, 0xd8, 0xa3, 0x75, 0xdb, 0x08, 0xf2, 0xdf, 0xeb, 0x73, 0x44,
+ 0x21, 0xd9, 0xb8, 0x6b, 0x0c, 0xb1, 0x06, 0xc3, 0xe0, 0x19, 0x61, 0x28, 0xf3, 0xa5, 0x8e, 0x89,
+ 0x6e, 0x1a, 0x9e, 0xe1, 0x1f, 0x93, 0xbe, 0x35, 0x0f, 0x45, 0x53, 0x00, 0xed, 0x18, 0x9e, 0x71,
+ 0xeb, 0x94, 0x56, 0xea, 0x86, 0x4d, 0xe4, 0x41, 0xd5, 0xb4, 0xa8, 0x47, 0xac, 0x03, 0x91, 0xcd,
+ 0x73, 0xae, 0x19, 0x4f, 0x48, 0xc7, 0xb8, 0x76, 0x22, 0x68, 0x92, 0xb0, 0x62, 0xc6, 0xfa, 0x90,
+ 0x0e, 0xd0, 0x37, 0x46, 0x7d, 0x2c, 0xe8, 0xbe, 0x9c, 0xed, 0x7c, 0x72, 0x8c, 0xee, 0x26, 0x83,
+ 0x91, 0x3c, 0xc5, 0xbe, 0xdf, 0x50, 0xaf, 0x03, 0x84, 0x76, 0x45, 0xe7, 0xa0, 0xc8, 0xbe, 0x12,
+ 0x75, 0x8d, 0x2e, 0x96, 0xa5, 0x69, 0xd8, 0x81, 0x10, 0xe4, 0xf8, 0x37, 0xcc, 0x72, 0x01, 0x7f,
+ 0x56, 0xff, 0x9d, 0x95, 0xf6, 0xa1, 0x95, 0x02, 0x87, 0x50, 0x22, 0x0e, 0xa1, 0x7e, 0x00, 0x95,
+ 0xf8, 0x6c, 0xd9, 0x9b, 0xdc, 0xbc, 0xfe, 0x9b, 0xbc, 0xc1, 0x5c, 0x8c, 0x8e, 0x86, 0xd2, 0x9d,
+ 0xd8, 0x23, 0xeb, 0x19, 0x5a, 0x36, 0xe7, 0xcc, 0x6a, 0xec, 0x91, 0xf7, 0x18, 0x4f, 0x78, 0x4a,
+ 0xc4, 0x7a, 0x8c, 0x27, 0xea, 0x7b, 0x50, 0x0c, 0xa6, 0x97, 0x3c, 0x04, 0x74, 0x15, 0x8a, 0xc1,
+ 0x15, 0x5b, 0x8a, 0x52, 0x2f, 0x7c, 0x99, 0x65, 0xb1, 0xcc, 0xf8, 0xea, 0x11, 0x54, 0xe2, 0x19,
+ 0x4d, 0xc2, 0x8a, 0xb8, 0x37, 0x5e, 0x4e, 0x5e, 0x9b, 0x3b, 0x22, 0x44, 0xab, 0xcd, 0x5f, 0x66,
+ 0xe0, 0x85, 0x67, 0x9e, 0xae, 0x9f, 0x60, 0x22, 0xfd, 0x7c, 0x13, 0xdc, 0xf7, 0x61, 0xc9, 0x25,
+ 0xd6, 0xd0, 0x20, 0x47, 0x32, 0x4b, 0x17, 0x59, 0xc9, 0xfc, 0x65, 0x6c, 0x59, 0xc2, 0xf1, 0xec,
+ 0xbc, 0xfe, 0xcd, 0x1c, 0x9c, 0x9d, 0x7a, 0x15, 0x95, 0xf6, 0x9e, 0xe7, 0x29, 0x2c, 0x9b, 0x98,
+ 0x5a, 0x04, 0x9b, 0xe2, 0x26, 0xc2, 0x9f, 0xff, 0xde, 0x71, 0xef, 0xc2, 0x1a, 0x3b, 0x02, 0x96,
+ 0xf7, 0xc9, 0xdc, 0x61, 0xc9, 0x8c, 0xf6, 0xa9, 0xbf, 0x56, 0xa0, 0x1c, 0x7d, 0x0b, 0x6d, 0xc2,
+ 0x7a, 0xb0, 0x4b, 0x39, 0x3d, 0xb9, 0xe3, 0x98, 0x58, 0x5c, 0xd2, 0x66, 0xb4, 0x55, 0x5f, 0x78,
+ 0xaf, 0xa7, 0xf9, 0x22, 0x74, 0x11, 0xd6, 0x8c, 0xc1, 0xc0, 0x79, 0xec, 0x4f, 0x40, 0x17, 0x97,
+ 0xd3, 0x7c, 0x1a, 0x59, 0x0d, 0x49, 0x19, 0xc7, 0xef, 0x70, 0x09, 0xba, 0x0a, 0x35, 0x4c, 0x3d,
+ 0x6b, 0x68, 0xb0, 0xfa, 0x7f, 0x2c, 0xad, 0xa3, 0x72, 0x2d, 0x9e, 0x0e, 0xe4, 0xd1, 0x5c, 0x85,
+ 0xaa, 0x9f, 0x28, 0x80, 0x26, 0xa7, 0x95, 0xb0, 0x30, 0xba, 0xe3, 0x0b, 0xe3, 0xce, 0x89, 0x1a,
+ 0x33, 0xba, 0x58, 0xfe, 0x96, 0x05, 0x75, 0xfa, 0x65, 0xd0, 0xa4, 0x07, 0x2a, 0x27, 0xe9, 0x81,
+ 0xff, 0xb4, 0x3a, 0x74, 0x04, 0xcb, 0xdd, 0x87, 0x86, 0x6d, 0xe3, 0xc1, 0xb8, 0x93, 0xde, 0x3d,
+ 0xf6, 0x75, 0x59, 0xa3, 0x29, 0x70, 0x45, 0xe7, 0x52, 0x37, 0xd2, 0xa2, 0xea, 0x67, 0x0a, 0x94,
+ 0xa3, 0xf2, 0x34, 0xc7, 0x9d, 0x17, 0x61, 0x6d, 0x60, 0x50, 0x4f, 0xf7, 0xcd, 0xee, 0x1f, 0x70,
+ 0x32, 0x47, 0xc8, 0x6b, 0x88, 0xc9, 0x3a, 0x42, 0x24, 0xbd, 0x0a, 0x5d, 0x81, 0xd3, 0x3d, 0x8b,
+ 0x50, 0x4f, 0x0f, 0x4c, 0x19, 0x3d, 0x14, 0xcd, 0x6b, 0x6b, 0x5c, 0xaa, 0x49, 0xa1, 0xd4, 0xaa,
+ 0x5f, 0x87, 0xf5, 0xc4, 0x4b, 0xe1, 0xb4, 0x05, 0x70, 0x0d, 0x4e, 0x27, 0xdf, 0xe8, 0xd5, 0x3f,
+ 0x57, 0xa0, 0x10, 0xe4, 0xa5, 0xb7, 0xc4, 0x7e, 0x20, 0xfd, 0xe6, 0x4a, 0x4a, 0x7b, 0x07, 0x99,
+ 0x1d, 0xdb, 0xa3, 0x34, 0xb1, 0xa3, 0x98, 0x90, 0xe3, 0x3b, 0x56, 0xca, 0xb8, 0x14, 0x37, 0x75,
+ 0x66, 0xd2, 0xd4, 0x48, 0x8e, 0x4d, 0x9c, 0x1d, 0xf3, 0xe7, 0xfa, 0x4f, 0xb2, 0x50, 0xe6, 0x67,
+ 0x37, 0xbe, 0x39, 0xe2, 0x37, 0x78, 0x93, 0xf4, 0x99, 0x24, 0xfa, 0x5d, 0x28, 0x8a, 0xbb, 0x19,
+ 0xb6, 0xb0, 0xc5, 0xc1, 0xe0, 0x85, 0x94, 0x93, 0xe7, 0xf4, 0xb7, 0xf1, 0x91, 0x56, 0xa0, 0xf2,
+ 0x09, 0xdd, 0x86, 0x6c, 0x1f, 0x7b, 0xb3, 0xfe, 0xb0, 0xc1, 0x81, 0x6e, 0xe2, 0xc8, 0xcf, 0x05,
+ 0x0c, 0x05, 0xed, 0xc3, 0x82, 0xe1, 0xba, 0xd8, 0x36, 0xfd, 0xe4, 0xef, 0xda, 0x2c, 0x78, 0x5b,
+ 0x5c, 0x35, 0x84, 0x94, 0x58, 0xe8, 0xbf, 0x21, 0xdf, 0x1d, 0x60, 0x83, 0xf8, 0x59, 0xde, 0xd5,
+ 0x59, 0x40, 0x9b, 0x4c, 0x33, 0xc4, 0x14, 0x48, 0xd1, 0x9f, 0x11, 0x3e, 0xcf, 0xc0, 0x92, 0xfc,
+ 0x2c, 0x32, 0x32, 0xc5, 0xbf, 0x4b, 0xf2, 0xff, 0x06, 0xbb, 0x63, 0x86, 0x7b, 0x6d, 0x66, 0xc3,
+ 0x05, 0x97, 0xd4, 0xdc, 0x72, 0xf7, 0xe3, 0x96, 0x7b, 0x7d, 0x1e, 0xcb, 0x05, 0x98, 0xbe, 0xe9,
+ 0xb4, 0x98, 0xe9, 0xae, 0xcd, 0x61, 0xba, 0x00, 0x54, 0xda, 0x2e, 0x7a, 0x89, 0xfe, 0x9b, 0x02,
+ 0x14, 0x7c, 0xa7, 0x42, 0x1d, 0x58, 0x10, 0xbf, 0x64, 0xc9, 0xd4, 0xe7, 0xd5, 0x19, 0xbd, 0xb2,
+ 0xa1, 0x71, 0x6d, 0x36, 0x7c, 0x81, 0x83, 0x28, 0xac, 0x0e, 0x47, 0x03, 0xb6, 0xdf, 0xb9, 0xfa,
+ 0xc4, 0x19, 0xec, 0xd6, 0xac, 0xf0, 0x77, 0x24, 0x54, 0xf4, 0xd0, 0xb5, 0x3a, 0x8c, 0x77, 0x22,
+ 0x13, 0x96, 0x0f, 0x8c, 0xbe, 0x1e, 0x39, 0x66, 0xce, 0xce, 0xf4, 0xbf, 0x47, 0xc0, 0xb7, 0x6d,
+ 0xf4, 0xa3, 0x47, 0xca, 0xe5, 0x83, 0x48, 0x9b, 0x4d, 0xcd, 0xf2, 0x30, 0x31, 0x0e, 0x06, 0x38,
+ 0x3a, 0xb5, 0xdc, 0x7c, 0x53, 0x6b, 0x4b, 0xa8, 0xb1, 0xa9, 0x59, 0xf1, 0x4e, 0xf4, 0x0d, 0x05,
+ 0x6a, 0x81, 0x41, 0x1f, 0xe1, 0x23, 0x1a, 0xa5, 0xce, 0x73, 0xea, 0xd6, 0xbc, 0x56, 0xbd, 0x8d,
+ 0x8f, 0x68, 0x94, 0x7e, 0x7d, 0x98, 0x24, 0x50, 0x55, 0x58, 0x10, 0x9f, 0x39, 0x9a, 0x9a, 0x94,
+ 0x79, 0x6a, 0xa2, 0x12, 0xa8, 0x4e, 0x4c, 0x24, 0xcd, 0xc6, 0x56, 0x87, 0xa5, 0x70, 0x1e, 0x91,
+ 0x88, 0x1c, 0x9c, 0x80, 0xb7, 0x4d, 0x74, 0x1a, 0x16, 0xc4, 0x15, 0xbe, 0x8c, 0xc9, 0xb2, 0xa5,
+ 0x7e, 0x47, 0x81, 0xea, 0x84, 0x63, 0x3c, 0x67, 0x52, 0x7f, 0xea, 0xb9, 0x70, 0xea, 0x87, 0xb0,
+ 0x9e, 0x68, 0xc8, 0xe7, 0x3d, 0xfd, 0x8f, 0xa1, 0x1c, 0x75, 0xd3, 0x94, 0x74, 0xe1, 0xda, 0x88,
+ 0xd0, 0x05, 0x37, 0x26, 0xb3, 0x4c, 0x3c, 0xb8, 0x9b, 0x78, 0x1b, 0x56, 0x62, 0x9b, 0x0a, 0x7a,
+ 0x05, 0x50, 0xd7, 0xb1, 0x3d, 0xcb, 0x1e, 0x19, 0xe2, 0xde, 0x8e, 0x5f, 0x89, 0x08, 0x7f, 0xa9,
+ 0x46, 0x25, 0xfc, 0x2e, 0xa5, 0x7e, 0x1f, 0x2a, 0xf1, 0xe8, 0x3a, 0x23, 0x44, 0xb0, 0x6d, 0x67,
+ 0x22, 0xdb, 0xf6, 0x06, 0xa0, 0xc9, 0xdd, 0x29, 0x78, 0x53, 0x89, 0xbc, 0xb9, 0x0e, 0xab, 0x09,
+ 0xd1, 0xb8, 0xbe, 0x0a, 0xd5, 0x89, 0x9d, 0xa8, 0xbe, 0x26, 0x51, 0xc7, 0x62, 0x6c, 0xfd, 0xe7,
+ 0x39, 0x28, 0xec, 0x3a, 0xf2, 0x90, 0xe8, 0xff, 0xa0, 0x40, 0xf1, 0x21, 0x26, 0x96, 0x27, 0x16,
+ 0xc9, 0x72, 0xea, 0xf3, 0x06, 0x1f, 0xa2, 0xb1, 0x27, 0xf5, 0xc5, 0xad, 0x6f, 0x00, 0x37, 0x7f,
+ 0x11, 0x8e, 0x6a, 0xac, 0xbe, 0xa5, 0xd4, 0xe8, 0xfb, 0xa7, 0x0f, 0x7e, 0x93, 0xdf, 0x59, 0x11,
+ 0xa3, 0x8b, 0xf9, 0xc7, 0x2d, 0x6a, 0xa2, 0x91, 0x90, 0xd3, 0xe4, 0xd3, 0xa4, 0x54, 0x0b, 0x93,
+ 0x6e, 0xf7, 0x22, 0x94, 0x07, 0x4e, 0x5f, 0x1f, 0x38, 0xf2, 0xde, 0x75, 0x51, 0xbc, 0x32, 0x70,
+ 0xfa, 0xbb, 0xb2, 0x8b, 0x79, 0x9d, 0xf7, 0x90, 0x60, 0xc3, 0xac, 0x15, 0xb8, 0x50, 0xb6, 0xd4,
+ 0xff, 0x85, 0xdc, 0xae, 0x45, 0x3d, 0xd4, 0x01, 0xf6, 0xba, 0x8e, 0x6d, 0x8f, 0x58, 0xd8, 0x2f,
+ 0x38, 0x2e, 0xcc, 0x68, 0x54, 0x0d, 0x06, 0xe2, 0xc9, 0xc2, 0x54, 0x25, 0x50, 0xf0, 0x6d, 0x5c,
+ 0xef, 0x41, 0x8e, 0x99, 0x19, 0xad, 0x40, 0xe9, 0xfe, 0xdd, 0xbd, 0x4e, 0xab, 0xd9, 0xbe, 0xd1,
+ 0x6e, 0xed, 0x54, 0x4e, 0xa1, 0x22, 0xe4, 0xf7, 0xb5, 0xad, 0x66, 0xab, 0xa2, 0xb0, 0xc7, 0x9d,
+ 0xd6, 0xf6, 0xfd, 0x9b, 0x95, 0x0c, 0x2a, 0x40, 0xae, 0x7d, 0xf7, 0xc6, 0xbd, 0x4a, 0x16, 0x01,
+ 0x2c, 0xdc, 0xbd, 0xb7, 0xdf, 0x6e, 0xb6, 0x2a, 0x39, 0xd6, 0xfb, 0x60, 0x4b, 0xbb, 0x5b, 0xc9,
+ 0xb3, 0x57, 0x5b, 0x9a, 0x76, 0x4f, 0xab, 0x2c, 0xa0, 0x32, 0x14, 0x9a, 0x5a, 0x7b, 0xbf, 0xdd,
+ 0xdc, 0xda, 0xad, 0x2c, 0xd6, 0xcb, 0x00, 0xbb, 0x4e, 0xbf, 0xe9, 0xd8, 0x1e, 0x71, 0x06, 0xf5,
+ 0xdf, 0xe7, 0xb8, 0x27, 0x11, 0xef, 0x81, 0x43, 0x1e, 0x85, 0x7f, 0xb2, 0xfd, 0x0b, 0x14, 0x1f,
+ 0xf3, 0x8e, 0x70, 0x11, 0x17, 0x44, 0x47, 0xdb, 0x44, 0x07, 0x50, 0xe9, 0x0a, 0x75, 0xdd, 0xff,
+ 0x63, 0x5a, 0x7a, 0xc1, 0xdc, 0x7f, 0xf4, 0xac, 0x48, 0xc0, 0x96, 0xc4, 0x63, 0x1c, 0x03, 0xa7,
+ 0xdf, 0xb7, 0xec, 0x7e, 0xc8, 0x91, 0x3d, 0x26, 0x87, 0x04, 0x0c, 0x38, 0x4c, 0xa8, 0x1a, 0xc4,
+ 0xb3, 0x7a, 0x46, 0xd7, 0x0b, 0x49, 0x72, 0xc7, 0x23, 0xa9, 0xf8, 0x88, 0x01, 0x4b, 0x8f, 0xdf,
+ 0x88, 0x1d, 0x5a, 0x94, 0x39, 0x70, 0x40, 0x93, 0x3f, 0x1e, 0x4d, 0x35, 0x80, 0x0c, 0x78, 0xde,
+ 0x87, 0x05, 0xd7, 0x20, 0xc6, 0x90, 0xd6, 0x80, 0x3b, 0xe6, 0x0c, 0x3b, 0x71, 0xec, 0xeb, 0x37,
+ 0x3a, 0x1c, 0x47, 0xfe, 0x48, 0x26, 0x40, 0xd5, 0x6b, 0x50, 0x8a, 0x74, 0x7f, 0xd5, 0x19, 0x72,
+ 0x31, 0x5a, 0xc9, 0xff, 0x27, 0x0f, 0x6c, 0x21, 0x89, 0x0c, 0xae, 0x41, 0x5e, 0xac, 0x44, 0xf2,
+ 0xe2, 0xfa, 0x45, 0x16, 0xee, 0x1c, 0x37, 0xbd, 0x3b, 0xd6, 0x5f, 0x66, 0x1e, 0x1c, 0x6a, 0x3c,
+ 0x0b, 0x7d, 0xf3, 0x53, 0x05, 0x96, 0xb6, 0xb1, 0x31, 0xbc, 0x61, 0xcb, 0x05, 0x80, 0xbe, 0xab,
+ 0xc0, 0xa2, 0xff, 0x9c, 0x36, 0x69, 0x4e, 0xf8, 0xf9, 0x58, 0xbd, 0x36, 0x8f, 0xae, 0x08, 0xe6,
+ 0xa7, 0x36, 0x94, 0x8b, 0xca, 0xe6, 0x47, 0x00, 0x62, 0x64, 0xbc, 0x96, 0xb4, 0x65, 0x4d, 0x79,
+ 0x61, 0xc6, 0xba, 0x54, 0x9d, 0x55, 0x41, 0xb2, 0x7f, 0x4f, 0x81, 0x92, 0xa0, 0x17, 0x1b, 0xf9,
+ 0x13, 0xc8, 0x8b, 0x87, 0xcb, 0xb3, 0x24, 0x74, 0x72, 0x46, 0xea, 0x95, 0xd9, 0x94, 0xe4, 0xf6,
+ 0x25, 0x46, 0xf2, 0xfd, 0xe0, 0x13, 0xed, 0x8a, 0xf5, 0x8a, 0x9e, 0xc0, 0xa2, 0xff, 0x78, 0x65,
+ 0xd6, 0x2d, 0x8c, 0x05, 0x6e, 0xf5, 0x52, 0x7a, 0x2d, 0x3f, 0x2e, 0x8a, 0xb1, 0xfc, 0x2a, 0x03,
+ 0x35, 0x31, 0x96, 0xd6, 0x13, 0x0f, 0x13, 0xdb, 0x18, 0x08, 0x2f, 0xeb, 0x38, 0xc2, 0x73, 0x4a,
+ 0x11, 0xbf, 0x46, 0xd7, 0xe6, 0x5e, 0x70, 0xea, 0xeb, 0xf3, 0xa8, 0xfa, 0x56, 0x43, 0xdf, 0x56,
+ 0x00, 0xc2, 0x15, 0x80, 0xd2, 0xd7, 0xb7, 0xb1, 0x65, 0xa6, 0x5e, 0x9b, 0x43, 0xd3, 0x1f, 0xc5,
+ 0xf6, 0x16, 0xfc, 0xc7, 0x34, 0xed, 0xa8, 0xf2, 0x76, 0x51, 0x18, 0x74, 0xcb, 0xb5, 0xde, 0x5d,
+ 0x8e, 0x88, 0xf4, 0xc3, 0x4b, 0x07, 0x0b, 0x3c, 0x79, 0xb8, 0xfc, 0x8f, 0x00, 0x00, 0x00, 0xff,
+ 0xff, 0xdb, 0x09, 0x17, 0x4b, 0xda, 0x33, 0x00, 0x00,
}
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 dcb1ba1..20f6996 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
@@ -51,7 +51,7 @@
return proto.EnumName(BeamConstants_Constants_name, int32(x))
}
func (BeamConstants_Constants) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{0, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{0, 0}
}
type StandardPTransforms_Primitives int32
@@ -130,7 +130,7 @@
return proto.EnumName(StandardPTransforms_Primitives_name, int32(x))
}
func (StandardPTransforms_Primitives) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 0}
}
type StandardPTransforms_DeprecatedPrimitives int32
@@ -157,7 +157,7 @@
return proto.EnumName(StandardPTransforms_DeprecatedPrimitives_name, int32(x))
}
func (StandardPTransforms_DeprecatedPrimitives) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 1}
}
type StandardPTransforms_Composites int32
@@ -196,7 +196,7 @@
return proto.EnumName(StandardPTransforms_Composites_name, int32(x))
}
func (StandardPTransforms_Composites) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 2}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 2}
}
// Payload for all of these: CombinePayload
@@ -242,7 +242,7 @@
return proto.EnumName(StandardPTransforms_CombineComponents_name, int32(x))
}
func (StandardPTransforms_CombineComponents) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 3}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 3}
}
// Payload for all of these: ParDoPayload containing the user's SDF
@@ -302,7 +302,7 @@
return proto.EnumName(StandardPTransforms_SplittableParDoComponents_name, int32(x))
}
func (StandardPTransforms_SplittableParDoComponents) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 4}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 4}
}
type StandardSideInputTypes_Enum int32
@@ -310,12 +310,13 @@
const (
// Represents a view over a PCollection<V>.
//
- // The SDK is limited to perform state get requests using the StateKey.IterableSideInput.
+ // StateGetRequests performed on this side input must use
+ // StateKey.IterableSideInput.
StandardSideInputTypes_ITERABLE StandardSideInputTypes_Enum = 0
// Represents a view over a PCollection<KV<K, V>>.
//
- // The SDK is able to perform state get requests with the StateKey.IterableSideInput and
- // StateKey.MultimapSideInput
+ // StateGetRequests performed on this side input must use
+ // StateKey.IterableSideInput or StateKey.MultimapSideInput.
StandardSideInputTypes_MULTIMAP StandardSideInputTypes_Enum = 1
)
@@ -332,7 +333,7 @@
return proto.EnumName(StandardSideInputTypes_Enum_name, int32(x))
}
func (StandardSideInputTypes_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{5, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{5, 0}
}
type Parameter_Type_Enum int32
@@ -361,7 +362,7 @@
return proto.EnumName(Parameter_Type_Enum_name, int32(x))
}
func (Parameter_Type_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{8, 0, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{8, 0, 0}
}
type IsBounded_Enum int32
@@ -387,7 +388,7 @@
return proto.EnumName(IsBounded_Enum_name, int32(x))
}
func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{16, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{16, 0}
}
type StandardCoders_Enum int32
@@ -547,7 +548,7 @@
return proto.EnumName(StandardCoders_Enum_name, int32(x))
}
func (StandardCoders_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{23, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{23, 0}
}
type MergeStatus_Enum int32
@@ -584,7 +585,7 @@
return proto.EnumName(MergeStatus_Enum_name, int32(x))
}
func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{25, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{25, 0}
}
type AccumulationMode_Enum int32
@@ -616,7 +617,7 @@
return proto.EnumName(AccumulationMode_Enum_name, int32(x))
}
func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{26, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{26, 0}
}
type ClosingBehavior_Enum int32
@@ -645,7 +646,7 @@
return proto.EnumName(ClosingBehavior_Enum_name, int32(x))
}
func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{27, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{27, 0}
}
type OnTimeBehavior_Enum int32
@@ -674,7 +675,7 @@
return proto.EnumName(OnTimeBehavior_Enum_name, int32(x))
}
func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{28, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{28, 0}
}
type OutputTime_Enum int32
@@ -708,7 +709,7 @@
return proto.EnumName(OutputTime_Enum_name, int32(x))
}
func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{29, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{29, 0}
}
type TimeDomain_Enum int32
@@ -745,7 +746,7 @@
return proto.EnumName(TimeDomain_Enum_name, int32(x))
}
func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{30, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{30, 0}
}
type StandardEnvironments_Environments int32
@@ -771,7 +772,7 @@
return proto.EnumName(StandardEnvironments_Environments_name, int32(x))
}
func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{35, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{35, 0}
}
type DisplayData_Type_Enum int32
@@ -812,7 +813,7 @@
return proto.EnumName(DisplayData_Type_Enum_name, int32(x))
}
func (DisplayData_Type_Enum) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 2, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41, 2, 0}
}
type BeamConstants struct {
@@ -825,7 +826,7 @@
func (m *BeamConstants) String() string { return proto.CompactTextString(m) }
func (*BeamConstants) ProtoMessage() {}
func (*BeamConstants) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{0}
}
func (m *BeamConstants) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BeamConstants.Unmarshal(m, b)
@@ -867,7 +868,7 @@
func (m *Components) String() string { return proto.CompactTextString(m) }
func (*Components) ProtoMessage() {}
func (*Components) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{1}
}
func (m *Components) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Components.Unmarshal(m, b)
@@ -951,7 +952,7 @@
func (m *Pipeline) String() string { return proto.CompactTextString(m) }
func (*Pipeline) ProtoMessage() {}
func (*Pipeline) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{2}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{2}
}
func (m *Pipeline) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Pipeline.Unmarshal(m, b)
@@ -1065,7 +1066,7 @@
func (m *PTransform) String() string { return proto.CompactTextString(m) }
func (*PTransform) ProtoMessage() {}
func (*PTransform) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{3}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{3}
}
func (m *PTransform) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PTransform.Unmarshal(m, b)
@@ -1137,7 +1138,7 @@
func (m *StandardPTransforms) String() string { return proto.CompactTextString(m) }
func (*StandardPTransforms) ProtoMessage() {}
func (*StandardPTransforms) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4}
}
func (m *StandardPTransforms) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StandardPTransforms.Unmarshal(m, b)
@@ -1167,7 +1168,7 @@
func (m *StandardSideInputTypes) String() string { return proto.CompactTextString(m) }
func (*StandardSideInputTypes) ProtoMessage() {}
func (*StandardSideInputTypes) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{5}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{5}
}
func (m *StandardSideInputTypes) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StandardSideInputTypes.Unmarshal(m, b)
@@ -1217,7 +1218,7 @@
func (m *PCollection) String() string { return proto.CompactTextString(m) }
func (*PCollection) ProtoMessage() {}
func (*PCollection) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{6}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{6}
}
func (m *PCollection) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_PCollection.Unmarshal(m, b)
@@ -1302,7 +1303,7 @@
func (m *ParDoPayload) String() string { return proto.CompactTextString(m) }
func (*ParDoPayload) ProtoMessage() {}
func (*ParDoPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{7}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{7}
}
func (m *ParDoPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ParDoPayload.Unmarshal(m, b)
@@ -1403,7 +1404,7 @@
func (m *Parameter) String() string { return proto.CompactTextString(m) }
func (*Parameter) ProtoMessage() {}
func (*Parameter) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{8}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{8}
}
func (m *Parameter) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Parameter.Unmarshal(m, b)
@@ -1440,7 +1441,7 @@
func (m *Parameter_Type) String() string { return proto.CompactTextString(m) }
func (*Parameter_Type) ProtoMessage() {}
func (*Parameter_Type) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{8, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{8, 0}
}
func (m *Parameter_Type) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Parameter_Type.Unmarshal(m, b)
@@ -1477,7 +1478,7 @@
func (m *StateSpec) String() string { return proto.CompactTextString(m) }
func (*StateSpec) ProtoMessage() {}
func (*StateSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{9}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{9}
}
func (m *StateSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StateSpec.Unmarshal(m, b)
@@ -1707,7 +1708,7 @@
func (m *ReadModifyWriteStateSpec) String() string { return proto.CompactTextString(m) }
func (*ReadModifyWriteStateSpec) ProtoMessage() {}
func (*ReadModifyWriteStateSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{10}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{10}
}
func (m *ReadModifyWriteStateSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReadModifyWriteStateSpec.Unmarshal(m, b)
@@ -1745,7 +1746,7 @@
func (m *BagStateSpec) String() string { return proto.CompactTextString(m) }
func (*BagStateSpec) ProtoMessage() {}
func (*BagStateSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{11}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{11}
}
func (m *BagStateSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_BagStateSpec.Unmarshal(m, b)
@@ -1784,7 +1785,7 @@
func (m *CombiningStateSpec) String() string { return proto.CompactTextString(m) }
func (*CombiningStateSpec) ProtoMessage() {}
func (*CombiningStateSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{12}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{12}
}
func (m *CombiningStateSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CombiningStateSpec.Unmarshal(m, b)
@@ -1830,7 +1831,7 @@
func (m *MapStateSpec) String() string { return proto.CompactTextString(m) }
func (*MapStateSpec) ProtoMessage() {}
func (*MapStateSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{13}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{13}
}
func (m *MapStateSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_MapStateSpec.Unmarshal(m, b)
@@ -1875,7 +1876,7 @@
func (m *SetStateSpec) String() string { return proto.CompactTextString(m) }
func (*SetStateSpec) ProtoMessage() {}
func (*SetStateSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{14}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{14}
}
func (m *SetStateSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SetStateSpec.Unmarshal(m, b)
@@ -1914,7 +1915,7 @@
func (m *TimerSpec) String() string { return proto.CompactTextString(m) }
func (*TimerSpec) ProtoMessage() {}
func (*TimerSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{15}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{15}
}
func (m *TimerSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TimerSpec.Unmarshal(m, b)
@@ -1958,7 +1959,7 @@
func (m *IsBounded) String() string { return proto.CompactTextString(m) }
func (*IsBounded) ProtoMessage() {}
func (*IsBounded) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{16}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{16}
}
func (m *IsBounded) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_IsBounded.Unmarshal(m, b)
@@ -1993,7 +1994,7 @@
func (m *ReadPayload) String() string { return proto.CompactTextString(m) }
func (*ReadPayload) ProtoMessage() {}
func (*ReadPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{17}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{17}
}
func (m *ReadPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReadPayload.Unmarshal(m, b)
@@ -2040,7 +2041,7 @@
func (m *WindowIntoPayload) String() string { return proto.CompactTextString(m) }
func (*WindowIntoPayload) ProtoMessage() {}
func (*WindowIntoPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{18}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{18}
}
func (m *WindowIntoPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WindowIntoPayload.Unmarshal(m, b)
@@ -2082,7 +2083,7 @@
func (m *CombinePayload) String() string { return proto.CompactTextString(m) }
func (*CombinePayload) ProtoMessage() {}
func (*CombinePayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{19}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{19}
}
func (m *CombinePayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_CombinePayload.Unmarshal(m, b)
@@ -2130,7 +2131,7 @@
func (m *TestStreamPayload) String() string { return proto.CompactTextString(m) }
func (*TestStreamPayload) ProtoMessage() {}
func (*TestStreamPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20}
}
func (m *TestStreamPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TestStreamPayload.Unmarshal(m, b)
@@ -2179,7 +2180,7 @@
func (m *TestStreamPayload_Event) String() string { return proto.CompactTextString(m) }
func (*TestStreamPayload_Event) ProtoMessage() {}
func (*TestStreamPayload_Event) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20, 0}
}
func (m *TestStreamPayload_Event) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TestStreamPayload_Event.Unmarshal(m, b)
@@ -2351,7 +2352,7 @@
func (m *TestStreamPayload_Event_AdvanceWatermark) String() string { return proto.CompactTextString(m) }
func (*TestStreamPayload_Event_AdvanceWatermark) ProtoMessage() {}
func (*TestStreamPayload_Event_AdvanceWatermark) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20, 0, 0}
}
func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Unmarshal(m, b)
@@ -2393,7 +2394,7 @@
}
func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20, 0, 1}
}
func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Unmarshal(m, b)
@@ -2431,7 +2432,7 @@
func (m *TestStreamPayload_Event_AddElements) String() string { return proto.CompactTextString(m) }
func (*TestStreamPayload_Event_AddElements) ProtoMessage() {}
func (*TestStreamPayload_Event_AddElements) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0, 2}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20, 0, 2}
}
func (m *TestStreamPayload_Event_AddElements) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TestStreamPayload_Event_AddElements.Unmarshal(m, b)
@@ -2470,7 +2471,7 @@
func (m *TestStreamPayload_TimestampedElement) String() string { return proto.CompactTextString(m) }
func (*TestStreamPayload_TimestampedElement) ProtoMessage() {}
func (*TestStreamPayload_TimestampedElement) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20, 1}
}
func (m *TestStreamPayload_TimestampedElement) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TestStreamPayload_TimestampedElement.Unmarshal(m, b)
@@ -2522,7 +2523,7 @@
func (m *WriteFilesPayload) String() string { return proto.CompactTextString(m) }
func (*WriteFilesPayload) ProtoMessage() {}
func (*WriteFilesPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{21}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{21}
}
func (m *WriteFilesPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WriteFilesPayload.Unmarshal(m, b)
@@ -2599,7 +2600,7 @@
func (m *Coder) String() string { return proto.CompactTextString(m) }
func (*Coder) ProtoMessage() {}
func (*Coder) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{22}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{22}
}
func (m *Coder) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Coder.Unmarshal(m, b)
@@ -2643,7 +2644,7 @@
func (m *StandardCoders) String() string { return proto.CompactTextString(m) }
func (*StandardCoders) ProtoMessage() {}
func (*StandardCoders) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{23}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{23}
}
func (m *StandardCoders) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StandardCoders.Unmarshal(m, b)
@@ -2714,7 +2715,7 @@
func (m *WindowingStrategy) String() string { return proto.CompactTextString(m) }
func (*WindowingStrategy) ProtoMessage() {}
func (*WindowingStrategy) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{24}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{24}
}
func (m *WindowingStrategy) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_WindowingStrategy.Unmarshal(m, b)
@@ -2817,7 +2818,7 @@
func (m *MergeStatus) String() string { return proto.CompactTextString(m) }
func (*MergeStatus) ProtoMessage() {}
func (*MergeStatus) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{25}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{25}
}
func (m *MergeStatus) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_MergeStatus.Unmarshal(m, b)
@@ -2850,7 +2851,7 @@
func (m *AccumulationMode) String() string { return proto.CompactTextString(m) }
func (*AccumulationMode) ProtoMessage() {}
func (*AccumulationMode) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{26}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{26}
}
func (m *AccumulationMode) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_AccumulationMode.Unmarshal(m, b)
@@ -2882,7 +2883,7 @@
func (m *ClosingBehavior) String() string { return proto.CompactTextString(m) }
func (*ClosingBehavior) ProtoMessage() {}
func (*ClosingBehavior) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{27}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{27}
}
func (m *ClosingBehavior) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ClosingBehavior.Unmarshal(m, b)
@@ -2914,7 +2915,7 @@
func (m *OnTimeBehavior) String() string { return proto.CompactTextString(m) }
func (*OnTimeBehavior) ProtoMessage() {}
func (*OnTimeBehavior) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{28}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{28}
}
func (m *OnTimeBehavior) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_OnTimeBehavior.Unmarshal(m, b)
@@ -2946,7 +2947,7 @@
func (m *OutputTime) String() string { return proto.CompactTextString(m) }
func (*OutputTime) ProtoMessage() {}
func (*OutputTime) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{29}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{29}
}
func (m *OutputTime) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_OutputTime.Unmarshal(m, b)
@@ -2977,7 +2978,7 @@
func (m *TimeDomain) String() string { return proto.CompactTextString(m) }
func (*TimeDomain) ProtoMessage() {}
func (*TimeDomain) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{30}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{30}
}
func (m *TimeDomain) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TimeDomain.Unmarshal(m, b)
@@ -3027,7 +3028,7 @@
func (m *Trigger) String() string { return proto.CompactTextString(m) }
func (*Trigger) ProtoMessage() {}
func (*Trigger) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31}
}
func (m *Trigger) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger.Unmarshal(m, b)
@@ -3468,7 +3469,7 @@
func (m *Trigger_AfterAll) String() string { return proto.CompactTextString(m) }
func (*Trigger_AfterAll) ProtoMessage() {}
func (*Trigger_AfterAll) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 0}
}
func (m *Trigger_AfterAll) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_AfterAll.Unmarshal(m, b)
@@ -3507,7 +3508,7 @@
func (m *Trigger_AfterAny) String() string { return proto.CompactTextString(m) }
func (*Trigger_AfterAny) ProtoMessage() {}
func (*Trigger_AfterAny) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 1}
}
func (m *Trigger_AfterAny) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_AfterAny.Unmarshal(m, b)
@@ -3547,7 +3548,7 @@
func (m *Trigger_AfterEach) String() string { return proto.CompactTextString(m) }
func (*Trigger_AfterEach) ProtoMessage() {}
func (*Trigger_AfterEach) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 2}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 2}
}
func (m *Trigger_AfterEach) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_AfterEach.Unmarshal(m, b)
@@ -3593,7 +3594,7 @@
func (m *Trigger_AfterEndOfWindow) String() string { return proto.CompactTextString(m) }
func (*Trigger_AfterEndOfWindow) ProtoMessage() {}
func (*Trigger_AfterEndOfWindow) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 3}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 3}
}
func (m *Trigger_AfterEndOfWindow) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_AfterEndOfWindow.Unmarshal(m, b)
@@ -3641,7 +3642,7 @@
func (m *Trigger_AfterProcessingTime) String() string { return proto.CompactTextString(m) }
func (*Trigger_AfterProcessingTime) ProtoMessage() {}
func (*Trigger_AfterProcessingTime) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 4}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 4}
}
func (m *Trigger_AfterProcessingTime) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_AfterProcessingTime.Unmarshal(m, b)
@@ -3682,7 +3683,7 @@
func (m *Trigger_AfterSynchronizedProcessingTime) String() string { return proto.CompactTextString(m) }
func (*Trigger_AfterSynchronizedProcessingTime) ProtoMessage() {}
func (*Trigger_AfterSynchronizedProcessingTime) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 5}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 5}
}
func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Unmarshal(m, b)
@@ -3714,7 +3715,7 @@
func (m *Trigger_Default) String() string { return proto.CompactTextString(m) }
func (*Trigger_Default) ProtoMessage() {}
func (*Trigger_Default) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 6}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 6}
}
func (m *Trigger_Default) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_Default.Unmarshal(m, b)
@@ -3746,7 +3747,7 @@
func (m *Trigger_ElementCount) String() string { return proto.CompactTextString(m) }
func (*Trigger_ElementCount) ProtoMessage() {}
func (*Trigger_ElementCount) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 7}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 7}
}
func (m *Trigger_ElementCount) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_ElementCount.Unmarshal(m, b)
@@ -3785,7 +3786,7 @@
func (m *Trigger_Never) String() string { return proto.CompactTextString(m) }
func (*Trigger_Never) ProtoMessage() {}
func (*Trigger_Never) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 8}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 8}
}
func (m *Trigger_Never) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_Never.Unmarshal(m, b)
@@ -3817,7 +3818,7 @@
func (m *Trigger_Always) String() string { return proto.CompactTextString(m) }
func (*Trigger_Always) ProtoMessage() {}
func (*Trigger_Always) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 9}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 9}
}
func (m *Trigger_Always) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_Always.Unmarshal(m, b)
@@ -3853,7 +3854,7 @@
func (m *Trigger_OrFinally) String() string { return proto.CompactTextString(m) }
func (*Trigger_OrFinally) ProtoMessage() {}
func (*Trigger_OrFinally) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 10}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 10}
}
func (m *Trigger_OrFinally) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_OrFinally.Unmarshal(m, b)
@@ -3901,7 +3902,7 @@
func (m *Trigger_Repeat) String() string { return proto.CompactTextString(m) }
func (*Trigger_Repeat) ProtoMessage() {}
func (*Trigger_Repeat) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 11}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31, 11}
}
func (m *Trigger_Repeat) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Trigger_Repeat.Unmarshal(m, b)
@@ -3946,7 +3947,7 @@
func (m *TimestampTransform) String() string { return proto.CompactTextString(m) }
func (*TimestampTransform) ProtoMessage() {}
func (*TimestampTransform) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{32}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{32}
}
func (m *TimestampTransform) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TimestampTransform.Unmarshal(m, b)
@@ -4087,7 +4088,7 @@
func (m *TimestampTransform_Delay) String() string { return proto.CompactTextString(m) }
func (*TimestampTransform_Delay) ProtoMessage() {}
func (*TimestampTransform_Delay) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{32, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{32, 0}
}
func (m *TimestampTransform_Delay) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TimestampTransform_Delay.Unmarshal(m, b)
@@ -4130,7 +4131,7 @@
func (m *TimestampTransform_AlignTo) String() string { return proto.CompactTextString(m) }
func (*TimestampTransform_AlignTo) ProtoMessage() {}
func (*TimestampTransform_AlignTo) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{32, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{32, 1}
}
func (m *TimestampTransform_AlignTo) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_TimestampTransform_AlignTo.Unmarshal(m, b)
@@ -4198,7 +4199,7 @@
func (m *SideInput) String() string { return proto.CompactTextString(m) }
func (*SideInput) ProtoMessage() {}
func (*SideInput) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{33}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{33}
}
func (m *SideInput) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SideInput.Unmarshal(m, b)
@@ -4256,7 +4257,7 @@
func (m *Environment) String() string { return proto.CompactTextString(m) }
func (*Environment) ProtoMessage() {}
func (*Environment) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{34}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{34}
}
func (m *Environment) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_Environment.Unmarshal(m, b)
@@ -4300,7 +4301,7 @@
func (m *StandardEnvironments) String() string { return proto.CompactTextString(m) }
func (*StandardEnvironments) ProtoMessage() {}
func (*StandardEnvironments) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{35}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{35}
}
func (m *StandardEnvironments) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_StandardEnvironments.Unmarshal(m, b)
@@ -4332,7 +4333,7 @@
func (m *DockerPayload) String() string { return proto.CompactTextString(m) }
func (*DockerPayload) ProtoMessage() {}
func (*DockerPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{36}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{36}
}
func (m *DockerPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DockerPayload.Unmarshal(m, b)
@@ -4373,7 +4374,7 @@
func (m *ProcessPayload) String() string { return proto.CompactTextString(m) }
func (*ProcessPayload) ProtoMessage() {}
func (*ProcessPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{37}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{37}
}
func (m *ProcessPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ProcessPayload.Unmarshal(m, b)
@@ -4433,7 +4434,7 @@
func (m *ExternalPayload) String() string { return proto.CompactTextString(m) }
func (*ExternalPayload) ProtoMessage() {}
func (*ExternalPayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{38}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{38}
}
func (m *ExternalPayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExternalPayload.Unmarshal(m, b)
@@ -4484,7 +4485,7 @@
func (m *SdkFunctionSpec) String() string { return proto.CompactTextString(m) }
func (*SdkFunctionSpec) ProtoMessage() {}
func (*SdkFunctionSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{39}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{39}
}
func (m *SdkFunctionSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_SdkFunctionSpec.Unmarshal(m, b)
@@ -4563,7 +4564,7 @@
func (m *FunctionSpec) String() string { return proto.CompactTextString(m) }
func (*FunctionSpec) ProtoMessage() {}
func (*FunctionSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{40}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{40}
}
func (m *FunctionSpec) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_FunctionSpec.Unmarshal(m, b)
@@ -4610,7 +4611,7 @@
func (m *DisplayData) String() string { return proto.CompactTextString(m) }
func (*DisplayData) ProtoMessage() {}
func (*DisplayData) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41}
}
func (m *DisplayData) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DisplayData.Unmarshal(m, b)
@@ -4654,7 +4655,7 @@
func (m *DisplayData_Identifier) String() string { return proto.CompactTextString(m) }
func (*DisplayData_Identifier) ProtoMessage() {}
func (*DisplayData_Identifier) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41, 0}
}
func (m *DisplayData_Identifier) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DisplayData_Identifier.Unmarshal(m, b)
@@ -4718,7 +4719,7 @@
func (m *DisplayData_Item) String() string { return proto.CompactTextString(m) }
func (*DisplayData_Item) ProtoMessage() {}
func (*DisplayData_Item) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41, 1}
}
func (m *DisplayData_Item) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DisplayData_Item.Unmarshal(m, b)
@@ -4790,7 +4791,7 @@
func (m *DisplayData_Type) String() string { return proto.CompactTextString(m) }
func (*DisplayData_Type) ProtoMessage() {}
func (*DisplayData_Type) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 2}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41, 2}
}
func (m *DisplayData_Type) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_DisplayData_Type.Unmarshal(m, b)
@@ -4844,7 +4845,7 @@
func (m *MessageWithComponents) String() string { return proto.CompactTextString(m) }
func (*MessageWithComponents) ProtoMessage() {}
func (*MessageWithComponents) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{42}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{42}
}
func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_MessageWithComponents.Unmarshal(m, b)
@@ -5288,7 +5289,7 @@
func (m *ExecutableStagePayload) String() string { return proto.CompactTextString(m) }
func (*ExecutableStagePayload) ProtoMessage() {}
func (*ExecutableStagePayload) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{43}
}
func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecutableStagePayload.Unmarshal(m, b)
@@ -5380,7 +5381,7 @@
func (m *ExecutableStagePayload_SideInputId) String() string { return proto.CompactTextString(m) }
func (*ExecutableStagePayload_SideInputId) ProtoMessage() {}
func (*ExecutableStagePayload_SideInputId) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43, 0}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{43, 0}
}
func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecutableStagePayload_SideInputId.Unmarshal(m, b)
@@ -5430,7 +5431,7 @@
func (m *ExecutableStagePayload_UserStateId) String() string { return proto.CompactTextString(m) }
func (*ExecutableStagePayload_UserStateId) ProtoMessage() {}
func (*ExecutableStagePayload_UserStateId) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43, 1}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{43, 1}
}
func (m *ExecutableStagePayload_UserStateId) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecutableStagePayload_UserStateId.Unmarshal(m, b)
@@ -5480,7 +5481,7 @@
func (m *ExecutableStagePayload_TimerId) String() string { return proto.CompactTextString(m) }
func (*ExecutableStagePayload_TimerId) ProtoMessage() {}
func (*ExecutableStagePayload_TimerId) Descriptor() ([]byte, []int) {
- return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43, 2}
+ return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{43, 2}
}
func (m *ExecutableStagePayload_TimerId) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ExecutableStagePayload_TimerId.Unmarshal(m, b)
@@ -5639,10 +5640,10 @@
}
func init() {
- proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_b87c0d18be5b2d09)
+ proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_d0b62631d7ed3b4f)
}
-var fileDescriptor_beam_runner_api_b87c0d18be5b2d09 = []byte{
+var fileDescriptor_beam_runner_api_d0b62631d7ed3b4f = []byte{
// 5131 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xdf, 0x6f, 0x1b, 0x57,
0x76, 0xbf, 0xf8, 0x53, 0xe4, 0x21, 0x45, 0x8d, 0xae, 0x64, 0x47, 0x9e, 0xcd, 0xc6, 0xf6, 0xc4,
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 9ceb37e..3bec152 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -37,6 +37,7 @@
import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
import org.apache.beam.sdk.transforms.DoFn.OnTimerContext;
import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
+import org.apache.beam.sdk.transforms.Materializations.MultimapView;
import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
@@ -520,7 +521,18 @@
Materializations.MULTIMAP_MATERIALIZATION_URN,
view.getViewFn().getMaterialization().getUrn());
return ((ViewFn<Materializations.MultimapView, T>) view.getViewFn())
- .apply(o -> Collections.emptyList());
+ .apply(
+ new MultimapView() {
+ @Override
+ public Iterable get() {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public Iterable get(@Nullable Object o) {
+ return Collections.emptyList();
+ }
+ });
}
@Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java
index 9334ddd..b035883 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java
@@ -17,6 +17,8 @@
*/
package org.apache.beam.sdk.transforms;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.StandardSideInputTypes;
@@ -37,22 +39,48 @@
* specified windowed values.
*/
@Experimental(Kind.CORE_RUNNERS_ONLY)
- public static final String ITERABLE_MATERIALIZATION_URN =
- StandardSideInputTypes.Enum.ITERABLE
- .getValueDescriptor()
- .getOptions()
- .getExtension(RunnerApi.beamUrn);
+ public static final String ITERABLE_MATERIALIZATION_URN = "beam:side_input:iterable:v1";
+
+ static {
+ // Use a constant above so that they can be used in switch case statements and validate
+ // that the constant hasn't changed from the proto definition on class load.
+ checkState(
+ ITERABLE_MATERIALIZATION_URN.equals(
+ StandardSideInputTypes.Enum.ITERABLE
+ .getValueDescriptor()
+ .getOptions()
+ .getExtension(RunnerApi.beamUrn)),
+ "Expected %s to match proto constant but found %s.",
+ ITERABLE_MATERIALIZATION_URN,
+ StandardSideInputTypes.Enum.ITERABLE
+ .getValueDescriptor()
+ .getOptions()
+ .getExtension(RunnerApi.beamUrn));
+ }
/**
* The URN for a {@link Materialization} where the primitive view type is a multimap of fully
* specified windowed values.
*/
@Experimental(Kind.CORE_RUNNERS_ONLY)
- public static final String MULTIMAP_MATERIALIZATION_URN =
- StandardSideInputTypes.Enum.MULTIMAP
- .getValueDescriptor()
- .getOptions()
- .getExtension(RunnerApi.beamUrn);
+ public static final String MULTIMAP_MATERIALIZATION_URN = "beam:side_input:multimap:v1";
+
+ static {
+ // Use a constant above so that they can be used in switch case statements and validate
+ // that the constant hasn't changed from the proto definition on class load.
+ checkState(
+ MULTIMAP_MATERIALIZATION_URN.equals(
+ StandardSideInputTypes.Enum.MULTIMAP
+ .getValueDescriptor()
+ .getOptions()
+ .getExtension(RunnerApi.beamUrn)),
+ "Expected %s to match proto constant but found %s.",
+ MULTIMAP_MATERIALIZATION_URN,
+ StandardSideInputTypes.Enum.MULTIMAP
+ .getValueDescriptor()
+ .getOptions()
+ .getExtension(RunnerApi.beamUrn));
+ }
/**
* Represents the {@code PrimitiveViewT} supplied to the {@link ViewFn} when it declares to use
@@ -61,6 +89,7 @@
* @param <V>
*/
public interface IterableView<V> {
+ /** Returns an iterable for all values. */
Iterable<V> get();
}
@@ -69,6 +98,10 @@
* the {@link Materializations#MULTIMAP_MATERIALIZATION_URN multimap materialization}.
*/
public interface MultimapView<K, V> {
+ /** Returns an iterable of all keys. */
+ Iterable<K> get();
+
+ /** Returns an iterable of all the values for the specified key. */
Iterable<V> get(@Nullable K k);
}
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java
index fff60e6..996b87e 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java
@@ -57,6 +57,24 @@
}
@Override
+ public Iterable<K> get() {
+ StateRequest.Builder requestBuilder = StateRequest.newBuilder();
+ requestBuilder
+ .setInstructionId(instructionId)
+ .getStateKeyBuilder()
+ .getMultimapKeysSideInputBuilder()
+ .setTransformId(ptransformId)
+ .setSideInputId(sideInputId)
+ .setWindow(encodedWindow);
+
+ return new LazyCachingIteratorToIterable<>(
+ new DataStreams.DataStreamDecoder(
+ keyCoder,
+ DataStreams.inbound(
+ StateFetchingIterators.forFirstChunk(beamFnStateClient, requestBuilder.build()))));
+ }
+
+ @Override
public Iterable<V> get(K k) {
ByteString.Output output = ByteString.newOutput();
try {
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 77dd858..da5266f 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
@@ -1016,7 +1016,7 @@
* Max batch byte size. Messages are base64 encoded which encodes each set of three bytes into
* four bytes.
*/
- private static final int MAX_PUBLISH_BATCH_BYTE_SIZE_DEFAULT = ((10 * 1024 * 1024) / 4) * 3;
+ private static final int MAX_PUBLISH_BATCH_BYTE_SIZE_DEFAULT = ((10 * 1000 * 1000) / 4) * 3;
private static final int MAX_PUBLISH_BATCH_SIZE = 100;
diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py
index ec8db53..15c3fe2 100644
--- a/sdks/python/apache_beam/internal/pickler.py
+++ b/sdks/python/apache_beam/internal/pickler.py
@@ -33,12 +33,16 @@
import base64
import logging
import sys
+import threading
import traceback
import types
import zlib
import dill
+# Pickling, especially unpickling, can cause broken module imports
+# if executed concurrently, see: BEAM-8651.
+pickle_lock = threading.Lock()
# Dill 0.28.0 renamed dill.dill to dill._dill:
# https://github.com/uqfoundation/dill/commit/f0972ecc7a41d0b8acada6042d557068cac69baa
# TODO: Remove this once Beam depends on dill >= 0.2.8
@@ -228,17 +232,17 @@
# encoding. This should be cleaned up.
def dumps(o, enable_trace=True):
"""For internal use only; no backwards-compatibility guarantees."""
-
- try:
- s = dill.dumps(o)
- except Exception: # pylint: disable=broad-except
- if enable_trace:
- dill.dill._trace(True) # pylint: disable=protected-access
+ with pickle_lock:
+ try:
s = dill.dumps(o)
- else:
- raise
- finally:
- dill.dill._trace(False) # pylint: disable=protected-access
+ except Exception: # pylint: disable=broad-except
+ if enable_trace:
+ dill.dill._trace(True) # pylint: disable=protected-access
+ s = dill.dumps(o)
+ else:
+ raise
+ finally:
+ dill.dill._trace(False) # pylint: disable=protected-access
# Compress as compactly as possible to decrease peak memory usage (of multiple
# in-memory copies) and free up some possibly large and no-longer-needed
@@ -257,16 +261,17 @@
s = zlib.decompress(c)
del c # Free up some possibly large and no-longer-needed memory.
- try:
- return dill.loads(s)
- except Exception: # pylint: disable=broad-except
- if enable_trace:
- dill.dill._trace(True) # pylint: disable=protected-access
+ with pickle_lock:
+ try:
return dill.loads(s)
- else:
- raise
- finally:
- dill.dill._trace(False) # pylint: disable=protected-access
+ except Exception: # pylint: disable=broad-except
+ if enable_trace:
+ dill.dill._trace(True) # pylint: disable=protected-access
+ return dill.loads(s)
+ else:
+ raise
+ finally:
+ dill.dill._trace(False) # pylint: disable=protected-access
def dump_session(file_path):
@@ -278,10 +283,12 @@
create and load the dump twice to have consistent results in the worker and
the running session. Check: https://github.com/uqfoundation/dill/issues/195
"""
- dill.dump_session(file_path)
- dill.load_session(file_path)
- return dill.dump_session(file_path)
+ with pickle_lock:
+ dill.dump_session(file_path)
+ dill.load_session(file_path)
+ return dill.dump_session(file_path)
def load_session(file_path):
- return dill.load_session(file_path)
+ with pickle_lock:
+ return dill.load_session(file_path)
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 e74be08..ea9d02b 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
@@ -285,11 +285,11 @@
# Here's where we would use a different type of partitioning
# (e.g. also by key) for a different access pattern.
if access_pattern.urn == common_urns.side_inputs.ITERABLE.urn:
- self._kv_extrator = lambda value: ('', value)
+ self._kv_extractor = lambda value: ('', value)
self._key_coder = coders.SingletonCoder('')
self._value_coder = coder.wrapped_value_coder
elif access_pattern.urn == common_urns.side_inputs.MULTIMAP.urn:
- self._kv_extrator = lambda value: value
+ self._kv_extractor = lambda value: value
self._key_coder = coder.wrapped_value_coder.key_coder()
self._value_coder = (
coder.wrapped_value_coder.value_coder())
@@ -305,7 +305,7 @@
while input_stream.size() > 0:
windowed_value = self._windowed_value_coder.get_impl(
).decode_from_stream(input_stream, True)
- key, value = self._kv_extrator(windowed_value.value)
+ key, value = self._kv_extractor(windowed_value.value)
for window in windowed_value.windows:
self._values_by_window[key, window].append(value)
@@ -501,14 +501,27 @@
elements_by_window = _WindowGroupingBuffer(si, value_coder)
for element_data in pcoll_buffers[buffer_id]:
elements_by_window.append(element_data)
- for key, window, elements_data in elements_by_window.encoded_items():
- state_key = beam_fn_api_pb2.StateKey(
- multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput(
- transform_id=transform_id,
- side_input_id=tag,
- window=window,
- key=key))
- worker_handler.state.append_raw(state_key, elements_data)
+
+ if si.urn == common_urns.side_inputs.ITERABLE.urn:
+ for _, window, elements_data in elements_by_window.encoded_items():
+ state_key = beam_fn_api_pb2.StateKey(
+ iterable_side_input=beam_fn_api_pb2.StateKey.IterableSideInput(
+ transform_id=transform_id,
+ side_input_id=tag,
+ window=window))
+ worker_handler.state.append_raw(state_key, elements_data)
+ elif si.urn == common_urns.side_inputs.MULTIMAP.urn:
+ for key, window, elements_data in elements_by_window.encoded_items():
+ state_key = beam_fn_api_pb2.StateKey(
+ multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput(
+ transform_id=transform_id,
+ side_input_id=tag,
+ window=window,
+ key=key))
+ worker_handler.state.append_raw(state_key, elements_data)
+ else:
+ raise ValueError(
+ "Unknown access pattern: '%s'" % si.urn)
def _run_bundle_multiple_times_for_testing(
self, worker_handler_list, process_bundle_descriptor, data_input,
@@ -1377,6 +1390,9 @@
pass
def host_from_worker(self):
+ # TODO(BEAM-8646): Reconcile the behavior on Windows platform.
+ if sys.platform == 'win32':
+ return 'localhost'
import socket
return socket.getfqdn()
diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py
index 7bee5fe..c62f194 100644
--- a/sdks/python/apache_beam/runners/worker/bundle_processor.py
+++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py
@@ -238,20 +238,25 @@
def __getitem__(self, window):
target_window = self._side_input_data.window_mapping_fn(window)
if target_window not in self._cache:
- state_key = beam_fn_api_pb2.StateKey(
- multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput(
- transform_id=self._transform_id,
- side_input_id=self._tag,
- window=self._target_window_coder.encode(target_window),
- key=b''))
state_handler = self._state_handler
access_pattern = self._side_input_data.access_pattern
if access_pattern == common_urns.side_inputs.ITERABLE.urn:
+ state_key = beam_fn_api_pb2.StateKey(
+ iterable_side_input=beam_fn_api_pb2.StateKey.IterableSideInput(
+ transform_id=self._transform_id,
+ side_input_id=self._tag,
+ window=self._target_window_coder.encode(target_window)))
raw_view = _StateBackedIterable(
state_handler, state_key, self._element_coder)
elif access_pattern == common_urns.side_inputs.MULTIMAP.urn:
+ state_key = beam_fn_api_pb2.StateKey(
+ multimap_side_input=beam_fn_api_pb2.StateKey.MultimapSideInput(
+ transform_id=self._transform_id,
+ side_input_id=self._tag,
+ window=self._target_window_coder.encode(target_window),
+ key=b''))
cache = {}
key_coder_impl = self._element_coder.key_coder().get_impl()
value_coder = self._element_coder.value_coder()
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index c5b3318..dabab00 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -48,14 +48,17 @@
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
+from apache_beam.transforms import WindowInto
from apache_beam.transforms import window
from apache_beam.transforms.core import _GroupByKeyOnly
from apache_beam.transforms.display import DisplayData
from apache_beam.transforms.display import DisplayDataItem
from apache_beam.transforms.ptransform import PTransform
+from apache_beam.transforms.window import TimestampedValue
from apache_beam.typehints import with_input_types
from apache_beam.typehints import with_output_types
from apache_beam.typehints.typehints_test import TypeHintTestCase
+from apache_beam.utils.timestamp import Timestamp
from apache_beam.utils.windowed_value import WindowedValue
# Disable frequent lint warning due to pipe operator for chaining transforms.
@@ -310,6 +313,7 @@
expected_error_prefix = 'FlatMap and ParDo must return an iterable.'
self.assertStartswith(cm.exception.args[0], expected_error_prefix)
+ @attr('ValidatesRunner')
def test_do_fn_with_finish(self):
class MyDoFn(beam.DoFn):
def process(self, element):
@@ -332,6 +336,33 @@
assert_that(result, matcher())
pipeline.run()
+ @attr('ValidatesRunner')
+ def test_do_fn_with_windowing_in_finish_bundle(self):
+ windowfn = window.FixedWindows(2)
+
+ class MyDoFn(beam.DoFn):
+ def process(self, element):
+ yield TimestampedValue('process'+ str(element), 5)
+
+ def finish_bundle(self):
+ yield WindowedValue('finish', 1, [windowfn])
+
+ pipeline = TestPipeline()
+ result = (pipeline
+ | 'Start' >> beam.Create([x for x in range(3)])
+ | 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_finish = [('finish', Timestamp(1), Timestamp(0), Timestamp(2))]
+
+ assert_that(result, equal_to(expected_process + expected_finish))
+ pipeline.run()
+
+ @attr('ValidatesRunner')
def test_do_fn_with_start(self):
class MyDoFn(beam.DoFn):
def __init__(self):
diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py
index aab45d9..d3c073b 100644
--- a/sdks/python/apache_beam/typehints/decorators.py
+++ b/sdks/python/apache_beam/typehints/decorators.py
@@ -340,6 +340,22 @@
return 'IOTypeHints[inputs=%s, outputs=%s]' % (
self.input_types, self.output_types)
+ def __eq__(self, other):
+ def same(a, b):
+ if a is None or not any(a):
+ return b is None or not any(b)
+ else:
+ return a == b
+ return (
+ same(self.input_types, other.input_types)
+ and same(self.output_types, other.output_types))
+
+ def __ne__(self, other):
+ return not self == other
+
+ def __hash__(self):
+ return hash(str(self))
+
class WithTypeHints(object):
"""A mixin class that provides the ability to set and retrieve type hints.
@@ -351,8 +367,9 @@
def _get_or_create_type_hints(self):
# __init__ may have not been called
try:
- return self._type_hints
- except AttributeError:
+ # Only return an instance bound to self (see BEAM-8629).
+ return self.__dict__['_type_hints']
+ except KeyError:
self._type_hints = IOTypeHints()
return self._type_hints
diff --git a/sdks/python/apache_beam/typehints/decorators_test.py b/sdks/python/apache_beam/typehints/decorators_test.py
index 645fae6..21f62f3 100644
--- a/sdks/python/apache_beam/typehints/decorators_test.py
+++ b/sdks/python/apache_beam/typehints/decorators_test.py
@@ -125,6 +125,22 @@
self.assertEqual(th.input_types, ((int, ), {}))
self.assertEqual(th.output_types, ((str, ), {}))
+ def test_inherits_does_not_modify(self):
+ # See BEAM-8629.
+ @decorators.with_output_types(int)
+ class Subclass(WithTypeHints):
+ def __init__(self):
+ pass # intentionally avoiding super call
+ # These should be equal, but not the same object lest mutating the instance
+ # mutates the class.
+ self.assertFalse(
+ Subclass()._get_or_create_type_hints() is Subclass._type_hints)
+ self.assertEqual(
+ Subclass().get_type_hints(), Subclass._type_hints)
+ self.assertNotEqual(
+ Subclass().with_input_types(str)._type_hints,
+ Subclass._type_hints)
+
if __name__ == '__main__':
unittest.main()
diff --git a/website/src/contribute/dependencies.md b/website/src/contribute/dependencies.md
index c8c4ccc..2ba75d8 100644
--- a/website/src/contribute/dependencies.md
+++ b/website/src/contribute/dependencies.md
@@ -36,9 +36,9 @@
This picture can become even more complicated during runtime. Runner specific code might be incompatible with dependencies included by certain modules and if these dependencies leak into runtime, a pipeline might end up in a broken state.
-The overall issue is not common to Beam and well known in the industry as the Diamond [Dependency problem \(or Dependency Hell\)](https://en.wikipedia.org/wiki/Dependency_hell).
+The overall issue is not unique to Beam and is well known in the industry as the [Diamond Dependency problem \(or Dependency Hell\)](https://en.wikipedia.org/wiki/Dependency_hell).
-One common solution for the diamond dependency problem is [semantic versioning](https://semver.org/). The basic idea is that dependencies will be versioned in the form _x.y.z_ where _x_ is the _major version_, _y_ is the _minor version_, and _z_ is the _patch version_. A major version change may be backwards incompatible and is expected to be rare. Minor and patch versions may be released more regularly but are expected to be backwards compatible. But in practice, important fixes (such as security patches) might get released in the form of minor or patch version updates and it will be healthy for the Beam project to depend on recently released minor versions of dependencies.
+One common solution for the diamond dependency problem is [semantic versioning](https://semver.org/). The basic idea is that dependencies are versioned in the form _x.y.z_ where _x_ is the _major version_, _y_ is the _minor version_, and _z_ is the _patch version_. A major version change may be backwards incompatible and is expected to be rare. Minor and patch versions may be released more regularly but are expected to be backwards compatible. But in practice, important fixes (such as security patches) might get released in the form of minor or patch version updates and it will be healthy for the Beam project to depend on recently released minor versions of dependencies.
## Identifying outdated dependencies
@@ -88,8 +88,8 @@
__Dependencies of Java SDK components that may cause issues to other components if leaked should be vendored.__
-[Vendoring](https://www.ardanlabs.com/blog/2013/10/manage-dependencies-with-godep.html) is the process of creating copies of third party dependencies. Combined with repackaging, vendoring will allow Beam components to depend on third party libraries without causing conflicts to other components. Vendoring should be done in a case-by-case basis since this can increase the total number of dependencies deployed in user's enviroment.
+[Vendoring](https://www.ardanlabs.com/blog/2013/10/manage-dependencies-with-godep.html) is the process of creating copies of third party dependencies. Combined with repackaging, vendoring allows Beam components to depend on third party libraries without causing conflicts to other components. Vendoring should be done in a case-by-case basis since this can increase the total number of dependencies deployed in user's enviroment.
## Dependency updates and backwards compatibility
-Beam releases [adhere to]({{ site.baseurl }}/get-started/downloads/) semantic versioning. Hence, community members should take care when updating dependencies. Minor version updates to dependencies should be backwards compatible in most cases. Some updates to dependencies though may result in backwards incompatible API or functionality changes to Beam. PR reviewers and committers should take care to detect any dependency updates that could potentially introduce backwards incompatible changes to Beam before merging and PRs that update dependencies should include a statement regarding this verification in the form of a PR comment. Dependency updates that result in backwards incompatible changes to non-experimental features of Beam should be held till next major version release of Beam. Any exceptions to this policy should only occur in extreme cases (for example, due to a security vulnerability of an existing dependency that is only fixed in a subsequent major version) and should be discussed in the Beam dev list. Note that backwards incompatible changes to experimental features may be introduced in a minor version release.
+Beam releases [adhere to]({{ site.baseurl }}/get-started/downloads/) semantic versioning. Hence, community members should take care when updating dependencies. Minor version updates to dependencies should be backwards compatible in most cases. Some updates to dependencies though may result in backwards incompatible API or functionality changes to Beam. PR reviewers and committers should take care to detect any dependency updates that could potentially introduce backwards incompatible changes to Beam before merging and PRs that update dependencies should include a statement regarding this verification in the form of a PR comment. Dependency updates that result in backwards incompatible changes to non-experimental features of Beam should be held till the next major version release of Beam. Any exceptions to this policy should only occur in extreme cases (for example, due to a security vulnerability of an existing dependency that is only fixed in a subsequent major version) and should be discussed on the Beam dev list. Note that backwards incompatible changes to experimental features may be introduced in a minor version release.