|
77 | 77 | import org.apache.pinot.spi.env.PinotConfiguration; |
78 | 78 | import org.apache.pinot.spi.filesystem.PinotFSFactory; |
79 | 79 | import org.apache.pinot.spi.stream.LongMsgOffset; |
| 80 | +import org.apache.pinot.spi.stream.LongMsgOffsetFactory; |
| 81 | +import org.apache.pinot.spi.stream.OffsetCriteria; |
80 | 82 | import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus; |
81 | 83 | import org.apache.pinot.spi.stream.PartitionGroupMetadata; |
82 | 84 | import org.apache.pinot.spi.stream.StreamConfig; |
| 85 | +import org.apache.pinot.spi.stream.StreamConfigProperties; |
| 86 | +import org.apache.pinot.spi.stream.StreamConsumerFactory; |
| 87 | +import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider; |
| 88 | +import org.apache.pinot.spi.stream.StreamMetadataProvider; |
83 | 89 | import org.apache.pinot.spi.utils.CommonConstants; |
84 | 90 | import org.apache.pinot.spi.utils.CommonConstants.Helix; |
85 | 91 | import org.apache.pinot.spi.utils.CommonConstants.Helix.Instance; |
|
91 | 97 | import org.apache.pinot.util.TestUtils; |
92 | 98 | import org.apache.zookeeper.data.Stat; |
93 | 99 | import org.joda.time.Interval; |
| 100 | +import org.mockito.MockedStatic; |
94 | 101 | import org.testng.Assert; |
95 | 102 | import org.testng.annotations.AfterClass; |
96 | 103 | import org.testng.annotations.Test; |
@@ -122,6 +129,7 @@ public class PinotLLCRealtimeSegmentManagerTest { |
122 | 129 | static final String CRC = Long.toString(RANDOM.nextLong() & 0xFFFFFFFFL); |
123 | 130 | static final SegmentVersion SEGMENT_VERSION = RANDOM.nextBoolean() ? SegmentVersion.v1 : SegmentVersion.v3; |
124 | 131 | static final int NUM_DOCS = RANDOM.nextInt(Integer.MAX_VALUE) + 1; |
| 132 | + static final long LATEST_OFFSET = PARTITION_OFFSET.getOffset() * 2 + NUM_DOCS; |
125 | 133 | static final int SEGMENT_SIZE_IN_BYTES = 100000000; |
126 | 134 | @AfterClass |
127 | 135 | public void tearDown() |
@@ -325,6 +333,139 @@ public void testCommitSegment() { |
325 | 333 | assertNull(consumingSegmentZKMetadata); |
326 | 334 | } |
327 | 335 |
|
| 336 | + @Test |
| 337 | + public void testCommitSegmentWithOffsetAutoResetOnOffset() |
| 338 | + throws Exception { |
| 339 | + // Set up a new table with 2 replicas, 5 instances, 4 partition |
| 340 | + PinotHelixResourceManager mockHelixResourceManager = mock(PinotHelixResourceManager.class); |
| 341 | + FakePinotLLCRealtimeSegmentManager segmentManager = |
| 342 | + new FakePinotLLCRealtimeSegmentManager(mockHelixResourceManager); |
| 343 | + setUpNewTable(segmentManager, 2, 5, 4); |
| 344 | + Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields(); |
| 345 | + Map<String, String> streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(segmentManager._tableConfig).get(0); |
| 346 | + streamConfigMap.put(StreamConfigProperties.ENABLE_OFFSET_AUTO_RESET, String.valueOf(true)); |
| 347 | + streamConfigMap.put(StreamConfigProperties.OFFSET_AUTO_RESET_OFFSET_THRESHOLD_KEY, "100"); |
| 348 | + segmentManager.makeTableConfig(streamConfigMap); |
| 349 | + |
| 350 | + StreamConsumerFactory mockConsumerFactory = mock(StreamConsumerFactory.class); |
| 351 | + StreamMetadataProvider mockMetadataProvider = mock(StreamMetadataProvider.class); |
| 352 | + when(mockConsumerFactory.createPartitionMetadataProvider(anyString(), anyInt())).thenReturn(mockMetadataProvider); |
| 353 | + when(mockConsumerFactory.createStreamMsgOffsetFactory()).thenReturn(new LongMsgOffsetFactory()); |
| 354 | + when(mockMetadataProvider.fetchStreamPartitionOffset(eq(OffsetCriteria.LARGEST_OFFSET_CRITERIA), |
| 355 | + anyLong())).thenReturn(new LongMsgOffset(LATEST_OFFSET)); |
| 356 | + when(mockMetadataProvider.getOffsetAtTimestamp(eq(0), anyLong(), anyLong())).thenReturn(PARTITION_OFFSET); |
| 357 | + |
| 358 | + try (MockedStatic<StreamConsumerFactoryProvider> mockedStaticProvider = mockStatic( |
| 359 | + StreamConsumerFactoryProvider.class)) { |
| 360 | + |
| 361 | + mockedStaticProvider.when(() -> StreamConsumerFactoryProvider.create(segmentManager._streamConfigs.get(0))) |
| 362 | + .thenReturn(mockConsumerFactory); |
| 363 | + |
| 364 | + // Commit a segment for partition group 0 |
| 365 | + String committingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName(); |
| 366 | + String endOffset = new LongMsgOffset(PARTITION_OFFSET.getOffset() + NUM_DOCS).toString(); |
| 367 | + CommittingSegmentDescriptor committingSegmentDescriptor = |
| 368 | + new CommittingSegmentDescriptor(committingSegment, endOffset, SEGMENT_SIZE_IN_BYTES); |
| 369 | + committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata()); |
| 370 | + segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor); |
| 371 | + |
| 372 | + // Verify instance states for committed segment and new consuming segment |
| 373 | + Map<String, String> committedSegmentInstanceStateMap = instanceStatesMap.get(committingSegment); |
| 374 | + assertNotNull(committedSegmentInstanceStateMap); |
| 375 | + assertEquals(new HashSet<>(committedSegmentInstanceStateMap.values()), |
| 376 | + Collections.singleton(SegmentStateModel.ONLINE)); |
| 377 | + |
| 378 | + String consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 1, CURRENT_TIME_MS).getSegmentName(); |
| 379 | + Map<String, String> consumingSegmentInstanceStateMap = instanceStatesMap.get(consumingSegment); |
| 380 | + assertNotNull(consumingSegmentInstanceStateMap); |
| 381 | + assertEquals(new HashSet<>(consumingSegmentInstanceStateMap.values()), |
| 382 | + Collections.singleton(SegmentStateModel.CONSUMING)); |
| 383 | + |
| 384 | + // Verify segment ZK metadata for committed segment and new consuming segment |
| 385 | + SegmentZKMetadata committedSegmentZKMetadata = segmentManager._segmentZKMetadataMap.get(committingSegment); |
| 386 | + assertEquals(committedSegmentZKMetadata.getStatus(), Status.DONE); |
| 387 | + assertEquals(committedSegmentZKMetadata.getStartOffset(), PARTITION_OFFSET.toString()); |
| 388 | + assertEquals(committedSegmentZKMetadata.getEndOffset(), endOffset); |
| 389 | + assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS); |
| 390 | + assertEquals(committedSegmentZKMetadata.getCrc(), Long.parseLong(CRC)); |
| 391 | + assertEquals(committedSegmentZKMetadata.getIndexVersion(), SEGMENT_VERSION.name()); |
| 392 | + assertEquals(committedSegmentZKMetadata.getTotalDocs(), NUM_DOCS); |
| 393 | + assertEquals(committedSegmentZKMetadata.getSizeInBytes(), SEGMENT_SIZE_IN_BYTES); |
| 394 | + |
| 395 | + SegmentZKMetadata consumingSegmentZKMetadata = segmentManager._segmentZKMetadataMap.get(consumingSegment); |
| 396 | + assertEquals(consumingSegmentZKMetadata.getStatus(), Status.IN_PROGRESS); |
| 397 | + assertEquals(consumingSegmentZKMetadata.getStartOffset(), String.valueOf(LATEST_OFFSET)); |
| 398 | + assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS); |
| 399 | + } |
| 400 | + } |
| 401 | + |
| 402 | + @Test |
| 403 | + public void testCommitSegmentWithOffsetAutoResetOnTime() |
| 404 | + throws Exception { |
| 405 | + // Set up a new table with 2 replicas, 5 instances, 4 partition |
| 406 | + PinotHelixResourceManager mockHelixResourceManager = mock(PinotHelixResourceManager.class); |
| 407 | + FakePinotLLCRealtimeSegmentManager segmentManager = |
| 408 | + new FakePinotLLCRealtimeSegmentManager(mockHelixResourceManager); |
| 409 | + setUpNewTable(segmentManager, 2, 5, 4); |
| 410 | + Map<String, Map<String, String>> instanceStatesMap = segmentManager._idealState.getRecord().getMapFields(); |
| 411 | + Map<String, String> streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(segmentManager._tableConfig).get(0); |
| 412 | + streamConfigMap.put(StreamConfigProperties.ENABLE_OFFSET_AUTO_RESET, String.valueOf(true)); |
| 413 | + streamConfigMap.put(StreamConfigProperties.OFFSET_AUTO_RESET_TIMESEC_THRESHOLD_KEY, "1800"); |
| 414 | + segmentManager.makeTableConfig(streamConfigMap); |
| 415 | + |
| 416 | + StreamConsumerFactory mockConsumerFactory = mock(StreamConsumerFactory.class); |
| 417 | + StreamMetadataProvider mockMetadataProvider = mock(StreamMetadataProvider.class); |
| 418 | + when(mockConsumerFactory.createPartitionMetadataProvider(anyString(), anyInt())).thenReturn(mockMetadataProvider); |
| 419 | + when(mockConsumerFactory.createStreamMsgOffsetFactory()).thenReturn(new LongMsgOffsetFactory()); |
| 420 | + when(mockMetadataProvider.fetchStreamPartitionOffset(eq(OffsetCriteria.LARGEST_OFFSET_CRITERIA), |
| 421 | + anyLong())).thenReturn(new LongMsgOffset(LATEST_OFFSET)); |
| 422 | + when(mockMetadataProvider.getOffsetAtTimestamp(eq(0), anyLong(), anyLong())).thenReturn( |
| 423 | + new LongMsgOffset(PARTITION_OFFSET.getOffset() + 1L)); |
| 424 | + |
| 425 | + try (MockedStatic<StreamConsumerFactoryProvider> mockedStaticProvider = mockStatic( |
| 426 | + StreamConsumerFactoryProvider.class)) { |
| 427 | + |
| 428 | + mockedStaticProvider.when(() -> StreamConsumerFactoryProvider.create(segmentManager._streamConfigs.get(0))) |
| 429 | + .thenReturn(mockConsumerFactory); |
| 430 | + |
| 431 | + // Commit a segment for partition group 0 |
| 432 | + String committingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 0, CURRENT_TIME_MS).getSegmentName(); |
| 433 | + String endOffset = new LongMsgOffset(PARTITION_OFFSET.getOffset() + NUM_DOCS).toString(); |
| 434 | + CommittingSegmentDescriptor committingSegmentDescriptor = |
| 435 | + new CommittingSegmentDescriptor(committingSegment, endOffset, SEGMENT_SIZE_IN_BYTES); |
| 436 | + committingSegmentDescriptor.setSegmentMetadata(mockSegmentMetadata()); |
| 437 | + segmentManager.commitSegmentMetadata(REALTIME_TABLE_NAME, committingSegmentDescriptor); |
| 438 | + |
| 439 | + // Verify instance states for committed segment and new consuming segment |
| 440 | + Map<String, String> committedSegmentInstanceStateMap = instanceStatesMap.get(committingSegment); |
| 441 | + assertNotNull(committedSegmentInstanceStateMap); |
| 442 | + assertEquals(new HashSet<>(committedSegmentInstanceStateMap.values()), |
| 443 | + Collections.singleton(SegmentStateModel.ONLINE)); |
| 444 | + |
| 445 | + String consumingSegment = new LLCSegmentName(RAW_TABLE_NAME, 0, 1, CURRENT_TIME_MS).getSegmentName(); |
| 446 | + Map<String, String> consumingSegmentInstanceStateMap = instanceStatesMap.get(consumingSegment); |
| 447 | + assertNotNull(consumingSegmentInstanceStateMap); |
| 448 | + assertEquals(new HashSet<>(consumingSegmentInstanceStateMap.values()), |
| 449 | + Collections.singleton(SegmentStateModel.CONSUMING)); |
| 450 | + |
| 451 | + // Verify segment ZK metadata for committed segment and new consuming segment |
| 452 | + SegmentZKMetadata committedSegmentZKMetadata = segmentManager._segmentZKMetadataMap.get(committingSegment); |
| 453 | + assertEquals(committedSegmentZKMetadata.getStatus(), Status.DONE); |
| 454 | + assertEquals(committedSegmentZKMetadata.getStartOffset(), PARTITION_OFFSET.toString()); |
| 455 | + assertEquals(committedSegmentZKMetadata.getEndOffset(), endOffset); |
| 456 | + assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS); |
| 457 | + assertEquals(committedSegmentZKMetadata.getCrc(), Long.parseLong(CRC)); |
| 458 | + assertEquals(committedSegmentZKMetadata.getIndexVersion(), SEGMENT_VERSION.name()); |
| 459 | + assertEquals(committedSegmentZKMetadata.getTotalDocs(), NUM_DOCS); |
| 460 | + assertEquals(committedSegmentZKMetadata.getSizeInBytes(), SEGMENT_SIZE_IN_BYTES); |
| 461 | + |
| 462 | + SegmentZKMetadata consumingSegmentZKMetadata = segmentManager._segmentZKMetadataMap.get(consumingSegment); |
| 463 | + assertEquals(consumingSegmentZKMetadata.getStatus(), Status.IN_PROGRESS); |
| 464 | + assertEquals(consumingSegmentZKMetadata.getStartOffset(), String.valueOf(LATEST_OFFSET)); |
| 465 | + assertEquals(committedSegmentZKMetadata.getCreationTime(), CURRENT_TIME_MS); |
| 466 | + } |
| 467 | + } |
| 468 | + |
328 | 469 | /** |
329 | 470 | * Test cases for the scenario where stream partitions increase, and the validation manager is attempting to create |
330 | 471 | * segments for new partitions. This test assumes that all other factors remain the same (no error conditions or |
@@ -1725,6 +1866,13 @@ void makeTableConfig() { |
1725 | 1866 | _streamConfigs = IngestionConfigUtils.getStreamConfigs(_tableConfig); |
1726 | 1867 | } |
1727 | 1868 |
|
| 1869 | + void makeTableConfig(Map<String, String> streamConfigMap) { |
| 1870 | + _tableConfig = |
| 1871 | + new TableConfigBuilder(TableType.REALTIME).setTableName(RAW_TABLE_NAME).setNumReplicas(_numReplicas) |
| 1872 | + .setStreamConfigs(streamConfigMap).build(); |
| 1873 | + _streamConfigs = IngestionConfigUtils.getStreamConfigs(_tableConfig); |
| 1874 | + } |
| 1875 | + |
1728 | 1876 | void makeConsumingInstancePartitions() { |
1729 | 1877 | List<String> instances = new ArrayList<>(_numInstances); |
1730 | 1878 | for (int i = 0; i < _numInstances; i++) { |
|
0 commit comments