[
https://issues.apache.org/jira/browse/BEAM-3708?focusedWorklogId=116054&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-116054
]
ASF GitHub Bot logged work on BEAM-3708:
----------------------------------------
Author: ASF GitHub Bot
Created on: 26/Jun/18 17:31
Start Date: 26/Jun/18 17:31
Worklog Time Spent: 10m
Work Description: lukecwik closed pull request #5622: [BEAM-3708] Adding
Combine component implementations to Java SDK
URL: https://github.com/apache/beam/pull/5622
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
new file mode 100644
index 00000000000..849ebf41d56
--- /dev/null
+++
b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.fn.harness;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
+import org.apache.beam.model.pipeline.v1.RunnerApi.StandardPTransforms;
+import org.apache.beam.runners.core.construction.BeamUrns;
+import org.apache.beam.sdk.fn.function.ThrowingFunction;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * Executes different components of Combine PTransforms.
+ */
+public class CombineRunners<InputT, OutputT> {
+
+ /**
+ * A registrar which provides a factory to handle combine component
PTransforms.
+ */
+ @AutoService(PTransformRunnerFactory.Registrar.class)
+ public static class Registrar implements PTransformRunnerFactory.Registrar {
+
+ @Override
+ public Map<String, PTransformRunnerFactory> getPTransformRunnerFactories()
{
+ return ImmutableMap.of(
+
BeamUrns.getUrn(StandardPTransforms.CombineComponents.COMBINE_PER_KEY_PRECOMBINE),
+
MapFnRunners.forValueMapFnFactory(CombineRunners::createPrecombineMapFunction),
+
BeamUrns.getUrn(StandardPTransforms.CombineComponents.COMBINE_PER_KEY_MERGE_ACCUMULATORS),
+
MapFnRunners.forValueMapFnFactory(CombineRunners::createMergeAccumulatorsMapFunction),
+
BeamUrns.getUrn(StandardPTransforms.CombineComponents.COMBINE_PER_KEY_EXTRACT_OUTPUTS),
+
MapFnRunners.forValueMapFnFactory(CombineRunners::createExtractOutputsMapFunction),
+
BeamUrns.getUrn(StandardPTransforms.Composites.COMBINE_GROUPED_VALUES),
+
MapFnRunners.forValueMapFnFactory(CombineRunners::createCombineGroupedValuesMapFunction));
+ }
+ }
+
+ static <KeyT, InputT, AccumT>
+ ThrowingFunction<KV<KeyT, InputT>, KV<KeyT, AccumT>>
createPrecombineMapFunction(
+ String pTransformId, PTransform pTransform) throws IOException {
+ CombinePayload combinePayload =
CombinePayload.parseFrom(pTransform.getSpec().getPayload());
+ CombineFn<InputT, AccumT, ?> combineFn =
+ (CombineFn)
+ SerializableUtils.deserializeFromByteArray(
+
combinePayload.getCombineFn().getSpec().getPayload().toByteArray(),
"CombineFn");
+
+ return (KV<KeyT, InputT> input) ->
+ KV.of(input.getKey(),
combineFn.addInput(combineFn.createAccumulator(), input.getValue()));
+ }
+
+ static <KeyT, AccumT>
+ ThrowingFunction<KV<KeyT, Iterable<AccumT>>, KV<KeyT, AccumT>>
+ createMergeAccumulatorsMapFunction(String pTransformId, PTransform
pTransform)
+ throws IOException {
+ CombinePayload combinePayload =
CombinePayload.parseFrom(pTransform.getSpec().getPayload());
+ CombineFn<?, AccumT, ?> combineFn =
+ (CombineFn)
+ SerializableUtils.deserializeFromByteArray(
+
combinePayload.getCombineFn().getSpec().getPayload().toByteArray(),
"CombineFn");
+
+ return (KV<KeyT, Iterable<AccumT>> input) ->
+ KV.of(input.getKey(), combineFn.mergeAccumulators(input.getValue()));
+ }
+
+ static <KeyT, AccumT, OutputT>
+ ThrowingFunction<KV<KeyT, AccumT>, KV<KeyT, OutputT>>
createExtractOutputsMapFunction(
+ String pTransformId, PTransform pTransform) throws IOException {
+ CombinePayload combinePayload =
CombinePayload.parseFrom(pTransform.getSpec().getPayload());
+ CombineFn<?, AccumT, OutputT> combineFn =
+ (CombineFn)
+ SerializableUtils.deserializeFromByteArray(
+
combinePayload.getCombineFn().getSpec().getPayload().toByteArray(),
"CombineFn");
+
+ return (KV<KeyT, AccumT> input) ->
+ KV.of(input.getKey(), combineFn.extractOutput(input.getValue()));
+ }
+
+ static <KeyT, InputT, AccumT, OutputT>
+ ThrowingFunction<KV<KeyT, Iterable<InputT>>, KV<KeyT, OutputT>>
+ createCombineGroupedValuesMapFunction(String pTransformId, PTransform
pTransform)
+ throws IOException {
+ CombinePayload combinePayload =
CombinePayload.parseFrom(pTransform.getSpec().getPayload());
+ CombineFn<InputT, AccumT, OutputT> combineFn =
+ (CombineFn)
+ SerializableUtils.deserializeFromByteArray(
+
combinePayload.getCombineFn().getSpec().getPayload().toByteArray(),
"CombineFn");
+
+ return (KV<KeyT, Iterable<InputT>> input) -> {
+ AccumT accumulator = combineFn.createAccumulator();
+ Iterable<InputT> inputValues = input.getValue();
+ for (InputT inputValue : inputValues) {
+ accumulator = combineFn.addInput(accumulator, inputValue);
+ }
+ return KV.of(input.getKey(), combineFn.extractOutput(accumulator));
+ };
+ }
+}
diff --git
a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
new file mode 100644
index 00000000000..3e15aad8a2f
--- /dev/null
+++
b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.fn.harness;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.core.IsEqual.equalTo;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.PipelineTranslation;
+import org.apache.beam.runners.core.construction.SdkComponents;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.fn.data.FnDataReceiver;
+import org.apache.beam.sdk.fn.function.ThrowingRunnable;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link CombineRunners}.
+ */
+@RunWith(JUnit4.class)
+public class CombineRunnersTest {
+ // CombineFn that converts strings to ints and sums them up to an
accumulator, and negates the
+ // value of the accumulator when extracting outputs. These operations are
chosen to avoid
+ // autoboxing and provide a way to easily check that certain combine steps
actually executed.
+ private static class TestCombineFn extends CombineFn<String, Integer,
Integer> {
+
+ @Override
+ public Integer createAccumulator() {
+ return 0;
+ }
+
+ @Override
+ public Integer addInput(Integer accum, String input) {
+ accum += Integer.parseInt(input);
+ return accum;
+ }
+
+ @Override
+ public Integer mergeAccumulators(Iterable<Integer> accums) {
+ Integer merged = 0;
+ for (Integer accum : accums) {
+ merged += accum;
+ }
+ return merged;
+ }
+
+ @Override
+ public Integer extractOutput(Integer accum) {
+ return -accum;
+ }
+ }
+
+ private static final String TEST_COMBINE_ID = "combineId";
+
+ private RunnerApi.PTransform pTransform;
+ private String inputPCollectionId;
+ private String outputPCollectionId;
+
+ @Before
+ public void createPipeline() throws Exception {
+ // Create pipeline with an input pCollection, combine, and output
pCollection.
+ TestCombineFn combineFn = new TestCombineFn();
+ Combine.PerKey<String, String, Integer> combine =
Combine.perKey(combineFn);
+
+ Pipeline p = Pipeline.create();
+ PCollection<KV<String, String>> inputPCollection =
p.apply(Create.of(KV.of("unused", "0")));
+ inputPCollection.setCoder(KvCoder.of(StringUtf8Coder.of(),
StringUtf8Coder.of()));
+ PCollection<KV<String, Integer>> outputPCollection = inputPCollection
+ .apply(TEST_COMBINE_ID, combine);
+ outputPCollection.setCoder(KvCoder.of(StringUtf8Coder.of(),
BigEndianIntegerCoder.of()));
+
+ // Create FnApi protos needed for the runner.
+ SdkComponents sdkComponents = SdkComponents.create();
+ RunnerApi.Pipeline pProto = PipelineTranslation.toProto(p, sdkComponents);
+ inputPCollectionId = sdkComponents.registerPCollection(inputPCollection);
+ outputPCollectionId = sdkComponents.registerPCollection(outputPCollection);
+ pTransform = pProto.getComponents().getTransformsOrThrow(TEST_COMBINE_ID);
+ }
+
+ /**
+ * Create a Precombine that is given keyed elements and validates that the
outputted elements
+ * values' are accumulators that were correctly derived from the input.
+ */
+ @Test
+ public void testPrecombine() throws Exception {
+ // Create a map of consumers and an output target to check output values.
+ Multimap<String, FnDataReceiver<WindowedValue<?>>> consumers =
HashMultimap.create();
+ Deque<WindowedValue<KV<String, Integer>>> mainOutputValues = new
ArrayDeque<>();
+
consumers.put(Iterables.getOnlyElement(pTransform.getOutputsMap().values()),
+ (FnDataReceiver) (FnDataReceiver<WindowedValue<KV<String, Integer>>>)
+ mainOutputValues::add);
+
+ List<ThrowingRunnable> startFunctions = new ArrayList<>();
+ List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+ // Create runner.
+ MapFnRunners
+ .forValueMapFnFactory(CombineRunners::createPrecombineMapFunction)
+ .createRunnerForPTransform(
+ PipelineOptionsFactory.create(),
+ null,
+ null,
+ TEST_COMBINE_ID,
+ pTransform,
+ null,
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ consumers,
+ startFunctions::add,
+ finishFunctions::add,
+ null);
+
+ assertThat(startFunctions, empty());
+ assertThat(finishFunctions, empty());
+
+ // Send elements to runner and check outputs.
+ mainOutputValues.clear();
+ assertThat(consumers.keySet(), containsInAnyOrder(inputPCollectionId,
outputPCollectionId));
+
+ FnDataReceiver<WindowedValue<?>> input =
+ Iterables.getOnlyElement(consumers.get(inputPCollectionId));
+ input.accept(valueInGlobalWindow(KV.of("A", "1")));
+ input.accept(valueInGlobalWindow(KV.of("A", "2")));
+ input.accept(valueInGlobalWindow(KV.of("A", "6")));
+ input.accept(valueInGlobalWindow(KV.of("B", "2")));
+ input.accept(valueInGlobalWindow(KV.of("C", "3")));
+
+ // Check that all values for "A" were converted to accumulators regardless
of how they were
+ // combined by the Precombine optimization.
+ Integer sum = 0;
+ while ("A".equals(mainOutputValues.getFirst().getValue().getKey())) {
+ sum += mainOutputValues.getFirst().getValue().getValue();
+ mainOutputValues.removeFirst();
+ }
+ assertThat(sum, equalTo(9));
+
+ assertThat(
+ mainOutputValues,
+ contains(
+ valueInGlobalWindow(KV.of("B", 2)),
+ valueInGlobalWindow(KV.of("C", 3))));
+ }
+
+ /**
+ * Create a Merge Accumulators function that is given keyed lists of
accumulators and validates
+ * that the accumulators of each list were merged.
+ */
+ @Test
+ public void testMergeAccumulators() throws Exception {
+ // Create a map of consumers and an output target to check output values.
+ Multimap<String, FnDataReceiver<WindowedValue<?>>> consumers =
HashMultimap.create();
+ Deque<WindowedValue<KV<String, Integer>>> mainOutputValues = new
ArrayDeque<>();
+
consumers.put(Iterables.getOnlyElement(pTransform.getOutputsMap().values()),
+ (FnDataReceiver) (FnDataReceiver<WindowedValue<KV<String, Integer>>>)
+ mainOutputValues::add);
+
+ List<ThrowingRunnable> startFunctions = new ArrayList<>();
+ List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+ // Create runner.
+ MapFnRunners
+
.forValueMapFnFactory(CombineRunners::createMergeAccumulatorsMapFunction)
+ .createRunnerForPTransform(
+ PipelineOptionsFactory.create(),
+ null,
+ null,
+ TEST_COMBINE_ID,
+ pTransform,
+ null,
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ consumers,
+ startFunctions::add,
+ finishFunctions::add,
+ null);
+
+ assertThat(startFunctions, empty());
+ assertThat(finishFunctions, empty());
+
+ // Send elements to runner and check outputs.
+ mainOutputValues.clear();
+ assertThat(consumers.keySet(), containsInAnyOrder(inputPCollectionId,
outputPCollectionId));
+
+ FnDataReceiver<WindowedValue<?>> input =
+ Iterables.getOnlyElement(consumers.get(inputPCollectionId));
+ input.accept(valueInGlobalWindow(KV.of("A", Arrays.asList(1, 2, 6))));
+ input.accept(valueInGlobalWindow(KV.of("B", Arrays.asList(2, 3))));
+ input.accept(valueInGlobalWindow(KV.of("C", Arrays.asList(5, 2))));
+
+ assertThat(
+ mainOutputValues,
+ contains(
+ valueInGlobalWindow(KV.of("A", 9)),
+ valueInGlobalWindow(KV.of("B", 5)),
+ valueInGlobalWindow(KV.of("C", 7))));
+ }
+
+ /**
+ * Create an Extract Outputs function that is given keyed accumulators and
validates that the
+ * accumulators were turned into the output type.
+ */
+ @Test
+ public void testExtractOutputs() throws Exception {
+ // Create a map of consumers and an output target to check output values.
+ Multimap<String, FnDataReceiver<WindowedValue<?>>> consumers =
HashMultimap.create();
+ Deque<WindowedValue<KV<String, Integer>>> mainOutputValues = new
ArrayDeque<>();
+
consumers.put(Iterables.getOnlyElement(pTransform.getOutputsMap().values()),
+ (FnDataReceiver) (FnDataReceiver<WindowedValue<KV<String, Integer>>>)
+ mainOutputValues::add);
+
+ List<ThrowingRunnable> startFunctions = new ArrayList<>();
+ List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+ // Create runner.
+ MapFnRunners
+ .forValueMapFnFactory(CombineRunners::createExtractOutputsMapFunction)
+ .createRunnerForPTransform(
+ PipelineOptionsFactory.create(),
+ null,
+ null,
+ TEST_COMBINE_ID,
+ pTransform,
+ null,
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ consumers,
+ startFunctions::add,
+ finishFunctions::add,
+ null);
+
+ assertThat(startFunctions, empty());
+ assertThat(finishFunctions, empty());
+
+ // Send elements to runner and check outputs.
+ mainOutputValues.clear();
+ assertThat(consumers.keySet(), containsInAnyOrder(inputPCollectionId,
outputPCollectionId));
+
+ FnDataReceiver<WindowedValue<?>> input =
+ Iterables.getOnlyElement(consumers.get(inputPCollectionId));
+ input.accept(valueInGlobalWindow(KV.of("A", 9)));
+ input.accept(valueInGlobalWindow(KV.of("B", 5)));
+ input.accept(valueInGlobalWindow(KV.of("C", 7)));
+
+ assertThat(
+ mainOutputValues,
+ contains(
+ valueInGlobalWindow(KV.of("A", -9)),
+ valueInGlobalWindow(KV.of("B", -5)),
+ valueInGlobalWindow(KV.of("C", -7))));
+ }
+
+ /**
+ * Create a Combine Grouped Values function that is given lists of values
that are grouped by key
+ * and validates that the lists are properly combined.
+ */
+ @Test
+ public void testCombineGroupedValues() throws Exception {
+ // Create a map of consumers and an output target to check output values.
+ Multimap<String, FnDataReceiver<WindowedValue<?>>> consumers =
HashMultimap.create();
+ Deque<WindowedValue<KV<String, Integer>>> mainOutputValues = new
ArrayDeque<>();
+
consumers.put(Iterables.getOnlyElement(pTransform.getOutputsMap().values()),
+ (FnDataReceiver) (FnDataReceiver<WindowedValue<KV<String, Integer>>>)
+ mainOutputValues::add);
+
+ List<ThrowingRunnable> startFunctions = new ArrayList<>();
+ List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+ // Create runner.
+ MapFnRunners
+
.forValueMapFnFactory(CombineRunners::createCombineGroupedValuesMapFunction)
+ .createRunnerForPTransform(
+ PipelineOptionsFactory.create(),
+ null,
+ null,
+ TEST_COMBINE_ID,
+ pTransform,
+ null,
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ Collections.emptyMap(),
+ consumers,
+ startFunctions::add,
+ finishFunctions::add,
+ null);
+
+ assertThat(startFunctions, empty());
+ assertThat(finishFunctions, empty());
+
+ // Send elements to runner and check outputs.
+ mainOutputValues.clear();
+ assertThat(consumers.keySet(), containsInAnyOrder(inputPCollectionId,
outputPCollectionId));
+
+ FnDataReceiver<WindowedValue<?>> input =
+ Iterables.getOnlyElement(consumers.get(inputPCollectionId));
+ input.accept(valueInGlobalWindow(KV.of("A", Arrays.asList("1", "2",
"6"))));
+ input.accept(valueInGlobalWindow(KV.of("B", Arrays.asList("2", "3"))));
+ input.accept(valueInGlobalWindow(KV.of("C", Arrays.asList("5", "2"))));
+
+ assertThat(
+ mainOutputValues,
+ contains(
+ valueInGlobalWindow(KV.of("A", -9)),
+ valueInGlobalWindow(KV.of("B", -5)),
+ valueInGlobalWindow(KV.of("C", -7))));
+ }
+}
diff --git
a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
index 3b58517c5cb..3521790f65e 100644
---
a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
+++
b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
@@ -21,7 +21,6 @@
import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.junit.Assert.assertThat;
@@ -33,8 +32,8 @@
import com.google.common.collect.Multimap;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.Iterator;
import java.util.List;
+import org.apache.beam.fn.harness.data.MultiplexingFnDataReceiver;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.sdk.fn.data.FnDataReceiver;
@@ -110,9 +109,8 @@ public void testCreatingAndProcessingDoFlatten() throws
Exception {
}
/**
- * Create a Flatten that has 4 inputs (inputATarget1, inputATarget2,
inputBTarget, inputCTarget)
- * and one output (mainOutput). Validate that inputs are flattened together
and directed to the
- * output.
+ * Create a Flatten that consumes data from the same PCollection duplicated
through two outputs
+ * and validates that inputs are flattened together and directed to the
output.
*/
@Test
public void testFlattenWithDuplicateInputCollectionProducesMultipleOutputs()
throws Exception {
@@ -157,16 +155,15 @@ public void
testFlattenWithDuplicateInputCollectionProducesMultipleOutputs() thr
assertThat(consumers.keySet(), containsInAnyOrder("inputATarget",
"mainOutputTarget"));
assertThat(consumers.get("inputATarget"), hasSize(2));
- Iterator<FnDataReceiver<WindowedValue<?>>> targets =
consumers.get("inputATarget").iterator();
- FnDataReceiver<WindowedValue<?>> first = targets.next();
- FnDataReceiver<WindowedValue<?>> second = targets.next();
+ FnDataReceiver<WindowedValue<?>> input =
+ MultiplexingFnDataReceiver.forConsumers(consumers.get("inputATarget"));
// Both of these are the flatten consumer
- assertThat(first, equalTo(second));
+ //assertThat(first, equalTo(second));
- first.accept(WindowedValue.valueInGlobalWindow("A1"));
- second.accept(WindowedValue.valueInGlobalWindow("A1"));
- first.accept(WindowedValue.valueInGlobalWindow("A2"));
- second.accept(WindowedValue.valueInGlobalWindow("A2"));
+ input.accept(WindowedValue.valueInGlobalWindow("A1"));
+ //second.accept(WindowedValue.valueInGlobalWindow("A1"));
+ input.accept(WindowedValue.valueInGlobalWindow("A2"));
+ //second.accept(WindowedValue.valueInGlobalWindow("A2"));
assertThat(
mainOutputValues,
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 116054)
Time Spent: 2h 40m (was: 2.5h)
> Implement the portable lifted Combiner transforms in Java SDK
> -------------------------------------------------------------
>
> Key: BEAM-3708
> URL: https://issues.apache.org/jira/browse/BEAM-3708
> Project: Beam
> Issue Type: Sub-task
> Components: sdk-java-core, sdk-java-harness
> Reporter: Daniel Oliveira
> Assignee: Daniel Oliveira
> Priority: Major
> Labels: portability
> Time Spent: 2h 40m
> Remaining Estimate: 0h
>
> Lifted combines are split into separate parts with different URNs. These
> parts need to be implemented in the Java SDK harness so that the SDK can
> actually execute them when receiving Combine transforms with the
> corresponding URNs.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)