|
49 | 49 | import java.time.Duration; |
50 | 50 | import java.util.ArrayList; |
51 | 51 | import java.util.Arrays; |
| 52 | +import java.util.Collections; |
52 | 53 | import java.util.List; |
53 | 54 | import java.util.Map; |
54 | 55 | import java.util.OptionalInt; |
@@ -4182,6 +4183,119 @@ public void testEmptyBatch() throws Exception { |
4182 | 4183 | assertNull(complete1.get(5, TimeUnit.SECONDS)); |
4183 | 4184 | } |
4184 | 4185 |
|
| 4186 | + @Test |
| 4187 | + public void testRecordAppendLingerTime() throws Exception { |
| 4188 | + MockTimer timer = new MockTimer(); |
| 4189 | + |
| 4190 | + // Writer sleeps for 10ms before appending records. |
| 4191 | + MockPartitionWriter writer = new MockPartitionWriter(timer.time(), Integer.MAX_VALUE, false); |
| 4192 | + CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); |
| 4193 | + |
| 4194 | + CoordinatorRuntime<MockCoordinatorShard, String> runtime = |
| 4195 | + new CoordinatorRuntime.Builder<MockCoordinatorShard, String>() |
| 4196 | + .withTime(timer.time()) |
| 4197 | + .withTimer(timer) |
| 4198 | + .withDefaultWriteTimeOut(Duration.ofMillis(20)) |
| 4199 | + .withLoader(new MockCoordinatorLoader()) |
| 4200 | + .withEventProcessor(new DirectEventProcessor()) |
| 4201 | + .withPartitionWriter(writer) |
| 4202 | + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) |
| 4203 | + .withCoordinatorRuntimeMetrics(runtimeMetrics) |
| 4204 | + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) |
| 4205 | + .withSerializer(new StringSerializer()) |
| 4206 | + .withAppendLingerMs(10) |
| 4207 | + .withExecutorService(mock(ExecutorService.class)) |
| 4208 | + .build(); |
| 4209 | + |
| 4210 | + // Schedule the loading. |
| 4211 | + runtime.scheduleLoadOperation(TP, 10); |
| 4212 | + |
| 4213 | + // Verify the initial state. |
| 4214 | + CoordinatorRuntime<MockCoordinatorShard, String>.CoordinatorContext ctx = runtime.contextOrThrow(TP); |
| 4215 | + assertNull(ctx.currentBatch); |
| 4216 | + |
| 4217 | + // Get the max batch size. |
| 4218 | + int maxBatchSize = writer.config(TP).maxMessageSize(); |
| 4219 | + |
| 4220 | + // Create records with a quarter of the max batch size each. Keep in mind that |
| 4221 | + // each batch has a header so it is not possible to have those four records |
| 4222 | + // in one single batch. |
| 4223 | + List<String> records = Stream.of('1', '2', '3', '4').map(c -> { |
| 4224 | + char[] payload = new char[maxBatchSize / 4]; |
| 4225 | + Arrays.fill(payload, c); |
| 4226 | + return new String(payload); |
| 4227 | + }).collect(Collectors.toList()); |
| 4228 | + |
| 4229 | + // Write #1 with two records. |
| 4230 | + long firstBatchTimestamp = timer.time().milliseconds(); |
| 4231 | + CompletableFuture<String> write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), |
| 4232 | + state -> new CoordinatorResult<>(records.subList(0, 2), "response1") |
| 4233 | + ); |
| 4234 | + |
| 4235 | + // A batch has been created. |
| 4236 | + assertNotNull(ctx.currentBatch); |
| 4237 | + |
| 4238 | + // Write #2 with one record. |
| 4239 | + CompletableFuture<String> write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), |
| 4240 | + state -> new CoordinatorResult<>(records.subList(2, 3), "response2") |
| 4241 | + ); |
| 4242 | + |
| 4243 | + // Verify the state. Records are replayed but no batch written. |
| 4244 | + assertEquals(Collections.emptyList(), writer.entries(TP)); |
| 4245 | + verify(runtimeMetrics, times(0)).recordFlushTime(10); |
| 4246 | + |
| 4247 | + // Write #3 with one record. This one cannot go into the existing batch |
| 4248 | + // so the existing batch should be flushed and a new one should be created. |
| 4249 | + long secondBatchTimestamp = timer.time().milliseconds(); |
| 4250 | + CompletableFuture<String> write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(50), |
| 4251 | + state -> new CoordinatorResult<>(records.subList(3, 4), "response3") |
| 4252 | + ); |
| 4253 | + |
| 4254 | + // Verify the state. Records are replayed. The previous batch |
| 4255 | + // got flushed with all the records but the new one from #3. |
| 4256 | + // The new batch's timestamp comes from before the flush. |
| 4257 | + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); |
| 4258 | + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); |
| 4259 | + assertEquals(List.of( |
| 4260 | + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), |
| 4261 | + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), |
| 4262 | + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), |
| 4263 | + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) |
| 4264 | + ), ctx.coordinator.coordinator().fullRecords()); |
| 4265 | + assertEquals(List.of( |
| 4266 | + records(firstBatchTimestamp, records.subList(0, 3)) |
| 4267 | + ), writer.entries(TP)); |
| 4268 | + verify(runtimeMetrics, times(1)).recordLingerTime(0); |
| 4269 | + |
| 4270 | + // Advance past the linger time. |
| 4271 | + timer.advanceClock(11); |
| 4272 | + |
| 4273 | + // Verify the state. The pending batch is flushed. |
| 4274 | + assertEquals(4L, ctx.coordinator.lastWrittenOffset()); |
| 4275 | + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); |
| 4276 | + assertEquals(List.of( |
| 4277 | + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), |
| 4278 | + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), |
| 4279 | + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), |
| 4280 | + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) |
| 4281 | + ), ctx.coordinator.coordinator().fullRecords()); |
| 4282 | + assertEquals(List.of( |
| 4283 | + records(secondBatchTimestamp, records.subList(0, 3)), |
| 4284 | + records(secondBatchTimestamp, records.subList(3, 4)) |
| 4285 | + ), writer.entries(TP)); |
| 4286 | + verify(runtimeMetrics, times(1)).recordLingerTime(21); |
| 4287 | + |
| 4288 | + // Commit and verify that writes are completed. |
| 4289 | + writer.commit(TP); |
| 4290 | + assertTrue(write1.isDone()); |
| 4291 | + assertTrue(write2.isDone()); |
| 4292 | + assertTrue(write3.isDone()); |
| 4293 | + assertEquals(4L, ctx.coordinator.lastCommittedOffset()); |
| 4294 | + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); |
| 4295 | + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); |
| 4296 | + assertEquals("response3", write3.get(5, TimeUnit.SECONDS)); |
| 4297 | + } |
| 4298 | + |
4185 | 4299 | @Test |
4186 | 4300 | public void testRecordFlushTime() throws Exception { |
4187 | 4301 | MockTimer timer = new MockTimer(); |
|
0 commit comments