-
Notifications
You must be signed in to change notification settings - Fork 150
Implement timeout and key routing for Stream.partition #376
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
b6f8af0
464f478
3747d8b
4425ad4
014a516
bf2e356
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -164,6 +164,65 @@ def test_partition(): | |
| assert L == [(0, 1), (2, 3), (4, 5), (6, 7), (8, 9)] | ||
|
|
||
|
|
||
| def test_partition_timeout(): | ||
| source = Stream() | ||
| L = source.partition(10, timeout=0.01).sink_to_list() | ||
|
|
||
| for i in range(5): | ||
| source.emit(i) | ||
|
|
||
| sleep(0.1) | ||
|
|
||
| assert L == [(0, 1, 2, 3, 4)] | ||
|
|
||
|
|
||
| def test_partition_timeout_cancel(): | ||
| source = Stream() | ||
| L = source.partition(3, timeout=0.1).sink_to_list() | ||
|
|
||
| for i in range(3): | ||
| source.emit(i) | ||
|
|
||
| sleep(0.09) | ||
| source.emit(3) | ||
| sleep(0.02) | ||
|
|
||
| assert L == [(0, 1, 2)] | ||
|
|
||
| sleep(0.09) | ||
|
|
||
| assert L == [(0, 1, 2), (3,)] | ||
|
|
||
|
|
||
| def test_partition_key(): | ||
| source = Stream() | ||
| L = source.partition(2, key=0).sink_to_list() | ||
|
|
||
| for i in range(4): | ||
| source.emit((i % 2, i)) | ||
|
|
||
| assert L == [((0, 0), (0, 2)), ((1, 1), (1, 3))] | ||
|
|
||
|
|
||
| def test_partition_key_callable(): | ||
| source = Stream() | ||
| L = source.partition(2, key=lambda x: x % 2).sink_to_list() | ||
|
|
||
| for i in range(10): | ||
| source.emit(i) | ||
|
|
||
| assert L == [(0, 2), (1, 3), (4, 6), (5, 7)] | ||
|
|
||
|
|
||
| def test_partition_size_one(): | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the goal of this test?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here: I didn't have
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm thinking if we should add a check in
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For the sake of test and debug, I think it's OK to allow a redundant partition with n=1. |
||
| source = Stream() | ||
|
|
||
| source.partition(1, timeout=.01).sink(lambda x: None) | ||
|
|
||
| for i in range(10): | ||
| source.emit(i) | ||
|
|
||
|
|
||
| def test_sliding_window(): | ||
| source = Stream() | ||
| L = source.sliding_window(2).sink_to_list() | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to create new variables
bufferandmetadata_bufferhere?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
cc: @jsmaupin for review because this PR is changing how metadata is being handled in
partition.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We really don't have to, but since they're referenced multiple times, I figured it'd me more readable than
self._metadata_buffer[key].There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the cc. The metadata changes look okay to me.