[
https://issues.apache.org/jira/browse/BEAM-3708?focusedWorklogId=113974&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-113974
]
ASF GitHub Bot logged work on BEAM-3708:
----------------------------------------
Author: ASF GitHub Bot
Created on: 20/Jun/18 23:24
Start Date: 20/Jun/18 23:24
Worklog Time Spent: 10m
Work Description: lukecwik commented on a change in pull request #5622:
[BEAM-3708] Adding Combine component implementations to Java SDK
URL: https://github.com/apache/beam/pull/5622#discussion_r196970538
##########
File path:
sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.number.IsCloseTo.closeTo;
+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 {
+ private static class TestCombineFn extends CombineFn<Integer, Double,
String> {
+
+ @Override
+ public Double createAccumulator() {
+ return 0.0;
+ }
+
+ @Override
+ public Double addInput(Double accum, Integer input) {
+ accum += input;
+ return accum;
+ }
+
+ @Override
+ public Double mergeAccumulators(Iterable<Double> accums) {
+ Double merged = 0.0;
+ for (Double accum : accums) {
+ merged += accum;
+ }
+ return merged;
+ }
+
+ @Override
+ public String extractOutput(Double accum) {
+ return accum.toString();
+ }
+ }
+
+ 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, Integer, String> combine =
Combine.perKey(combineFn);
+
+ Pipeline p = Pipeline.create();
+ PCollection<KV<String, Integer>> inputPCollection =
p.apply(Create.of(KV.of("unused", 0)));
+ inputPCollection.setCoder(KvCoder.of(StringUtf8Coder.of(),
BigEndianIntegerCoder.of()));
+ PCollection<KV<String, String>> outputPCollection = inputPCollection
+ .apply(TEST_COMBINE_ID, combine);
+ outputPCollection.setCoder(KvCoder.of(StringUtf8Coder.of(),
StringUtf8Coder.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, Double>>> mainOutputValues = new
ArrayDeque<>();
+
consumers.put(Iterables.getOnlyElement(pTransform.getOutputsMap().values()),
+ (FnDataReceiver) (FnDataReceiver<WindowedValue<KV<String, Double>>>)
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);
+
+ 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.
+ Double sum = 0.0;
+ while ("A".equals(mainOutputValues.getFirst().getValue().getKey())) {
+ sum += mainOutputValues.getFirst().getValue().getValue();
Review comment:
because of autoboxing, the precombine may or may not have actually exercised
the create accumulator step and instead just returned the value.
Consider using a combine where the input is a string that gets converted to
an integer and then summed.
Will get rid of any autoboxing concerns and will get rid of the `closeTo`
check.
----------------------------------------------------------------
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: 113974)
Time Spent: 1h 10m (was: 1h)
> 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: 1h 10m
> 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)