MINOR: update CachingPersistentWindowStoreTest (#16701)

Refactor test to move off deprecated `transform()` in favor of
`process()`.

Reviewers: Bill Bejeck <bill@confluent.io>
This commit is contained in:
Matthias J. Sax 2024-07-29 12:45:13 -07:00 committed by GitHub
parent 2cf87bff9b
commit b6c1cb0eec
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
1 changed files with 18 additions and 23 deletions

View File

@ -32,11 +32,12 @@ import org.apache.kafka.streams.TopologyTestDriver;
import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.TimeWindowedDeserializer; import org.apache.kafka.streams.kstream.TimeWindowedDeserializer;
import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.kstream.internals.TimeWindow;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.api.Processor;
import org.apache.kafka.streams.processor.api.ProcessorContext;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.query.Position;
@ -102,7 +103,7 @@ public class CachingPersistentWindowStoreTest {
private static final String TOPIC = "topic"; private static final String TOPIC = "topic";
private static final String CACHE_NAMESPACE = "0_0-store-name"; private static final String CACHE_NAMESPACE = "0_0-store-name";
private InternalMockProcessorContext context; private InternalMockProcessorContext<?, ?> context;
private RocksDBSegmentedBytesStore bytesStore; private RocksDBSegmentedBytesStore bytesStore;
private WindowStore<Bytes, byte[]> underlyingStore; private WindowStore<Bytes, byte[]> underlyingStore;
private CachingWindowStore cachingStore; private CachingWindowStore cachingStore;
@ -138,8 +139,8 @@ public class CachingPersistentWindowStoreTest {
final WindowStore<Bytes, byte[]> inner = mock(WindowStore.class); final WindowStore<Bytes, byte[]> inner = mock(WindowStore.class);
final CachingWindowStore outer = new CachingWindowStore(inner, WINDOW_SIZE, SEGMENT_INTERVAL); final CachingWindowStore outer = new CachingWindowStore(inner, WINDOW_SIZE, SEGMENT_INTERVAL);
when(inner.name()).thenReturn("store"); when(inner.name()).thenReturn("store");
outer.init((ProcessorContext) context, outer); outer.init((org.apache.kafka.streams.processor.ProcessorContext) context, outer);
verify(inner).init((ProcessorContext) context, outer); verify(inner).init((org.apache.kafka.streams.processor.ProcessorContext) context, outer);
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -153,7 +154,6 @@ public class CachingPersistentWindowStoreTest {
} }
@Test @Test
@SuppressWarnings("deprecation")
public void shouldNotReturnDuplicatesInRanges() { public void shouldNotReturnDuplicatesInRanges() {
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
@ -167,16 +167,15 @@ public class CachingPersistentWindowStoreTest {
builder.stream(TOPIC, builder.stream(TOPIC,
Consumed.with(Serdes.String(), Serdes.String())) Consumed.with(Serdes.String(), Serdes.String()))
.transform(() -> new Transformer<String, String, KeyValue<String, String>>() { .process(() -> new Processor<String, String, String, String>() {
private WindowStore<String, String> store; private WindowStore<String, String> store;
private int numRecordsProcessed; private int numRecordsProcessed;
private ProcessorContext context; private ProcessorContext<String, String> context;
@SuppressWarnings("unchecked")
@Override @Override
public void init(final ProcessorContext processorContext) { public void init(final ProcessorContext<String, String> processorContext) {
this.context = processorContext; this.context = processorContext;
this.store = (WindowStore<String, String>) processorContext.getStateStore("store-name"); this.store = processorContext.getStateStore("store-name");
int count = 0; int count = 0;
try (final KeyValueIterator<Windowed<String>, String> all = store.all()) { try (final KeyValueIterator<Windowed<String>, String> all = store.all()) {
@ -190,7 +189,7 @@ public class CachingPersistentWindowStoreTest {
} }
@Override @Override
public KeyValue<String, String> transform(final String key, final String value) { public void process(final Record<String, String> record) {
int count = 0; int count = 0;
try (final KeyValueIterator<Windowed<String>, String> all = store.all()) { try (final KeyValueIterator<Windowed<String>, String> all = store.all()) {
@ -202,22 +201,18 @@ public class CachingPersistentWindowStoreTest {
assertThat(count, equalTo(numRecordsProcessed)); assertThat(count, equalTo(numRecordsProcessed));
store.put(value, value, context.timestamp()); store.put(record.value(), record.value(), record.timestamp());
numRecordsProcessed++; numRecordsProcessed++;
return new KeyValue<>(key, value); context.forward(record);
}
@Override
public void close() {
} }
}, "store-name"); }, "store-name");
final Properties streamsConfiguration = new Properties(); final Properties streamsConfiguration = new Properties();
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L);