Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-10562: Properly invoke new StateStoreContext init #9388

Merged
merged 7 commits into from
Oct 8, 2020
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
wip
  • Loading branch information
John Roesler committed Oct 6, 2020
commit f4448fd066d2839e8f15c29a7f91015023eb06f0
Original file line number Diff line number Diff line change
Expand Up @@ -225,6 +225,7 @@ public String name() {
return name;
}

@Deprecated
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are a handful of these also, just passing the deprecation on to the callers.

@Override
public void init(final ProcessorContext context,
final StateStore root) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ public String name() {
return name;
}

@Deprecated
@Override
public void init(final ProcessorContext context, final StateStore root) {
this.context = (InternalProcessorContext) context;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ public String name() {
return name;
}

@Deprecated
@Override
public void init(final ProcessorContext context, final StateStore root) {
this.context = context;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;

import java.io.File;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;

class KeyValueSegment extends RocksDBStore implements Comparable<KeyValueSegment>, Segment {
Expand All @@ -45,8 +47,8 @@ public int compareTo(final KeyValueSegment segment) {
}

@Override
public void openDB(final ProcessorContext context) {
super.openDB(context);
public void openDB(final Map<String, Object> configs, final File stateDir) {
super.openDB(configs, stateDir);
Comment on lines +49 to +50
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was able to remove the type-dependency of the context by re-specifying the interface in terms of the only two properties it needed.

// skip the registering step
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public KeyValueSegment getOrCreateSegment(final long segmentId,
throw new IllegalStateException("KeyValueSegment already exists. Possible concurrent access.");
}

newSegment.openDB(context);
newSegment.openDB(context.appConfigs(), context.stateDir());
return newSegment;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;

import java.io.File;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;

class TimestampedSegment extends RocksDBTimestampedStore implements Comparable<TimestampedSegment>, Segment {
Expand All @@ -45,10 +47,9 @@ public int compareTo(final TimestampedSegment segment) {
}

@Override
public void openDB(final ProcessorContext context) {
super.openDB(context);
public void openDB(final Map<String, Object> configs, final File stateDir) {
super.openDB(configs, stateDir);
// skip the registering step
internalProcessorContext = context;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public TimestampedSegment getOrCreateSegment(final long segmentId,
throw new IllegalStateException("TimestampedSegment already exists. Possible concurrent access.");
}

newSegment.openDB(context);
newSegment.openDB(context.appConfigs(), context.stateDir());
return newSegment;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.To;
import org.apache.kafka.streams.processor.internals.Task.TaskType;
import org.apache.kafka.streams.state.KeyValueStore;
Expand Down Expand Up @@ -147,7 +148,7 @@ public void shouldNotAllowToSchedulePunctuations() {
public void shouldNotAllowInitForKeyValueStore() {
final StateStore store = globalContext.getStateStore(GLOBAL_KEY_VALUE_STORE_NAME);
try {
store.init((ProcessorContext) null, null);
store.init((StateStoreContext) null, null);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was actually a bug before, which this PR fixes: the wrapping layers should transmit the init call straight down, rather than translating it. There are a whole set of new unit tests making sure that this works properly for both the new and old init methods.

fail("Should have thrown UnsupportedOperationException.");
} catch (final UnsupportedOperationException expected) { }
}
Expand All @@ -156,7 +157,7 @@ public void shouldNotAllowInitForKeyValueStore() {
public void shouldNotAllowInitForTimestampedKeyValueStore() {
final StateStore store = globalContext.getStateStore(GLOBAL_TIMESTAMPED_KEY_VALUE_STORE_NAME);
try {
store.init((ProcessorContext) null, null);
store.init((StateStoreContext) null, null);
fail("Should have thrown UnsupportedOperationException.");
} catch (final UnsupportedOperationException expected) { }
}
Expand All @@ -165,7 +166,7 @@ public void shouldNotAllowInitForTimestampedKeyValueStore() {
public void shouldNotAllowInitForWindowStore() {
final StateStore store = globalContext.getStateStore(GLOBAL_WINDOW_STORE_NAME);
try {
store.init((ProcessorContext) null, null);
store.init((StateStoreContext) null, null);
fail("Should have thrown UnsupportedOperationException.");
} catch (final UnsupportedOperationException expected) { }
}
Expand All @@ -174,7 +175,7 @@ public void shouldNotAllowInitForWindowStore() {
public void shouldNotAllowInitForTimestampedWindowStore() {
final StateStore store = globalContext.getStateStore(GLOBAL_TIMESTAMPED_WINDOW_STORE_NAME);
try {
store.init((ProcessorContext) null, null);
store.init((StateStoreContext) null, null);
fail("Should have thrown UnsupportedOperationException.");
} catch (final UnsupportedOperationException expected) { }
}
Expand All @@ -183,7 +184,7 @@ public void shouldNotAllowInitForTimestampedWindowStore() {
public void shouldNotAllowInitForSessionStore() {
final StateStore store = globalContext.getStateStore(GLOBAL_SESSION_STORE_NAME);
try {
store.init((ProcessorContext) null, null);
store.init((StateStoreContext) null, null);
fail("Should have thrown UnsupportedOperationException.");
} catch (final UnsupportedOperationException expected) { }
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.PunctuationType;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.To;
import org.apache.kafka.streams.processor.internals.Task.TaskType;
Expand Down Expand Up @@ -767,7 +768,7 @@ private void verifyStoreCannotBeInitializedOrClosed(final StateStore store) {
assertTrue(store.persistent());
assertTrue(store.isOpen());

checkThrowsUnsupportedOperation(() -> store.init((ProcessorContext) null, null), "init()");
checkThrowsUnsupportedOperation(() -> store.init((StateStoreContext) null, null), "init()");
checkThrowsUnsupportedOperation(store::close, "close()");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
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.processor.StateStoreContext;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
Expand Down Expand Up @@ -331,7 +332,7 @@ public void addEntryToRestoreLog(final K key, final V value) {
* @return the processing context; never null
* @see #addEntryToRestoreLog(Object, Object)
*/
public ProcessorContext context() {
public StateStoreContext context() {
return context;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ public String name() {
return "";
}

@Deprecated
@Override
public void init(final ProcessorContext context, final StateStore root) {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
Expand Down Expand Up @@ -51,7 +52,7 @@

public abstract class AbstractKeyValueStoreTest {

protected abstract <K, V> KeyValueStore<K, V> createKeyValueStore(final ProcessorContext context);
protected abstract <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context);

protected InternalMockProcessorContext context;
protected KeyValueStore<Integer, String> store;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ public void after() {

@SuppressWarnings("unchecked")
@Override
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final ProcessorContext context) {
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context) {
final StoreBuilder<KeyValueStore<K, V>> storeBuilder = Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore("cache-store"),
(Serde<K>) context.keySerde(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,15 +64,18 @@ private void init() {
store.init((StateStoreContext) context, store);
}

@Test void shouldDelegateDeprecatedInit() {
@SuppressWarnings("deprecation")
@Test
public void shouldDelegateDeprecatedInit() {
inner.init((ProcessorContext) context, store);
EasyMock.expectLastCall();
EasyMock.replay(inner);
store.init((ProcessorContext) context, store);
EasyMock.verify(inner);
}

@Test void shouldDelegateInit() {
@Test
public void shouldDelegateInit() {
inner.init((StateStoreContext) context, store);
EasyMock.expectLastCall();
EasyMock.replay(inner);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
Expand All @@ -28,7 +29,7 @@ public class InMemoryKeyValueLoggedStoreTest extends AbstractKeyValueStoreTest {

@SuppressWarnings("unchecked")
@Override
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final ProcessorContext context) {
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context) {
final StoreBuilder<KeyValueStore<K, V>> storeBuilder = Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore("my-store"),
(Serde<K>) context.keySerde(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
Expand All @@ -31,7 +32,7 @@ public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest {

@SuppressWarnings("unchecked")
@Override
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final ProcessorContext context) {
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context) {
final StoreBuilder<KeyValueStore<K, V>> storeBuilder = Stores.keyValueStoreBuilder(
Stores.inMemoryKeyValueStore("my-store"),
(Serde<K>) context.keySerde(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
Expand All @@ -36,7 +37,7 @@ public class InMemoryLRUCacheStoreTest extends AbstractKeyValueStoreTest {

@SuppressWarnings("unchecked")
@Override
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final ProcessorContext context) {
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context) {
final StoreBuilder<KeyValueStore<K, V>> storeBuilder = Stores.keyValueStoreBuilder(
Stores.lruMap("my-store", 10),
(Serde<K>) context.keySerde(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public void shouldDeleteStateDirectoryOnDestroy() throws Exception {
expect(mockContext.stateDir()).andReturn(directory);
replay(mockContext);

segment.openDB(mockContext);
segment.openDB(mockContext.appConfigs(), mockContext.stateDir());

assertTrue(new File(directoryPath, "window").exists());
assertTrue(new File(directoryPath + File.separator + "window", "segment").exists());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,6 +382,7 @@ public String name() {
return null;
}

@Deprecated
@Override
public void init(final ProcessorContext context, final StateStore root) {
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.RocksDBConfigSetter;
Expand All @@ -38,7 +39,7 @@ public class RocksDBKeyValueStoreTest extends AbstractKeyValueStoreTest {

@SuppressWarnings("unchecked")
@Override
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final ProcessorContext context) {
protected <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext context) {
final StoreBuilder<KeyValueStore<K, V>> storeBuilder = Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore("my-store"),
(Serde<K>) context.keySerde(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,7 @@ public void shouldAddValueProvidersWithoutStatisticsToInjectedMetricsRecorderWhe
metricsRecorder.addValueProviders(eq(DB_NAME), notNull(), notNull(), isNull());
replay(metricsRecorder);

rocksDBStore.openDB(context);
rocksDBStore.openDB(context.appConfigs(), context.stateDir());

verify(metricsRecorder);
reset(metricsRecorder);
Expand All @@ -171,7 +171,7 @@ public void shouldAddValueProvidersWithStatisticsToInjectedMetricsRecorderWhenRe
metricsRecorder.addValueProviders(eq(DB_NAME), notNull(), notNull(), notNull());
replay(metricsRecorder);

rocksDBStore.openDB(context);
rocksDBStore.openDB(context.appConfigs(), context.stateDir());

verify(metricsRecorder);
reset(metricsRecorder);
Expand All @@ -182,7 +182,7 @@ public void shouldRemoveValueProvidersFromInjectedMetricsRecorderOnClose() {
rocksDBStore = getRocksDBStoreWithRocksDBMetricsRecorder();
try {
context = getProcessorContext(RecordingLevel.DEBUG);
rocksDBStore.openDB(context);
rocksDBStore.openDB(context.appConfigs(), context.stateDir());
reset(metricsRecorder);
metricsRecorder.removeValueProviders(DB_NAME);
replay(metricsRecorder);
Expand Down Expand Up @@ -212,7 +212,7 @@ public void shouldNotSetStatisticsInValueProvidersWhenUserProvidesStatistics() {
metricsRecorder.addValueProviders(eq(DB_NAME), notNull(), notNull(), isNull());
replay(metricsRecorder);

rocksDBStore.openDB(context);
rocksDBStore.openDB(context.appConfigs(), context.stateDir());
verify(metricsRecorder);
reset(metricsRecorder);
}
Expand Down Expand Up @@ -242,7 +242,7 @@ public void shouldThrowWhenUserProvidesNewBlockBasedTableFormatConfig() {
"the block-based table format of RocksDB. Do not provide a new instance of BlockBasedTableConfig to " +
"the RocksDB options.",
ProcessorStateException.class,
() -> rocksDBStore.openDB(context)
() -> rocksDBStore.openDB(context.appConfigs(), context.stateDir())
);
}

Expand All @@ -268,7 +268,7 @@ public void shouldNotSetCacheInValueProvidersWhenUserProvidesPlainTableFormatCon
metricsRecorder.addValueProviders(eq(DB_NAME), notNull(), isNull(), notNull());
replay(metricsRecorder);

rocksDBStore.openDB(context);
rocksDBStore.openDB(context.appConfigs(), context.stateDir());
verify(metricsRecorder);
reset(metricsRecorder);
}
Expand Down Expand Up @@ -310,7 +310,7 @@ public void shouldThrowProcessorStateExceptionOnOpeningReadOnlyDir() {

assertTrue(tmpDir.setReadOnly());

assertThrows(ProcessorStateException.class, () -> rocksDBStore.openDB(tmpContext));
assertThrows(ProcessorStateException.class, () -> rocksDBStore.openDB(tmpContext.appConfigs(), tmpContext.stateDir()));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public void shouldDeleteStateDirectoryOnDestroy() throws Exception {
expect(mockContext.stateDir()).andReturn(directory);
replay(mockContext);

segment.openDB(mockContext);
segment.openDB(mockContext.appConfigs(), mockContext.stateDir());

assertTrue(new File(directoryPath, "window").exists());
assertTrue(new File(directoryPath + File.separator + "window", "segment").exists());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ public String name() {
return this.name;
}

@Deprecated
@Override
@SuppressWarnings("unchecked")
/* This is a "dummy" store used for testing;
Expand Down
Loading