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

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

                Author: ASF GitHub Bot
            Created on: 03/Aug/21 21:31
            Start Date: 03/Aug/21 21:31
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on a change in pull request #14852:
URL: https://github.com/apache/beam/pull/14852#discussion_r682116285



##########
File path: 
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java
##########
@@ -87,14 +94,25 @@ private BatchGroupIntoBatches(long batchSize) {
                   new DoFn<KV<K, Iterable<V>>, KV<K, Iterable<V>>>() {
                     @ProcessElement
                     public void process(ProcessContext c) {
-                      // Iterators.partition lazily creates the partitions as 
they are accessed
-                      // allowing it to partition very large iterators.
-                      Iterator<List<V>> iterator =
-                          
Iterators.partition(c.element().getValue().iterator(), (int) batchSize);
-
-                      // Note that GroupIntoBatches only outputs when the 
batch is non-empty.
-                      while (iterator.hasNext()) {
-                        c.output(KV.of(c.element().getKey(), iterator.next()));
+                      List<V> currentBatch = Lists.newArrayList();
+                      long batchSizeBytes = 0;
+                      for (V element : c.element().getValue()) {
+                        currentBatch.add(element);
+                        if (weigher != null) {
+                          batchSizeBytes += weigher.apply(element);
+                        }
+                        if (currentBatch.size() == maxBatchSizeElements
+                            || (maxBatchSizeBytes != Long.MAX_VALUE
+                                && batchSizeBytes >= maxBatchSizeBytes)) {
+                          c.output(KV.of(c.element().getKey(), currentBatch));
+                          // Call clear() since that allows us to reuse the 
array memory for
+                          // subsequent batches.
+                          currentBatch.clear();

Review comment:
       oof, this is a bug.
   
   It's probably ok in many cases, since the fused fn will be executed 
immediately when output is called. However the fused fn is allowed to store the 
element in memory and process it in finishBundle, and if that happens the data 
will be stomped. We need to remove this buggy optimization :P




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

    Worklog Id:     (was: 633175)
    Time Spent: 7h 20m  (was: 7h 10m)

> GroupIntoBatches should support byte-size batches
> -------------------------------------------------
>
>                 Key: BEAM-12378
>                 URL: https://issues.apache.org/jira/browse/BEAM-12378
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Reuven Lax
>            Priority: P3
>          Time Spent: 7h 20m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to