forked from apache/kafka
/
EventAccumulatorTest.java
233 lines (183 loc) · 8.01 KB
/
EventAccumulatorTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.coordinator.group.runtime;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
public class EventAccumulatorTest {
private class MockEvent implements EventAccumulator.Event<Integer> {
int key;
int value;
MockEvent(int key, int value) {
this.key = key;
this.value = value;
}
@Override
public Integer key() {
return key;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
MockEvent mockEvent = (MockEvent) o;
if (key != mockEvent.key) return false;
return value == mockEvent.value;
}
@Override
public int hashCode() {
int result = key;
result = 31 * result + value;
return result;
}
@Override
public String toString() {
return "MockEvent(key=" + key + ", value=" + value + ')';
}
}
@Test
public void testBasicOperations() {
EventAccumulator<Integer, MockEvent> accumulator = new EventAccumulator<>();
assertEquals(0, accumulator.size());
assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS));
List<MockEvent> events = Arrays.asList(
new MockEvent(1, 0),
new MockEvent(1, 1),
new MockEvent(1, 2),
new MockEvent(2, 0),
new MockEvent(2, 1),
new MockEvent(2, 3),
new MockEvent(3, 0),
new MockEvent(3, 1),
new MockEvent(3, 2)
);
events.forEach(accumulator::add);
assertEquals(9, accumulator.size());
Set<MockEvent> polledEvents = new HashSet<>();
for (int i = 0; i < events.size(); i++) {
MockEvent event = accumulator.poll(0, TimeUnit.MICROSECONDS);
assertNotNull(event);
polledEvents.add(event);
assertEquals(events.size() - 1 - i, accumulator.size());
accumulator.done(event);
}
assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS));
assertEquals(new HashSet<>(events), polledEvents);
assertEquals(0, accumulator.size());
accumulator.close();
}
@Test
public void testKeyConcurrentAndOrderingGuarantees() {
EventAccumulator<Integer, MockEvent> accumulator = new EventAccumulator<>();
MockEvent event0 = new MockEvent(1, 0);
MockEvent event1 = new MockEvent(1, 1);
MockEvent event2 = new MockEvent(1, 2);
accumulator.add(event0);
accumulator.add(event1);
accumulator.add(event2);
assertEquals(3, accumulator.size());
MockEvent event = null;
// Poll event0.
event = accumulator.poll(0, TimeUnit.MICROSECONDS);
assertEquals(event0, event);
// Poll returns null because key is inflight.
assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS));
accumulator.done(event);
// Poll event1.
event = accumulator.poll(0, TimeUnit.MICROSECONDS);
assertEquals(event1, event);
// Poll returns null because key is inflight.
assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS));
accumulator.done(event);
// Poll event2.
event = accumulator.poll(0, TimeUnit.MICROSECONDS);
assertEquals(event2, event);
// Poll returns null because key is inflight.
assertNull(accumulator.poll(0, TimeUnit.MICROSECONDS));
accumulator.done(event);
accumulator.close();
}
@Test
public void testDoneUnblockWaitingThreads() throws ExecutionException, InterruptedException, TimeoutException {
EventAccumulator<Integer, MockEvent> accumulator = new EventAccumulator<>();
MockEvent event0 = new MockEvent(1, 0);
MockEvent event1 = new MockEvent(1, 1);
MockEvent event2 = new MockEvent(1, 2);
CompletableFuture<MockEvent> future0 = CompletableFuture.supplyAsync(accumulator::poll);
CompletableFuture<MockEvent> future1 = CompletableFuture.supplyAsync(accumulator::poll);
CompletableFuture<MockEvent> future2 = CompletableFuture.supplyAsync(accumulator::poll);
List<CompletableFuture<MockEvent>> futures = Arrays.asList(future0, future1, future2);
assertFalse(future0.isDone());
assertFalse(future1.isDone());
assertFalse(future2.isDone());
accumulator.add(event0);
accumulator.add(event1);
accumulator.add(event2);
// One future should be completed with event0.
assertEquals(event0, CompletableFuture
.anyOf(futures.toArray(new CompletableFuture[0]))
.get(5, TimeUnit.SECONDS));
futures = futures.stream().filter(future -> !future.isDone()).collect(Collectors.toList());
assertEquals(2, futures.size());
// Processing of event0 is done.
accumulator.done(event0);
// One future should be completed with event1.
assertEquals(event1, CompletableFuture
.anyOf(futures.toArray(new CompletableFuture[0]))
.get(5, TimeUnit.SECONDS));
futures = futures.stream().filter(future -> !future.isDone()).collect(Collectors.toList());
assertEquals(1, futures.size());
// Processing of event1 is done.
accumulator.done(event1);
// One future should be completed with event2.
assertEquals(event2, CompletableFuture
.anyOf(futures.toArray(new CompletableFuture[0]))
.get(5, TimeUnit.SECONDS));
futures = futures.stream().filter(future -> !future.isDone()).collect(Collectors.toList());
assertEquals(0, futures.size());
// Processing of event2 is done.
accumulator.done(event2);
assertEquals(0, accumulator.size());
accumulator.close();
}
@Test
public void testCloseUnblockWaitingThreads() throws ExecutionException, InterruptedException, TimeoutException {
EventAccumulator<Integer, MockEvent> accumulator = new EventAccumulator<>();
CompletableFuture<MockEvent> future0 = CompletableFuture.supplyAsync(accumulator::poll);
CompletableFuture<MockEvent> future1 = CompletableFuture.supplyAsync(accumulator::poll);
CompletableFuture<MockEvent> future2 = CompletableFuture.supplyAsync(accumulator::poll);
assertFalse(future0.isDone());
assertFalse(future1.isDone());
assertFalse(future2.isDone());
// Closing should release all the pending futures.
accumulator.close();
assertNull(future0.get(5, TimeUnit.SECONDS));
assertNull(future1.get(5, TimeUnit.SECONDS));
assertNull(future2.get(5, TimeUnit.SECONDS));
}
}