[
https://issues.apache.org/jira/browse/BEAM-3708?focusedWorklogId=117669&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-117669
]
ASF GitHub Bot logged work on BEAM-3708:
----------------------------------------
Author: ASF GitHub Bot
Created on: 29/Jun/18 23:41
Start Date: 29/Jun/18 23:41
Worklog Time Spent: 10m
Work Description: youngoli commented on a change in pull request #5795:
[BEAM-3708] Adding grouping table to Precombine step.
URL: https://github.com/apache/beam/pull/5795#discussion_r199302720
##########
File path:
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
##########
@@ -55,6 +83,127 @@
}
}
+ private static class PrecombineRunner<KeyT, InputT, AccumT> {
+ private PipelineOptions options;
+ private CombineFn<InputT, AccumT, ?> combineFn;
+ private FnDataReceiver<WindowedValue<KV<KeyT, AccumT>>> output;
+ private Coder<KeyT> keyCoder;
+ private GroupingTable<WindowedValue<KeyT>, InputT, AccumT> groupingTable;
+ private Coder<AccumT> accumCoder;
+
+ PrecombineRunner(
+ PipelineOptions options,
+ CombineFn<InputT, AccumT, ?> combineFn,
+ FnDataReceiver<WindowedValue<KV<KeyT, AccumT>>> output,
+ Coder<KeyT> keyCoder,
+ Coder<AccumT> accumCoder) {
+ this.options = options;
+ this.combineFn = combineFn;
+ this.output = output;
+ this.keyCoder = keyCoder;
+ this.accumCoder = accumCoder;
+ }
+
+ void startBundle() {
+ GroupingTables.Combiner<WindowedValue<KeyT>, InputT, AccumT, ?>
valueCombiner =
+ new ValueCombiner<>(
+ GlobalCombineFnRunners.create(combineFn),
NullSideInputReader.empty(), options);
+
+ groupingTable =
+ GroupingTables.combiningAndSampling(
+ new WindowingCoderGroupingKeyCreator<>(keyCoder),
+ PairInfo.create(),
+ valueCombiner,
+ new
CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)),
+ new CoderSizeEstimator<>(accumCoder),
+ 0.001 /*sizeEstimatorSampleRate*/);
+ }
+
+ void processElement(WindowedValue<KV<KeyT, InputT>> elem) throws Exception
{
+ groupingTable.put(
+ elem,
+ (Object outputElem) ->
+ output.accept((WindowedValue<KV<KeyT, AccumT>>) outputElem)
+ );
+ }
+
+ void finishBundle() throws Exception {
+ groupingTable.flush(
+ (Object outputElem) ->
+ output.accept((WindowedValue<KV<KeyT, AccumT>>) outputElem)
+ );
+ }
+ }
+
+ /** A factory for {@link PrecombineRunner}s. */
+ private static class PrecombineFactory<KeyT, InputT, AccumT>
+ implements PTransformRunnerFactory<PrecombineRunner<KeyT, InputT,
AccumT>> {
+
+ @Override
+ public PrecombineRunner<KeyT, InputT, AccumT> createRunnerForPTransform(
+ PipelineOptions pipelineOptions,
+ BeamFnDataClient beamFnDataClient,
+ BeamFnStateClient beamFnStateClient,
+ String pTransformId,
+ PTransform pTransform,
+ Supplier<String> processBundleInstructionId,
+ Map<String, PCollection> pCollections,
+ Map<String, RunnerApi.Coder> coders,
+ Map<String, RunnerApi.WindowingStrategy> windowingStrategies,
+ Multimap<String, FnDataReceiver<WindowedValue<?>>>
pCollectionIdsToConsumers,
+ Consumer<ThrowingRunnable> addStartFunction,
+ Consumer<ThrowingRunnable> addFinishFunction,
+ BundleSplitListener splitListener)
+ throws IOException {
+ // Get objects needed to create the runner.
+ RehydratedComponents rehydratedComponents =
+ RehydratedComponents.forComponents(
+ RunnerApi.Components.newBuilder()
+ .putAllCoders(coders)
+ .putAllWindowingStrategies(windowingStrategies)
+ .build());
+ String mainInputTag =
Iterables.getOnlyElement(pTransform.getInputsMap().keySet());
+ RunnerApi.PCollection mainInput =
pCollections.get(pTransform.getInputsOrThrow(mainInputTag));
+ WindowedValueCoder<KV<KeyT, InputT>> inputCoder =
+ (WindowedValueCoder<KV<KeyT, InputT>>)
+ rehydratedComponents.getCoder(mainInput.getCoderId());
+ Coder<KeyT> keyCoder = ((KvCoder<KeyT, InputT>)
inputCoder.getValueCoder()).getKeyCoder();
+
+ CombinePayload combinePayload =
CombinePayload.parseFrom(pTransform.getSpec().getPayload());
+ CombineFn<InputT, AccumT, ?> combineFn =
+ (CombineFn)
+ SerializableUtils.deserializeFromByteArray(
+
combinePayload.getCombineFn().getSpec().getPayload().toByteArray(),
"CombineFn");
+ Coder<AccumT> accumCoder =
+ (Coder<AccumT>)
rehydratedComponents.getCoder(combinePayload.getAccumulatorCoderId());
+
+ Collection<FnDataReceiver<WindowedValue<KV<KeyT, InputT>>>> consumers =
+ (Collection)
+ pCollectionIdsToConsumers.get(
+
Iterables.getOnlyElement(pTransform.getOutputsMap().values()));
+
+ // Create the runner.
+ PrecombineRunner<KeyT, InputT, AccumT> runner =
+ new PrecombineRunner(
+ pipelineOptions,
+ combineFn,
+ MultiplexingFnDataReceiver.forConsumers(consumers),
+ keyCoder,
+ accumCoder);
+
+ // Register the appropriate handlers.
+ addStartFunction.accept(runner::startBundle);
Review comment:
Fixed
----------------------------------------------------------------
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: 117669)
Time Spent: 3.5h (was: 3h 20m)
> 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: 3.5h
> 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)