johnyangk commented on a change in pull request #123: [NEMO-129] Support Beam's 
WindowedWordCount example
URL: https://github.com/apache/incubator-nemo/pull/123#discussion_r225482130
 
 

 ##########
 File path: 
compiler/frontend/beam/src/main/java/org/apache/nemo/compiler/frontend/beam/transform/GroupByKeyTransform.java
 ##########
 @@ -26,53 +32,172 @@
 
 /**
  * Group Beam KVs.
- * @param <I> input type.
+ * @param <K> key type.
+ * @param <InputT> input type.
  */
-public final class GroupByKeyTransform<I> implements Transform<I, 
WindowedValue<KV<Object, List>>> {
+public final class GroupByKeyTransform<K, InputT>
+    extends AbstractTransform<KV<K, InputT>, KeyedWorkItem<K, InputT>, KV<K, 
Iterable<InputT>>> {
   private static final Logger LOG = 
LoggerFactory.getLogger(GroupByKeyTransform.class.getName());
-  private final Map<Object, List> keyToValues;
-  private OutputCollector<WindowedValue<KV<Object, List>>> outputCollector;
+
+  private final SystemReduceFn reduceFn;
+  private transient TimerInternalsFactory timerInternalsFactory;
 
   /**
    * GroupByKey constructor.
    */
-  public GroupByKeyTransform() {
-    this.keyToValues = new HashMap<>();
+  public GroupByKeyTransform(final Map<TupleTag<?>, Coder<?>> outputCoders,
+                             final TupleTag<KV<K, Iterable<InputT>>> 
mainOutputTag,
+                             final List<TupleTag<?>> additionalOutputTags,
+                             final WindowingStrategy<?, ?> windowingStrategy,
+                             final Collection<PCollectionView<?>> sideInputs,
+                             final PipelineOptions options,
+                             final SystemReduceFn reduceFn) {
+    super(null, /* doFn */
+      null, /* inputCoder */
+      outputCoders,
+      mainOutputTag,
+      additionalOutputTags,
+      windowingStrategy,
+      sideInputs,
+      options);
+    this.reduceFn = reduceFn;
   }
 
+  /**
+   * This creates a new DoFn that groups elements by key and window.
+   * @param doFn original doFn.
+   * @return GroupAlsoByWindowViaWindowSetNewDoFn
+   */
   @Override
-  public void prepare(final Context context, final 
OutputCollector<WindowedValue<KV<Object, List>>> oc) {
-    this.outputCollector = oc;
+  protected DoFn wrapDoFn(final DoFn doFn) {
+    timerInternalsFactory = new InMemoryTimerInternalsFactory();
+    return
+      GroupAlsoByWindowViaWindowSetNewDoFn.create(
+        getWindowingStrategy(),
+        new InMemoryStateInternalsFactory(),
+        timerInternalsFactory,
+        getSideInputReader(),
+        reduceFn,
+        getOutputManager(),
+        getMainOutputTag());
   }
 
   @Override
-  public void onData(final I element) {
-    // TODO #129: support window in group by key for windowed groupByKey
-    final WindowedValue<KV> windowedValue = (WindowedValue<KV>) element;
-    final KV kv = windowedValue.getValue();
-    keyToValues.putIfAbsent(kv.getKey(), new ArrayList());
-    keyToValues.get(kv.getKey()).add(kv.getValue());
+  public void onData(final WindowedValue<KV<K, InputT>> element) {
+    // The GroupAlsoByWindowViaWindowSetNewDoFn requires KeyedWorkItem,
+    // so we convert the KV to KeyedWorkItem
+    final KV<K, InputT> kv = element.getValue();
+    final KeyedWorkItem<K, InputT> keyedWorkItem =
+      KeyedWorkItems.elementsWorkItem(kv.getKey(),
+        Collections.singletonList(element.withValue(kv.getValue())));
 
 Review comment:
   Your new per-key grouping is 1.5X slower. Much better than per-value 
grouping which is 4X.
   
   However, 1.5X is a pretty large overhead. It may be better to switch to the 
'batch mode' like the Flink runner when there's no window operator in the 
pipeline, such that workloads like TPC-H avoids this overhead by using our old 
GBK that does not considers windows.

----------------------------------------------------------------
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]


With regards,
Apache Git Services

Reply via email to