[ 
https://issues.apache.org/jira/browse/BEAM-3708?focusedWorklogId=118310&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-118310
 ]

ASF GitHub Bot logged work on BEAM-3708:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Jul/18 18:06
            Start Date: 02/Jul/18 18:06
    Worklog Time Spent: 10m 
      Work Description: lukecwik 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_r199303366
 
 

 ##########
 File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
 ##########
 @@ -51,17 +71,120 @@
     }
   }
 
-  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");
+  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() {
+      groupingTable =
+          PrecombineGroupingTable.combiningAndSampling(
+              options, combineFn, keyCoder, 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. */
+  @VisibleForTesting
+  public 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));
+
+      // Input coder may sometimes be WindowedValueCoder depending on runner, 
instead of the
+      // expected KvCoder.
+      Coder<?> uncastInputCoder = 
rehydratedComponents.getCoder(mainInput.getCoderId());
+      KvCoder<KeyT, InputT> inputCoder;
 
 Review comment:
   You don't use the `inputCoder` anywhere except to get the key coder.
   
   Consider dropping the local variable `inputCoder` and setting `keyCoder` 
directly.

----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 118310)
    Time Spent: 4h 20m  (was: 4h 10m)

> 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: 4h 20m
>  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)

Reply via email to