[
https://issues.apache.org/jira/browse/BEAM-10475?focusedWorklogId=503453&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-503453
]
ASF GitHub Bot logged work on BEAM-10475:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 22/Oct/20 00:20
Start Date: 22/Oct/20 00:20
Worklog Time Spent: 10m
Work Description: robertwb commented on a change in pull request #13144:
URL: https://github.com/apache/beam/pull/13144#discussion_r509806698
##########
File path: sdks/python/apache_beam/transforms/util.py
##########
@@ -777,33 +795,56 @@ def process(
window=DoFn.WindowParam,
element_state=DoFn.StateParam(ELEMENT_STATE),
count_state=DoFn.StateParam(COUNT_STATE),
- expiry_timer=DoFn.TimerParam(EXPIRY_TIMER)):
+ window_timer=DoFn.TimerParam(WINDOW_TIMER),
+ buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)):
# Allowed lateness not supported in Python SDK
#
https://beam.apache.org/documentation/programming-guide/#watermarks-and-late-data
- expiry_timer.set(window.end)
+ window_timer.set(window.end)
element_state.add(element)
count_state.add(1)
count = count_state.read()
+ if count == 1 and max_buffering_duration_secs is not None:
+ # This is the first element in batch. Start counting buffering time if
a
+ # limit was set.
+ buffering_timer.set(clock() + max_buffering_duration_secs)
if count >= batch_size:
batch = [element for element in element_state.read()]
key, _ = batch[0]
batch_values = [v for (k, v) in batch]
- yield (key, batch_values)
+ yield key, batch_values
element_state.clear()
count_state.clear()
+ buffering_timer.clear()
- @on_timer(EXPIRY_TIMER)
- def expiry(
+ @on_timer(WINDOW_TIMER)
+ def on_window_timer(
self,
element_state=DoFn.StateParam(ELEMENT_STATE),
- count_state=DoFn.StateParam(COUNT_STATE)):
+ count_state=DoFn.StateParam(COUNT_STATE),
+ buffering_timer=DoFn.TimerParam(BUFFERING_TIMER)):
+ batch = [element for element in element_state.read()]
+ if batch:
+ key, _ = batch[0]
Review comment:
Pull this (and the exact same logic above and below) into common helper
method.
##########
File path: sdks/python/apache_beam/transforms/util.py
##########
@@ -1009,7 +1050,7 @@ def _process(element):
@staticmethod
@typehints.with_input_types(str)
- @typehints.with_output_types(Union[List[str], List[Tuple[str, str]]])
+ @typehints.with_output_types(Union[List[str], Tuple[str, str]])
Review comment:
Was this wrong?
##########
File path: sdks/python/apache_beam/transforms/util_test.py
##########
@@ -644,6 +657,12 @@ def _create_test_data():
data.append(("key", scientists[index]))
return data
+ class _ExpandIterable(DoFn):
Review comment:
_ExpandValuesIterable?
This could also be `FlatMapTuple(lambda k, vs: vs)`
##########
File path: sdks/python/apache_beam/transforms/util_test.py
##########
@@ -110,10 +117,16 @@ def test_windowed_batches(self):
| util.BatchElements(
min_batch_size=5, max_batch_size=10, clock=FakeClock())
| beam.Map(len))
- assert_that(res, equal_to([
- 5, 5, 10, 10, # elements in [0, 30)
- 10, 7, # elements in [30, 47)
- ]))
+ assert_that(
+ res,
+ equal_to([
+ 5,
+ 5,
+ 10,
+ 10, # elements in [0, 30)
Review comment:
Nit: I prefer the previous formatting (as the comment applies to the
whole set of elements).
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 503453)
Time Spent: 13h 40m (was: 13.5h)
> GroupIntoBatches with Runner-determined Sharding
> ------------------------------------------------
>
> Key: BEAM-10475
> URL: https://issues.apache.org/jira/browse/BEAM-10475
> Project: Beam
> Issue Type: Improvement
> Components: runner-dataflow
> Reporter: Siyuan Chen
> Assignee: Siyuan Chen
> Priority: P2
> Labels: GCP, performance
> Time Spent: 13h 40m
> Remaining Estimate: 0h
>
> [https://s.apache.org/sharded-group-into-batches|https://s.apache.org/sharded-group-into-batches__]
> Improve the existing Beam transform, GroupIntoBatches, to allow runners to
> choose different sharding strategies depending on how the data needs to be
> grouped. The goal is to help with the situation where the elements to process
> need to be co-located to reduce the overhead that would otherwise be incurred
> per element, while not losing the ability to scale the parallelism. The
> essential idea is to build a stateful DoFn with shardable states.
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)