-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
Unordered mapUsingServiceAsync loses items when there are no watermarks [HZ-1928] #23245
Labels
Module: Jet
Issues/PRs for Jet
Source: Community
PR or issue was opened by a community user
to-jira
Type: Defect
Milestone
Comments
k-jamroz
added
Type: Defect
Source: Community
PR or issue was opened by a community user
Module: Jet
Issues/PRs for Jet
labels
Dec 22, 2022
It is likely that the problem was introduced in 5.2.z with multiple watermarks support. |
github-actions
bot
changed the title
Unordered mapUsingServiceAsync loses items when there are no watermarks
Unordered mapUsingServiceAsync loses items when there are no watermarks [HZ-1928]
Dec 22, 2022
Internal Jira issue: HZ-1928 |
Another problem with unordered async processor is that when the same element is processed multiple times. In such case internal counters are corrupted and processor crashes on assertion (and hangs the entire job). Reproducer (add in @Test
public void test_completedFutures_sameElementInterleavedWithWatermark() {
TestSupport
.verifyProcessor(getSupplier((ctx, item) -> completedFuture(singleton(item + "-1"))))
.hazelcastInstance(instance())
.input(asList("a", "a", wm(10), "a"))
.disableProgressAssertion()
.expectOutput(asList("a-1", "a-1", wm(10), "a-1"));
} causes:
|
k-jamroz
added a commit
to k-jamroz/hazelcast
that referenced
this issue
Dec 28, 2022
…d mapUsingServiceAsync hazelcast#23245
4 tasks
k-jamroz
added a commit
to k-jamroz/hazelcast
that referenced
this issue
Dec 28, 2022
…d mapUsingServiceAsync hazelcast#23245
k-jamroz
added a commit
to k-jamroz/hazelcast
that referenced
this issue
Dec 28, 2022
…and watermarks. Fixes hazelcast#23245
k-jamroz
added a commit
to k-jamroz/hazelcast
that referenced
this issue
Dec 28, 2022
…d mapUsingServiceAsync hazelcast#23245
k-jamroz
added a commit
to k-jamroz/hazelcast
that referenced
this issue
Dec 28, 2022
…and watermarks. Fixes hazelcast#23245
4 tasks
k-jamroz
added a commit
that referenced
this issue
Jan 12, 2023
] (#23271) mapUsingServiceAsync in unordered mode used wrong condition to check if the processing is complete and wrong number of currently processed items to propagate watermarks. This could lead to lost items when the pipeline was completed and to crashes during processing of streams containing the same item multiple times. Fixes #23245 Co-authored-by: Viliam Durina <viliam@hazelcast.com>
k-jamroz
added a commit
that referenced
this issue
Jan 12, 2023
] [5.2.z] (#23272) mapUsingServiceAsync in unordered mode used wrong condition to check if the processing is complete and wrong number of currently processed items to propagate watermarks. This could lead to lost items when the pipeline was completed and to crashes during processing of streams containing the same item multiple times. Fixes #23245 Backport of: #23271 Co-authored-by: Viliam Durina <viliam@hazelcast.com>
viliam-durina
pushed a commit
to viliam-durina/hazelcast
that referenced
this issue
Jan 14, 2023
…928] (hazelcast#23271) mapUsingServiceAsync in unordered mode used wrong condition to check if the processing is complete and wrong number of currently processed items to propagate watermarks. This could lead to lost items when the pipeline was completed and to crashes during processing of streams containing the same item multiple times. Fixes hazelcast#23245 Co-authored-by: Viliam Durina <viliam@hazelcast.com> (cherry picked from commit 2bec372)
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Labels
Module: Jet
Issues/PRs for Jet
Source: Community
PR or issue was opened by a community user
to-jira
Type: Defect
Describe the bug
Unordered
mapUsingServiceAsync
loses last items when there are no watermarks.To Reproduce
https://hazelcastcommunity.slack.com/archives/C013LM6JK7S/p1671652200501389
The text was updated successfully, but these errors were encountered: