ibzib commented on a change in pull request #12637:
URL: https://github.com/apache/beam/pull/12637#discussion_r492386778
##########
File path: sdks/python/apache_beam/runners/worker/data_plane_test.py
##########
@@ -108,16 +106,28 @@ def send(instruction_id, transform_id, data):
])
# Multiple interleaved writes to multiple instructions.
- send('1', transform_1, b'abc')
- send('2', transform_1, b'def')
+ stream11 = from_channel.output_stream('1', transform_1)
+ stream11.write(b'abc')
+ stream21 = from_channel.output_stream('2', transform_1)
+ stream21.write(b'def')
+ if not time_based_flush:
+ stream11.close()
self.assertEqual(
list(
itertools.islice(to_channel.input_elements('1', [transform_1]),
1)),
[
beam_fn_api_pb2.Elements.Data(
instruction_id='1', transform_id=transform_1, data=b'abc')
])
- send('2', transform_2, b'ghi')
+ if time_based_flush:
Review comment:
Write does not provide ordering guarantees in this case.
Elements are stored in a
[queue](https://github.com/apache/beam/blob/7b3d4251d244c10545fb37f1d93ebcad84a98681/sdks/python/apache_beam/runners/worker/data_plane.py#L371)
before being sent, to enable batching. Elements aren't added to that queue
until the [flush
callback](https://github.com/apache/beam/blob/7b3d4251d244c10545fb37f1d93ebcad84a98681/sdks/python/apache_beam/runners/worker/data_plane.py#L493)
is invoked. Because the flush callback is [invoked
periodically](https://github.com/apache/beam/blob/7b3d4251d244c10545fb37f1d93ebcad84a98681/sdks/python/apache_beam/runners/worker/data_plane.py#L182)
starting from when a stream is constructed, there is no guarantee that one
stream's callback is called before the other.
##########
File path: sdks/python/apache_beam/runners/worker/data_plane_test.py
##########
@@ -108,16 +106,28 @@ def send(instruction_id, transform_id, data):
])
# Multiple interleaved writes to multiple instructions.
- send('1', transform_1, b'abc')
- send('2', transform_1, b'def')
+ stream11 = from_channel.output_stream('1', transform_1)
+ stream11.write(b'abc')
+ stream21 = from_channel.output_stream('2', transform_1)
+ stream21.write(b'def')
+ if not time_based_flush:
+ stream11.close()
self.assertEqual(
list(
itertools.islice(to_channel.input_elements('1', [transform_1]),
1)),
[
beam_fn_api_pb2.Elements.Data(
instruction_id='1', transform_id=transform_1, data=b'abc')
])
- send('2', transform_2, b'ghi')
+ if time_based_flush:
Review comment:
> Please add details as comment.
Done.
----------------------------------------------------------------
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]