|
94 | 94 | import static org.junit.jupiter.api.Assertions.assertThrows; |
95 | 95 | import static org.junit.jupiter.api.Assertions.assertTrue; |
96 | 96 | import static org.mockito.ArgumentMatchers.any; |
| 97 | +import static org.mockito.ArgumentMatchers.anyLong; |
97 | 98 | import static org.mockito.ArgumentMatchers.argThat; |
98 | 99 | import static org.mockito.ArgumentMatchers.eq; |
| 100 | +import static org.mockito.Mockito.doThrow; |
99 | 101 | import static org.mockito.Mockito.mock; |
100 | 102 | import static org.mockito.Mockito.spy; |
101 | 103 | import static org.mockito.Mockito.times; |
@@ -1116,6 +1118,105 @@ public void testScheduleUnloadingWithStalePartitionEpoch() { |
1116 | 1118 | assertEquals(10, ctx.epoch); |
1117 | 1119 | } |
1118 | 1120 |
|
| 1121 | + @Test |
| 1122 | + public void testScheduleUnloadingWithDeferredEventExceptions() throws ExecutionException, InterruptedException, TimeoutException { |
| 1123 | + MockTimer timer = new MockTimer(); |
| 1124 | + MockPartitionWriter writer = new MockPartitionWriter(); |
| 1125 | + MockCoordinatorShardBuilderSupplier supplier = mock(MockCoordinatorShardBuilderSupplier.class); |
| 1126 | + MockCoordinatorShardBuilder builder = mock(MockCoordinatorShardBuilder.class); |
| 1127 | + MockCoordinatorShard coordinator = mock(MockCoordinatorShard.class); |
| 1128 | + CoordinatorRuntimeMetrics metrics = mock(CoordinatorRuntimeMetrics.class); |
| 1129 | + |
| 1130 | + // All operations will throw an exception when completed. |
| 1131 | + doThrow(new KafkaException("error")).when(metrics).recordEventPurgatoryTime(anyLong()); |
| 1132 | + |
| 1133 | + CoordinatorRuntime<MockCoordinatorShard, String> runtime = |
| 1134 | + new CoordinatorRuntime.Builder<MockCoordinatorShard, String>() |
| 1135 | + .withTime(timer.time()) |
| 1136 | + .withTimer(timer) |
| 1137 | + .withDefaultWriteTimeOut(Duration.ofMillis(20)) |
| 1138 | + .withLoader(new MockCoordinatorLoader()) |
| 1139 | + .withEventProcessor(new DirectEventProcessor()) |
| 1140 | + .withPartitionWriter(writer) |
| 1141 | + .withCoordinatorShardBuilderSupplier(supplier) |
| 1142 | + .withCoordinatorRuntimeMetrics(metrics) |
| 1143 | + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) |
| 1144 | + .withSerializer(new StringSerializer()) |
| 1145 | + .withAppendLingerMs(10) |
| 1146 | + .withExecutorService(mock(ExecutorService.class)) |
| 1147 | + .build(); |
| 1148 | + |
| 1149 | + when(builder.withSnapshotRegistry(any())).thenReturn(builder); |
| 1150 | + when(builder.withLogContext(any())).thenReturn(builder); |
| 1151 | + when(builder.withTime(any())).thenReturn(builder); |
| 1152 | + when(builder.withTimer(any())).thenReturn(builder); |
| 1153 | + when(builder.withCoordinatorMetrics(any())).thenReturn(builder); |
| 1154 | + when(builder.withTopicPartition(any())).thenReturn(builder); |
| 1155 | + when(builder.withExecutor(any())).thenReturn(builder); |
| 1156 | + when(builder.build()).thenReturn(coordinator); |
| 1157 | + when(supplier.get()).thenReturn(builder); |
| 1158 | + |
| 1159 | + // Load the coordinator. |
| 1160 | + runtime.scheduleLoadOperation(TP, 10); |
| 1161 | + CoordinatorRuntime<MockCoordinatorShard, String>.CoordinatorContext ctx = runtime.contextOrThrow(TP); |
| 1162 | + |
| 1163 | + // Get the max batch size. |
| 1164 | + int maxBatchSize = writer.config(TP).maxMessageSize(); |
| 1165 | + |
| 1166 | + // Create records with three quarters of the max batch size each, so that it is not |
| 1167 | + // possible to have more than one record in a single batch. |
| 1168 | + List<String> records = Stream.of('1', '2', '3').map(c -> { |
| 1169 | + char[] payload = new char[maxBatchSize * 3 / 4]; |
| 1170 | + Arrays.fill(payload, c); |
| 1171 | + return new String(payload); |
| 1172 | + }).collect(Collectors.toList()); |
| 1173 | + |
| 1174 | + // Write #1. |
| 1175 | + CompletableFuture<String> write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), |
| 1176 | + state -> new CoordinatorResult<>(List.of(records.get(0)), "response1") |
| 1177 | + ); |
| 1178 | + |
| 1179 | + // Write #2. |
| 1180 | + CompletableFuture<String> write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), |
| 1181 | + state -> new CoordinatorResult<>(List.of(records.get(1)), "response2") |
| 1182 | + ); |
| 1183 | + |
| 1184 | + // Write #3, to force the flush of write #2. |
| 1185 | + CompletableFuture<String> write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), |
| 1186 | + state -> new CoordinatorResult<>(List.of(records.get(1)), "response3") |
| 1187 | + ); |
| 1188 | + |
| 1189 | + // Records have been written to the log. |
| 1190 | + assertEquals(List.of( |
| 1191 | + records(timer.time().milliseconds(), records.get(0)), |
| 1192 | + records(timer.time().milliseconds(), records.get(1)) |
| 1193 | + ), writer.entries(TP)); |
| 1194 | + |
| 1195 | + // Verify that no writes are committed yet. |
| 1196 | + assertFalse(write1.isDone()); |
| 1197 | + assertFalse(write2.isDone()); |
| 1198 | + assertFalse(write3.isDone()); |
| 1199 | + |
| 1200 | + // Schedule the unloading. |
| 1201 | + runtime.scheduleUnloadOperation(TP, OptionalInt.of(ctx.epoch + 1)); |
| 1202 | + assertEquals(CLOSED, ctx.state); |
| 1203 | + |
| 1204 | + // All write completions throw exceptions after completing their futures. |
| 1205 | + // Despite the exceptions, the unload should still complete. |
| 1206 | + assertTrue(write1.isDone()); |
| 1207 | + assertTrue(write2.isDone()); |
| 1208 | + assertTrue(write3.isDone()); |
| 1209 | + assertFutureThrows(write1, NotCoordinatorException.class); |
| 1210 | + assertFutureThrows(write2, NotCoordinatorException.class); |
| 1211 | + assertFutureThrows(write3, NotCoordinatorException.class); |
| 1212 | + |
| 1213 | + // Verify that onUnloaded is called. |
| 1214 | + verify(coordinator, times(1)).onUnloaded(); |
| 1215 | + |
| 1216 | + // Getting the coordinator context fails because it no longer exists. |
| 1217 | + assertThrows(NotCoordinatorException.class, () -> runtime.contextOrThrow(TP)); |
| 1218 | + } |
| 1219 | + |
1119 | 1220 | @Test |
1120 | 1221 | public void testScheduleWriteOp() throws ExecutionException, InterruptedException, TimeoutException { |
1121 | 1222 | MockTimer timer = new MockTimer(); |
@@ -3080,6 +3181,83 @@ public void testHighWatermarkUpdate() { |
3080 | 3181 | assertTrue(write2.isDone()); |
3081 | 3182 | } |
3082 | 3183 |
|
| 3184 | + @Test |
| 3185 | + public void testHighWatermarkUpdateWithDeferredEventExceptions() throws ExecutionException, InterruptedException, TimeoutException { |
| 3186 | + MockTimer timer = new MockTimer(); |
| 3187 | + MockPartitionWriter writer = new MockPartitionWriter(); |
| 3188 | + CoordinatorRuntimeMetrics metrics = mock(CoordinatorRuntimeMetrics.class); |
| 3189 | + |
| 3190 | + // All operations will throw an exception when completed. |
| 3191 | + doThrow(new KafkaException("error")).when(metrics).recordEventPurgatoryTime(anyLong()); |
| 3192 | + |
| 3193 | + CoordinatorRuntime<MockCoordinatorShard, String> runtime = |
| 3194 | + new CoordinatorRuntime.Builder<MockCoordinatorShard, String>() |
| 3195 | + .withTime(timer.time()) |
| 3196 | + .withTimer(timer) |
| 3197 | + .withDefaultWriteTimeOut(Duration.ofMillis(20)) |
| 3198 | + .withLoader(new MockCoordinatorLoader()) |
| 3199 | + .withEventProcessor(new DirectEventProcessor()) |
| 3200 | + .withPartitionWriter(writer) |
| 3201 | + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) |
| 3202 | + .withCoordinatorRuntimeMetrics(metrics) |
| 3203 | + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) |
| 3204 | + .withSerializer(new StringSerializer()) |
| 3205 | + .withAppendLingerMs(10) |
| 3206 | + .withExecutorService(mock(ExecutorService.class)) |
| 3207 | + .build(); |
| 3208 | + |
| 3209 | + // Load the coordinator. |
| 3210 | + runtime.scheduleLoadOperation(TP, 10); |
| 3211 | + |
| 3212 | + // Get the max batch size. |
| 3213 | + int maxBatchSize = writer.config(TP).maxMessageSize(); |
| 3214 | + |
| 3215 | + // Create records with three quarters of the max batch size each, so that it is not |
| 3216 | + // possible to have more than one record in a single batch. |
| 3217 | + List<String> records = Stream.of('1', '2', '3').map(c -> { |
| 3218 | + char[] payload = new char[maxBatchSize * 3 / 4]; |
| 3219 | + Arrays.fill(payload, c); |
| 3220 | + return new String(payload); |
| 3221 | + }).collect(Collectors.toList()); |
| 3222 | + |
| 3223 | + // Write #1. |
| 3224 | + CompletableFuture<String> write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), |
| 3225 | + state -> new CoordinatorResult<>(List.of(records.get(0)), "response1") |
| 3226 | + ); |
| 3227 | + |
| 3228 | + // Write #2. |
| 3229 | + CompletableFuture<String> write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), |
| 3230 | + state -> new CoordinatorResult<>(List.of(records.get(1)), "response2") |
| 3231 | + ); |
| 3232 | + |
| 3233 | + // Write #3, to force the flush of write #2. |
| 3234 | + CompletableFuture<String> write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), |
| 3235 | + state -> new CoordinatorResult<>(List.of(records.get(1)), "response3") |
| 3236 | + ); |
| 3237 | + |
| 3238 | + // Records have been written to the log. |
| 3239 | + assertEquals(List.of( |
| 3240 | + records(timer.time().milliseconds(), records.get(0)), |
| 3241 | + records(timer.time().milliseconds(), records.get(1)) |
| 3242 | + ), writer.entries(TP)); |
| 3243 | + |
| 3244 | + // Verify that no writes are committed yet. |
| 3245 | + assertFalse(write1.isDone()); |
| 3246 | + assertFalse(write2.isDone()); |
| 3247 | + assertFalse(write3.isDone()); |
| 3248 | + |
| 3249 | + // Commit the first and second record. |
| 3250 | + writer.commit(TP, 2); |
| 3251 | + |
| 3252 | + // Write #1 and write #2's completions throw exceptions after completing their futures. |
| 3253 | + // Despite the exception from write #1, write #2 should still be completed. |
| 3254 | + assertTrue(write1.isDone()); |
| 3255 | + assertTrue(write2.isDone()); |
| 3256 | + assertFalse(write3.isDone()); |
| 3257 | + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); |
| 3258 | + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); |
| 3259 | + } |
| 3260 | + |
3083 | 3261 | @Test |
3084 | 3262 | public void testWriteEventWriteTimeoutTaskIsCancelledWhenHighWatermarkIsUpdated() { |
3085 | 3263 | MockTimer timer = new MockTimer(); |
|
0 commit comments