claudevdm commented on code in PR #38015:
URL: https://github.com/apache/beam/pull/38015#discussion_r3018590463


##########
sdks/python/apache_beam/io/gcp/bigquery_change_history.py:
##########
@@ -1170,16 +1315,46 @@ def expand(self, pbegin: beam.pvalue.PBegin) -> 
beam.PCollection:
                 row_filter=self._row_filter))
         | 'CommitQueryResults' >> beam.Reshuffle())
 
+    emit_raw = self._decompress_shards is not None
+
+    read_sdf = beam.ParDo(
+        _ReadStorageStreamsSDF(
+            batch_arrow_read=self._batch_arrow_read,
+            change_timestamp_column=self._change_timestamp_column,
+            max_split_rounds=self._max_split_rounds,
+            emit_raw_batches=emit_raw))
+    if emit_raw:
+      read_sdf = read_sdf.with_output_types(Tuple[bytes, bytes])
+    else:
+      read_sdf = read_sdf.with_output_types(Dict[str, Any])
+
     read_outputs = (
         query_results
-        | 'ReadStorageStreams' >> beam.ParDo(
-            _ReadStorageStreamsSDF(
-                batch_arrow_read=self._batch_arrow_read,
-                change_timestamp_column=self._change_timestamp_column)).
-        with_outputs(_CLEANUP_TAG, main='rows'))
+        | 'ReadStorageStreams' >> read_sdf.with_outputs(
+            _CLEANUP_TAG, main='rows'))
 
     _ = (
         read_outputs[_CLEANUP_TAG]
         | 'CleanupTempTables' >> beam.ParDo(_CleanupTempTablesFn()))
 
-    return read_outputs['rows']
+    if emit_raw:
+      # Fan out raw Arrow batches across decompress_shards workers
+      # via GBK, then decompress and convert to timestamped row dicts.
+      # Uses a discarding trigger so GBK fires per-element without
+      # waiting for the GlobalWindow to close.
+      num_shards = self._decompress_shards
+      rows = (
+          read_outputs['rows']
+          | 'ShardBatches' >>
+          beam.WithKeys(lambda _, n=num_shards: random.randint(0, n - 1))
+          | 'WindowForGBK' >> beam.WindowInto(
+              GlobalWindows(),
+              trigger=beam_trigger.Repeatedly(beam_trigger.AfterCount(1)),
+              accumulation_mode=(beam_trigger.AccumulationMode.DISCARDING))
+          | 'GroupByShardKey' >> beam.GroupByKey()

Review Comment:
   Less shuffle if the the pipeline can handle the load without fanning out. 
But probably better to just fanout since compressed data is very small shuffle 
load



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

Reply via email to