mirror of https://github.com/apache/kafka.git
				
				
				
			MINOR: Add additional tests for RemoteLogManager (#14311)
Reviewers: Divij Vaidya <diviv@amazon.com>
This commit is contained in:
		
							parent
							
								
									fcd382138e
								
							
						
					
					
						commit
						47a1a7c70e
					
				|  | @ -1314,7 +1314,8 @@ public class RemoteLogManager implements Closeable { | ||||||
|         } |         } | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     private int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { |     // Visible for testing | ||||||
|  |     int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { | ||||||
|         return indexCache.lookupOffset(remoteLogSegmentMetadata, offset); |         return indexCache.lookupOffset(remoteLogSegmentMetadata, offset); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|  | @ -1398,7 +1399,8 @@ public class RemoteLogManager implements Closeable { | ||||||
|                 : Optional.empty(); |                 : Optional.empty(); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     private RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long offset) throws IOException { |     // Visible for testing | ||||||
|  |     RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long offset) throws IOException { | ||||||
|         RecordBatch nextBatch; |         RecordBatch nextBatch; | ||||||
|         // Look for the batch which has the desired offset |         // Look for the batch which has the desired offset | ||||||
|         // We will always have a batch in that segment as it is a non-compacted topic. |         // We will always have a batch in that segment as it is a non-compacted topic. | ||||||
|  |  | ||||||
|  | @ -34,7 +34,10 @@ import org.apache.kafka.common.network.ListenerName; | ||||||
| import org.apache.kafka.common.record.CompressionType; | import org.apache.kafka.common.record.CompressionType; | ||||||
| import org.apache.kafka.common.record.FileRecords; | import org.apache.kafka.common.record.FileRecords; | ||||||
| import org.apache.kafka.common.record.MemoryRecords; | import org.apache.kafka.common.record.MemoryRecords; | ||||||
|  | import org.apache.kafka.common.record.RecordBatch; | ||||||
|  | import org.apache.kafka.common.record.RemoteLogInputStream; | ||||||
| import org.apache.kafka.common.record.SimpleRecord; | import org.apache.kafka.common.record.SimpleRecord; | ||||||
|  | import org.apache.kafka.common.requests.FetchRequest; | ||||||
| import org.apache.kafka.common.security.auth.SecurityProtocol; | import org.apache.kafka.common.security.auth.SecurityProtocol; | ||||||
| import org.apache.kafka.common.utils.MockTime; | import org.apache.kafka.common.utils.MockTime; | ||||||
| import org.apache.kafka.common.utils.Time; | import org.apache.kafka.common.utils.Time; | ||||||
|  | @ -58,15 +61,20 @@ import org.apache.kafka.storage.internals.checkpoint.InMemoryLeaderEpochCheckpoi | ||||||
| import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint; | import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint; | ||||||
| import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; | import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; | ||||||
| import org.apache.kafka.storage.internals.log.EpochEntry; | import org.apache.kafka.storage.internals.log.EpochEntry; | ||||||
|  | import org.apache.kafka.storage.internals.log.FetchDataInfo; | ||||||
|  | import org.apache.kafka.storage.internals.log.FetchIsolation; | ||||||
| import org.apache.kafka.storage.internals.log.LazyIndex; | import org.apache.kafka.storage.internals.log.LazyIndex; | ||||||
| import org.apache.kafka.storage.internals.log.LogConfig; | import org.apache.kafka.storage.internals.log.LogConfig; | ||||||
| import org.apache.kafka.storage.internals.log.OffsetIndex; | import org.apache.kafka.storage.internals.log.OffsetIndex; | ||||||
| import org.apache.kafka.storage.internals.log.ProducerStateManager; | import org.apache.kafka.storage.internals.log.ProducerStateManager; | ||||||
|  | import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; | ||||||
| import org.apache.kafka.storage.internals.log.TimeIndex; | import org.apache.kafka.storage.internals.log.TimeIndex; | ||||||
| import org.apache.kafka.storage.internals.log.TransactionIndex; | import org.apache.kafka.storage.internals.log.TransactionIndex; | ||||||
| import org.apache.kafka.test.TestUtils; | import org.apache.kafka.test.TestUtils; | ||||||
| import org.junit.jupiter.api.BeforeEach; | import org.junit.jupiter.api.BeforeEach; | ||||||
| import org.junit.jupiter.api.Test; | import org.junit.jupiter.api.Test; | ||||||
|  | import org.junit.jupiter.params.ParameterizedTest; | ||||||
|  | import org.junit.jupiter.params.provider.ValueSource; | ||||||
| import org.mockito.ArgumentCaptor; | import org.mockito.ArgumentCaptor; | ||||||
| import org.mockito.InOrder; | import org.mockito.InOrder; | ||||||
| import org.mockito.MockedConstruction; | import org.mockito.MockedConstruction; | ||||||
|  | @ -76,7 +84,9 @@ import scala.collection.JavaConverters; | ||||||
| 
 | 
 | ||||||
| import java.io.ByteArrayInputStream; | import java.io.ByteArrayInputStream; | ||||||
| import java.io.File; | import java.io.File; | ||||||
|  | import java.io.FileInputStream; | ||||||
| import java.io.IOException; | import java.io.IOException; | ||||||
|  | import java.nio.ByteBuffer; | ||||||
| import java.nio.file.Files; | import java.nio.file.Files; | ||||||
| import java.util.ArrayList; | import java.util.ArrayList; | ||||||
| import java.util.Arrays; | import java.util.Arrays; | ||||||
|  | @ -89,6 +99,7 @@ import java.util.List; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
| import java.util.NavigableMap; | import java.util.NavigableMap; | ||||||
| import java.util.Optional; | import java.util.Optional; | ||||||
|  | import java.util.OptionalInt; | ||||||
| import java.util.Properties; | import java.util.Properties; | ||||||
| import java.util.Set; | import java.util.Set; | ||||||
| import java.util.TreeMap; | import java.util.TreeMap; | ||||||
|  | @ -121,6 +132,7 @@ import static org.mockito.ArgumentMatchers.anyLong; | ||||||
| import static org.mockito.ArgumentMatchers.anyString; | import static org.mockito.ArgumentMatchers.anyString; | ||||||
| import static org.mockito.ArgumentMatchers.eq; | import static org.mockito.ArgumentMatchers.eq; | ||||||
| import static org.mockito.Mockito.doAnswer; | import static org.mockito.Mockito.doAnswer; | ||||||
|  | import static org.mockito.Mockito.doNothing; | ||||||
| import static org.mockito.Mockito.doThrow; | import static org.mockito.Mockito.doThrow; | ||||||
| import static org.mockito.Mockito.inOrder; | import static org.mockito.Mockito.inOrder; | ||||||
| import static org.mockito.Mockito.mock; | import static org.mockito.Mockito.mock; | ||||||
|  | @ -1731,6 +1743,146 @@ public class RemoteLogManagerTest { | ||||||
|         return myCheckpoint.read().stream().collect(Collectors.toMap(e -> e.epoch, e -> e.startOffset)); |         return myCheckpoint.read().stream().collect(Collectors.toMap(e -> e.epoch, e -> e.startOffset)); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|  |     @Test | ||||||
|  |     public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException, IOException { | ||||||
|  |         FileInputStream fileInputStream = mock(FileInputStream.class); | ||||||
|  |         ClassLoaderAwareRemoteStorageManager rsmManager = mock(ClassLoaderAwareRemoteStorageManager.class); | ||||||
|  |         RemoteLogSegmentMetadata segmentMetadata = mock(RemoteLogSegmentMetadata.class); | ||||||
|  |         LeaderEpochFileCache cache = mock(LeaderEpochFileCache.class); | ||||||
|  |         when(cache.epochForOffset(anyLong())).thenReturn(OptionalInt.of(1)); | ||||||
|  | 
 | ||||||
|  |         when(remoteStorageManager.fetchLogSegment(any(RemoteLogSegmentMetadata.class), anyInt())) | ||||||
|  |                 .thenAnswer(a -> fileInputStream); | ||||||
|  |         when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); | ||||||
|  | 
 | ||||||
|  |         int fetchOffset = 0; | ||||||
|  |         int fetchMaxBytes = 10; | ||||||
|  | 
 | ||||||
|  |         FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData( | ||||||
|  |                 Uuid.randomUuid(), fetchOffset, 0, fetchMaxBytes, Optional.empty() | ||||||
|  |         ); | ||||||
|  | 
 | ||||||
|  |         RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo( | ||||||
|  |                 0, false, tp, partitionData, FetchIsolation.TXN_COMMITTED, false | ||||||
|  |         ); | ||||||
|  | 
 | ||||||
|  |         try (RemoteLogManager remoteLogManager = new RemoteLogManager( | ||||||
|  |                 remoteLogManagerConfig, | ||||||
|  |                 brokerId, | ||||||
|  |                 logDir, | ||||||
|  |                 clusterId, | ||||||
|  |                 time, | ||||||
|  |                 tp -> Optional.of(mockLog), | ||||||
|  |                 (topicPartition, offset) -> { }, | ||||||
|  |                 brokerTopicStats) { | ||||||
|  |             public RemoteStorageManager createRemoteStorageManager() { | ||||||
|  |                 return rsmManager; | ||||||
|  |             } | ||||||
|  |             public RemoteLogMetadataManager createRemoteLogMetadataManager() { | ||||||
|  |                 return remoteLogMetadataManager; | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             public Optional<RemoteLogSegmentMetadata> fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, | ||||||
|  |                                                                                     int epochForOffset, long offset) { | ||||||
|  |                 return Optional.of(segmentMetadata); | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { | ||||||
|  |                 return 1; | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             // This is the key scenario that we are testing here | ||||||
|  |             RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long offset) throws IOException { | ||||||
|  |                 return null; | ||||||
|  |             } | ||||||
|  |         }) { | ||||||
|  |             FetchDataInfo fetchDataInfo = remoteLogManager.read(fetchInfo); | ||||||
|  |             assertEquals(fetchOffset, fetchDataInfo.fetchOffsetMetadata.messageOffset); | ||||||
|  |             assertFalse(fetchDataInfo.firstEntryIncomplete); | ||||||
|  |             assertEquals(MemoryRecords.EMPTY, fetchDataInfo.records); | ||||||
|  |             // FetchIsolation is TXN_COMMITTED | ||||||
|  |             assertTrue(fetchDataInfo.abortedTransactions.isPresent()); | ||||||
|  |             assertTrue(fetchDataInfo.abortedTransactions.get().isEmpty()); | ||||||
|  |         } | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|  |     @ParameterizedTest | ||||||
|  |     @ValueSource(booleans = {true, false}) | ||||||
|  |     public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) throws RemoteStorageException, IOException { | ||||||
|  |         FileInputStream fileInputStream = mock(FileInputStream.class); | ||||||
|  |         ClassLoaderAwareRemoteStorageManager rsmManager = mock(ClassLoaderAwareRemoteStorageManager.class); | ||||||
|  |         RemoteLogSegmentMetadata segmentMetadata = mock(RemoteLogSegmentMetadata.class); | ||||||
|  |         LeaderEpochFileCache cache = mock(LeaderEpochFileCache.class); | ||||||
|  |         when(cache.epochForOffset(anyLong())).thenReturn(OptionalInt.of(1)); | ||||||
|  | 
 | ||||||
|  |         when(remoteStorageManager.fetchLogSegment(any(RemoteLogSegmentMetadata.class), anyInt())) | ||||||
|  |                 .thenAnswer(a -> fileInputStream); | ||||||
|  |         when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); | ||||||
|  | 
 | ||||||
|  |         int fetchOffset = 0; | ||||||
|  |         int fetchMaxBytes = 10; | ||||||
|  |         int recordBatchSizeInBytes = fetchMaxBytes + 1; | ||||||
|  |         RecordBatch firstBatch = mock(RecordBatch.class); | ||||||
|  |         ArgumentCaptor<ByteBuffer> capture = ArgumentCaptor.forClass(ByteBuffer.class); | ||||||
|  | 
 | ||||||
|  |         FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData( | ||||||
|  |                 Uuid.randomUuid(), fetchOffset, 0, fetchMaxBytes, Optional.empty() | ||||||
|  |         ); | ||||||
|  | 
 | ||||||
|  |         RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo( | ||||||
|  |                 0, minOneMessage, tp, partitionData, FetchIsolation.HIGH_WATERMARK, false | ||||||
|  |         ); | ||||||
|  | 
 | ||||||
|  |         try (RemoteLogManager remoteLogManager = new RemoteLogManager( | ||||||
|  |                 remoteLogManagerConfig, | ||||||
|  |                 brokerId, | ||||||
|  |                 logDir, | ||||||
|  |                 clusterId, | ||||||
|  |                 time, | ||||||
|  |                 tp -> Optional.of(mockLog), | ||||||
|  |                 (topicPartition, offset) -> { }, | ||||||
|  |                 brokerTopicStats) { | ||||||
|  |             public RemoteStorageManager createRemoteStorageManager() { | ||||||
|  |                 return rsmManager; | ||||||
|  |             } | ||||||
|  |             public RemoteLogMetadataManager createRemoteLogMetadataManager() { | ||||||
|  |                 return remoteLogMetadataManager; | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             public Optional<RemoteLogSegmentMetadata> fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, | ||||||
|  |                                                                                     int epochForOffset, long offset) { | ||||||
|  |                 return Optional.of(segmentMetadata); | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { | ||||||
|  |                 return 1; | ||||||
|  |             } | ||||||
|  | 
 | ||||||
|  |             RecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long offset) throws IOException { | ||||||
|  |                 when(firstBatch.sizeInBytes()).thenReturn(recordBatchSizeInBytes); | ||||||
|  |                 doNothing().when(firstBatch).writeTo(capture.capture()); | ||||||
|  |                 return firstBatch; | ||||||
|  |             } | ||||||
|  |         }) { | ||||||
|  |             FetchDataInfo fetchDataInfo = remoteLogManager.read(fetchInfo); | ||||||
|  |             // Common assertions | ||||||
|  |             assertEquals(fetchOffset, fetchDataInfo.fetchOffsetMetadata.messageOffset); | ||||||
|  |             assertFalse(fetchDataInfo.firstEntryIncomplete); | ||||||
|  |             // FetchIsolation is HIGH_WATERMARK | ||||||
|  |             assertEquals(Optional.empty(), fetchDataInfo.abortedTransactions); | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  |             if (minOneMessage) { | ||||||
|  |                 // Verify that the byte buffer has capacity equal to the size of the first batch | ||||||
|  |                 assertEquals(recordBatchSizeInBytes, capture.getValue().capacity()); | ||||||
|  |             } else { | ||||||
|  |                 // Verify that the first batch is never written to the buffer | ||||||
|  |                 verify(firstBatch, never()).writeTo(any(ByteBuffer.class)); | ||||||
|  |                 assertEquals(MemoryRecords.EMPTY, fetchDataInfo.records); | ||||||
|  |             } | ||||||
|  |         } | ||||||
|  |     } | ||||||
|  | 
 | ||||||
|     private Partition mockPartition(TopicIdPartition topicIdPartition) { |     private Partition mockPartition(TopicIdPartition topicIdPartition) { | ||||||
|         TopicPartition tp = topicIdPartition.topicPartition(); |         TopicPartition tp = topicIdPartition.topicPartition(); | ||||||
|         Partition partition = mock(Partition.class); |         Partition partition = mock(Partition.class); | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue