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