@ -31,8 +31,12 @@ import org.apache.kafka.streams.StreamsConfig;
@@ -31,8 +31,12 @@ import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.LockException ;
import org.apache.kafka.streams.errors.ProcessorStateException ;
import org.apache.kafka.streams.errors.StreamsException ;
import org.apache.kafka.streams.processor.ProcessorContext ;
import org.apache.kafka.streams.processor.StateRestoreCallback ;
import org.apache.kafka.streams.processor.StateStore ;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint ;
import org.apache.kafka.streams.state.RecordConverter ;
import org.apache.kafka.streams.state.internals.WrappedStateStore ;
import org.apache.kafka.test.InternalMockProcessorContext ;
import org.apache.kafka.test.MockStateRestoreListener ;
import org.apache.kafka.test.NoOpReadOnlyStore ;
@ -90,7 +94,7 @@ public class GlobalStateManagerImplTest {
@@ -90,7 +94,7 @@ public class GlobalStateManagerImplTest {
private InternalMockProcessorContext processorContext ;
@Before
public void before ( ) throws IOException {
public void before ( ) {
final Map < String , String > storeToTopic = new HashMap < > ( ) ;
storeToTopic . put ( storeName1 , t1 . topic ( ) ) ;
@ -99,7 +103,7 @@ public class GlobalStateManagerImplTest {
@@ -99,7 +103,7 @@ public class GlobalStateManagerImplTest {
storeToTopic . put ( storeName4 , t4 . topic ( ) ) ;
store1 = new NoOpReadOnlyStore < > ( storeName1 , true ) ;
store2 = new NoOpReadOnly Store< > ( storeName2 , true ) ;
store2 = new Converter Store< > ( storeName2 , true ) ;
store3 = new NoOpReadOnlyStore < > ( storeName3 ) ;
store4 = new NoOpReadOnlyStore < > ( storeName4 ) ;
@ -198,7 +202,7 @@ public class GlobalStateManagerImplTest {
@@ -198,7 +202,7 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldThrowIllegalArgumentExceptionIfAttemptingToRegisterStoreTwice ( ) {
stateManager . initialize ( ) ;
initializeConsumer ( 2 , 1 , t1 ) ;
initializeConsumer ( 2 , 0 , t1 ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
try {
stateManager . register ( store1 , stateRestoreCallback ) ;
@ -219,9 +223,138 @@ public class GlobalStateManagerImplTest {
@@ -219,9 +223,138 @@ public class GlobalStateManagerImplTest {
}
}
@Test
public void shouldUseDefaultRecordConverterIfStoreDoesNotImplementRecordConverter ( ) {
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . initialize ( ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
final KeyValue < byte [ ] , byte [ ] > restoredRecord = stateRestoreCallback . restored . get ( 0 ) ;
assertEquals ( 3 , restoredRecord . key . length ) ;
assertEquals ( 5 , restoredRecord . value . length ) ;
}
@Test
public void shouldUseDefaultRecordConverterIfInnerStoreDoesNotImplementRecordConverter ( ) {
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . initialize ( ) ;
stateManager . register ( new WrappedStateStore ( ) {
@Override
public StateStore inner ( ) {
return store1 ;
}
@Override
public StateStore wrappedStore ( ) {
return store1 ;
}
@Override
public String name ( ) {
return store1 . name ( ) ;
}
@Override
public void init ( final ProcessorContext context , final StateStore root ) {
store1 . init ( context , root ) ;
}
@Override
public void flush ( ) {
store1 . flush ( ) ;
}
@Override
public void close ( ) {
store1 . close ( ) ;
}
@Override
public boolean persistent ( ) {
return store1 . persistent ( ) ;
}
@Override
public boolean isOpen ( ) {
return store1 . isOpen ( ) ;
}
} , stateRestoreCallback ) ;
final KeyValue < byte [ ] , byte [ ] > restoredRecord = stateRestoreCallback . restored . get ( 0 ) ;
assertEquals ( 3 , restoredRecord . key . length ) ;
assertEquals ( 5 , restoredRecord . value . length ) ;
}
@Test
public void shouldUseStoreAsRecordConverterIfStoreImplementsRecordConverter ( ) {
initializeConsumer ( 1 , 0 , t2 ) ;
stateManager . initialize ( ) ;
stateManager . register ( store2 , stateRestoreCallback ) ;
final KeyValue < byte [ ] , byte [ ] > restoredRecord = stateRestoreCallback . restored . get ( 0 ) ;
assertEquals ( 0 , restoredRecord . key . length ) ;
assertEquals ( 0 , restoredRecord . value . length ) ;
}
@Test
public void shouldUseStoreAsRecordConverterIfInnerStoreImplementsRecordConverter ( ) {
initializeConsumer ( 1 , 0 , t2 ) ;
stateManager . initialize ( ) ;
stateManager . register ( new WrappedStateStore ( ) {
@Override
public StateStore inner ( ) {
return store2 ;
}
@Override
public StateStore wrappedStore ( ) {
return store2 ;
}
@Override
public String name ( ) {
return store2 . name ( ) ;
}
@Override
public void init ( final ProcessorContext context , final StateStore root ) {
store2 . init ( context , root ) ;
}
@Override
public void flush ( ) {
store2 . flush ( ) ;
}
@Override
public void close ( ) {
store2 . close ( ) ;
}
@Override
public boolean persistent ( ) {
return store2 . persistent ( ) ;
}
@Override
public boolean isOpen ( ) {
return store2 . isOpen ( ) ;
}
} , stateRestoreCallback ) ;
final KeyValue < byte [ ] , byte [ ] > restoredRecord = stateRestoreCallback . restored . get ( 0 ) ;
assertEquals ( 0 , restoredRecord . key . length ) ;
assertEquals ( 0 , restoredRecord . value . length ) ;
}
@Test
public void shouldRestoreRecordsUpToHighwatermark ( ) {
initializeConsumer ( 2 , 1 , t1 ) ;
initializeConsumer ( 2 , 0 , t1 ) ;
stateManager . initialize ( ) ;
@ -231,7 +364,7 @@ public class GlobalStateManagerImplTest {
@@ -231,7 +364,7 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldRecoverFromInvalidOffsetExceptionAndRestoreRecords ( ) {
initializeConsumer ( 2 , 1 , t1 ) ;
initializeConsumer ( 2 , 0 , t1 ) ;
consumer . setException ( new InvalidOffsetException ( "Try Again!" ) {
public Set < TopicPartition > partitions ( ) {
return Collections . singleton ( t1 ) ;
@ -252,7 +385,7 @@ public class GlobalStateManagerImplTest {
@@ -252,7 +385,7 @@ public class GlobalStateManagerImplTest {
stateManager . register ( store1 , stateRestoreCallback ) ;
assertThat ( stateRestoreListener . restoreStartOffset , equalTo ( 1L ) ) ;
assertThat ( stateRestoreListener . restoreEndOffset , equalTo ( 5 L) ) ;
assertThat ( stateRestoreListener . restoreEndOffset , equalTo ( 6 L) ) ;
assertThat ( stateRestoreListener . totalNumRestored , equalTo ( 5L ) ) ;
@ -263,11 +396,11 @@ public class GlobalStateManagerImplTest {
@@ -263,11 +396,11 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldRestoreRecordsFromCheckpointToHighwatermark ( ) throws IOException {
initializeConsumer ( 5 , 6 , t1 ) ;
initializeConsumer ( 5 , 5 , t1 ) ;
final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint ( new File ( stateManager . baseDir ( ) ,
ProcessorStateManager . CHECKPOINT_FILE_NAME ) ) ;
offsetCheckpoint . write ( Collections . singletonMap ( t1 , 6 L) ) ;
offsetCheckpoint . write ( Collections . singletonMap ( t1 , 5 L) ) ;
stateManager . initialize ( ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
@ -279,9 +412,9 @@ public class GlobalStateManagerImplTest {
@@ -279,9 +412,9 @@ public class GlobalStateManagerImplTest {
public void shouldFlushStateStores ( ) {
stateManager . initialize ( ) ;
// register the stores
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
initializeConsumer ( 1 , 1 , t2 ) ;
initializeConsumer ( 1 , 0 , t2 ) ;
stateManager . register ( store2 , stateRestoreCallback ) ;
stateManager . flush ( ) ;
@ -293,7 +426,7 @@ public class GlobalStateManagerImplTest {
@@ -293,7 +426,7 @@ public class GlobalStateManagerImplTest {
public void shouldThrowProcessorStateStoreExceptionIfStoreFlushFailed ( ) {
stateManager . initialize ( ) ;
// register the stores
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . register ( new NoOpReadOnlyStore ( store1 . name ( ) ) {
@Override
public void flush ( ) {
@ -308,12 +441,12 @@ public class GlobalStateManagerImplTest {
@@ -308,12 +441,12 @@ public class GlobalStateManagerImplTest {
public void shouldCloseStateStores ( ) throws IOException {
stateManager . initialize ( ) ;
// register the stores
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
initializeConsumer ( 1 , 1 , t2 ) ;
initializeConsumer ( 1 , 0 , t2 ) ;
stateManager . register ( store2 , stateRestoreCallback ) ;
stateManager . close ( Collections . < TopicPartition , Long > emptyMap ( ) ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
assertFalse ( store1 . isOpen ( ) ) ;
assertFalse ( store2 . isOpen ( ) ) ;
}
@ -321,7 +454,7 @@ public class GlobalStateManagerImplTest {
@@ -321,7 +454,7 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldWriteCheckpointsOnClose ( ) throws IOException {
stateManager . initialize ( ) ;
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
final Map < TopicPartition , Long > expected = Collections . singletonMap ( t1 , 25L ) ;
stateManager . close ( expected ) ;
@ -332,7 +465,7 @@ public class GlobalStateManagerImplTest {
@@ -332,7 +465,7 @@ public class GlobalStateManagerImplTest {
@Test ( expected = ProcessorStateException . class )
public void shouldThrowProcessorStateStoreExceptionIfStoreCloseFailed ( ) throws IOException {
stateManager . initialize ( ) ;
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . register ( new NoOpReadOnlyStore ( store1 . name ( ) ) {
@Override
public void close ( ) {
@ -340,7 +473,7 @@ public class GlobalStateManagerImplTest {
@@ -340,7 +473,7 @@ public class GlobalStateManagerImplTest {
}
} , stateRestoreCallback ) ;
stateManager . close ( Collections . < TopicPartition , Long > emptyMap ( ) ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
}
@Test
@ -370,7 +503,7 @@ public class GlobalStateManagerImplTest {
@@ -370,7 +503,7 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldNotCloseStoresIfCloseAlreadyCalled ( ) throws IOException {
stateManager . initialize ( ) ;
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
stateManager . register ( new NoOpReadOnlyStore ( "t1-store" ) {
@Override
public void close ( ) {
@ -380,17 +513,15 @@ public class GlobalStateManagerImplTest {
@@ -380,17 +513,15 @@ public class GlobalStateManagerImplTest {
super . close ( ) ;
}
} , stateRestoreCallback ) ;
stateManager . close ( Collections . < TopicPartition , Long > emptyMap ( ) ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
stateManager . close ( Collections . < TopicPartition , Long > emptyMap ( ) ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
}
@Test
public void shouldAttemptToCloseAllStoresEvenWhenSomeException ( ) throws IOException {
stateManager . initialize ( ) ;
initializeConsumer ( 1 , 1 , t1 ) ;
initializeConsumer ( 1 , 1 , t2 ) ;
initializeConsumer ( 1 , 0 , t1 ) ;
final NoOpReadOnlyStore store = new NoOpReadOnlyStore ( "t1-store" ) {
@Override
public void close ( ) {
@ -400,10 +531,11 @@ public class GlobalStateManagerImplTest {
@@ -400,10 +531,11 @@ public class GlobalStateManagerImplTest {
} ;
stateManager . register ( store , stateRestoreCallback ) ;
initializeConsumer ( 1 , 0 , t2 ) ;
stateManager . register ( store2 , stateRestoreCallback ) ;
try {
stateManager . close ( Collections . < TopicPartition , Long > emptyMap ( ) ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
} catch ( final ProcessorStateException e ) {
// expected
}
@ -443,9 +575,9 @@ public class GlobalStateManagerImplTest {
@@ -443,9 +575,9 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldNotRemoveOffsetsOfUnUpdatedTablesDuringCheckpoint ( ) {
stateManager . initialize ( ) ;
initializeConsumer ( 10 , 1 , t1 ) ;
initializeConsumer ( 10 , 0 , t1 ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
initializeConsumer ( 20 , 1 , t2 ) ;
initializeConsumer ( 20 , 0 , t2 ) ;
stateManager . register ( store2 , stateRestoreCallback ) ;
final Map < TopicPartition , Long > initialCheckpoint = stateManager . checkpointed ( ) ;
@ -461,12 +593,12 @@ public class GlobalStateManagerImplTest {
@@ -461,12 +593,12 @@ public class GlobalStateManagerImplTest {
final HashMap < TopicPartition , Long > startOffsets = new HashMap < > ( ) ;
startOffsets . put ( t1 , 1L ) ;
final HashMap < TopicPartition , Long > endOffsets = new HashMap < > ( ) ;
endOffsets . put ( t1 , 2 L) ;
endOffsets . put ( t1 , 3 L) ;
consumer . updatePartitions ( t1 . topic ( ) , Collections . singletonList ( new PartitionInfo ( t1 . topic ( ) , t1 . partition ( ) , null , null , null ) ) ) ;
consumer . assign ( Collections . singletonList ( t1 ) ) ;
consumer . updateEndOffsets ( endOffsets ) ;
consumer . updateBeginningOffsets ( startOffsets ) ;
consumer . addRecord ( new ConsumerRecord < > ( t1 . topic ( ) , t1 . partition ( ) , 1 , ( byte [ ] ) null , "null" . getBytes ( ) ) ) ;
consumer . addRecord ( new ConsumerRecord < > ( t1 . topic ( ) , t1 . partition ( ) , 1 , null , "null" . getBytes ( ) ) ) ;
final byte [ ] expectedKey = "key" . getBytes ( ) ;
final byte [ ] expectedValue = "value" . getBytes ( ) ;
consumer . addRecord ( new ConsumerRecord < > ( t1 . topic ( ) , t1 . partition ( ) , 2 , expectedKey , expectedValue ) ) ;
@ -480,19 +612,19 @@ public class GlobalStateManagerImplTest {
@@ -480,19 +612,19 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldCheckpointRestoredOffsetsToFile ( ) throws IOException {
stateManager . initialize ( ) ;
initializeConsumer ( 10 , 1 , t1 ) ;
initializeConsumer ( 10 , 0 , t1 ) ;
stateManager . register ( store1 , stateRestoreCallback ) ;
stateManager . close ( Collections . < TopicPartition , Long > emptyMap ( ) ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
final Map < TopicPartition , Long > checkpointMap = stateManager . checkpointed ( ) ;
assertThat ( checkpointMap , equalTo ( Collections . singletonMap ( t1 , 11 L ) ) ) ;
assertThat ( checkpointMap , equalTo ( Collections . singletonMap ( t1 , 10 L ) ) ) ;
assertThat ( readOffsetsCheckpoint ( ) , equalTo ( checkpointMap ) ) ;
}
@Test
public void shouldSkipGlobalInMemoryStoreOffsetsToFile ( ) throws IOException {
stateManager . initialize ( ) ;
initializeConsumer ( 10 , 1 , t3 ) ;
initializeConsumer ( 10 , 0 , t3 ) ;
stateManager . register ( store3 , stateRestoreCallback ) ;
stateManager . close ( Collections . emptyMap ( ) ) ;
@ -666,9 +798,9 @@ public class GlobalStateManagerImplTest {
@@ -666,9 +798,9 @@ public class GlobalStateManagerImplTest {
private void initializeConsumer ( final long numRecords , final long startOffset , final TopicPartition topicPartition ) {
final HashMap < TopicPartition , Long > startOffsets = new HashMap < > ( ) ;
startOffsets . put ( topicPartition , 1L ) ;
startOffsets . put ( topicPartition , startOffset ) ;
final HashMap < TopicPartition , Long > endOffsets = new HashMap < > ( ) ;
endOffsets . put ( topicPartition , startOffset + numRecords - 1 ) ;
endOffsets . put ( topicPartition , startOffset + numRecords ) ;
consumer . updatePartitions ( topicPartition . topic ( ) , Collections . singletonList ( new PartitionInfo ( topicPartition . topic ( ) , topicPartition . partition ( ) , null , null , null ) ) ) ;
consumer . assign ( Collections . singletonList ( topicPartition ) ) ;
consumer . updateEndOffsets ( endOffsets ) ;
@ -694,4 +826,20 @@ public class GlobalStateManagerImplTest {
@@ -694,4 +826,20 @@ public class GlobalStateManagerImplTest {
restored . add ( KeyValue . pair ( key , value ) ) ;
}
}
private class ConverterStore < K , V > extends NoOpReadOnlyStore < K , V > implements RecordConverter {
ConverterStore ( final String name ,
final boolean rocksdbStore ) {
super ( name , rocksdbStore ) ;
}
@Override
public ConsumerRecord < byte [ ] , byte [ ] > convert ( final ConsumerRecord < byte [ ] , byte [ ] > record ) {
return new ConsumerRecord < > ( "" , 0 , 0L , "" . getBytes ( ) , "" . getBytes ( ) ) ;
}
}
}