Skip to content

Commit b6c1cb0

Browse files
authored
MINOR: update CachingPersistentWindowStoreTest (#16701)
Refactor test to move off deprecated `transform()` in favor of `process()`. Reviewers: Bill Bejeck <[email protected]>
1 parent 2cf87bf commit b6c1cb0

File tree

1 file changed

+18
-23
lines changed

1 file changed

+18
-23
lines changed

streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java

Lines changed: 18 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -32,11 +32,12 @@
3232
import org.apache.kafka.streams.errors.InvalidStateStoreException;
3333
import org.apache.kafka.streams.kstream.Consumed;
3434
import org.apache.kafka.streams.kstream.TimeWindowedDeserializer;
35-
import org.apache.kafka.streams.kstream.Transformer;
3635
import org.apache.kafka.streams.kstream.Windowed;
3736
import org.apache.kafka.streams.kstream.internals.TimeWindow;
38-
import org.apache.kafka.streams.processor.ProcessorContext;
3937
import org.apache.kafka.streams.processor.StateStoreContext;
38+
import org.apache.kafka.streams.processor.api.Processor;
39+
import org.apache.kafka.streams.processor.api.ProcessorContext;
40+
import org.apache.kafka.streams.processor.api.Record;
4041
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
4142
import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
4243
import org.apache.kafka.streams.query.Position;
@@ -102,7 +103,7 @@ public class CachingPersistentWindowStoreTest {
102103
private static final String TOPIC = "topic";
103104
private static final String CACHE_NAMESPACE = "0_0-store-name";
104105

105-
private InternalMockProcessorContext context;
106+
private InternalMockProcessorContext<?, ?> context;
106107
private RocksDBSegmentedBytesStore bytesStore;
107108
private WindowStore<Bytes, byte[]> underlyingStore;
108109
private CachingWindowStore cachingStore;
@@ -138,8 +139,8 @@ public void shouldDelegateDeprecatedInit() {
138139
final WindowStore<Bytes, byte[]> inner = mock(WindowStore.class);
139140
final CachingWindowStore outer = new CachingWindowStore(inner, WINDOW_SIZE, SEGMENT_INTERVAL);
140141
when(inner.name()).thenReturn("store");
141-
outer.init((ProcessorContext) context, outer);
142-
verify(inner).init((ProcessorContext) context, outer);
142+
outer.init((org.apache.kafka.streams.processor.ProcessorContext) context, outer);
143+
verify(inner).init((org.apache.kafka.streams.processor.ProcessorContext) context, outer);
143144
}
144145

145146
@SuppressWarnings("unchecked")
@@ -153,30 +154,28 @@ public void shouldDelegateInit() {
153154
}
154155

155156
@Test
156-
@SuppressWarnings("deprecation")
157157
public void shouldNotReturnDuplicatesInRanges() {
158158
final StreamsBuilder builder = new StreamsBuilder();
159159

160160
final StoreBuilder<WindowStore<String, String>> storeBuilder = Stores.windowStoreBuilder(
161-
Stores.persistentWindowStore("store-name", ofHours(1L), ofMinutes(1L), false),
162-
Serdes.String(),
163-
Serdes.String())
161+
Stores.persistentWindowStore("store-name", ofHours(1L), ofMinutes(1L), false),
162+
Serdes.String(),
163+
Serdes.String())
164164
.withCachingEnabled();
165165

166166
builder.addStateStore(storeBuilder);
167167

168168
builder.stream(TOPIC,
169169
Consumed.with(Serdes.String(), Serdes.String()))
170-
.transform(() -> new Transformer<String, String, KeyValue<String, String>>() {
170+
.process(() -> new Processor<String, String, String, String>() {
171171
private WindowStore<String, String> store;
172172
private int numRecordsProcessed;
173-
private ProcessorContext context;
173+
private ProcessorContext<String, String> context;
174174

175-
@SuppressWarnings("unchecked")
176175
@Override
177-
public void init(final ProcessorContext processorContext) {
176+
public void init(final ProcessorContext<String, String> processorContext) {
178177
this.context = processorContext;
179-
this.store = (WindowStore<String, String>) processorContext.getStateStore("store-name");
178+
this.store = processorContext.getStateStore("store-name");
180179
int count = 0;
181180

182181
try (final KeyValueIterator<Windowed<String>, String> all = store.all()) {
@@ -190,7 +189,7 @@ public void init(final ProcessorContext processorContext) {
190189
}
191190

192191
@Override
193-
public KeyValue<String, String> transform(final String key, final String value) {
192+
public void process(final Record<String, String> record) {
194193
int count = 0;
195194

196195
try (final KeyValueIterator<Windowed<String>, String> all = store.all()) {
@@ -202,22 +201,18 @@ public KeyValue<String, String> transform(final String key, final String value)
202201

203202
assertThat(count, equalTo(numRecordsProcessed));
204203

205-
store.put(value, value, context.timestamp());
204+
store.put(record.value(), record.value(), record.timestamp());
206205

207206
numRecordsProcessed++;
208207

209-
return new KeyValue<>(key, value);
210-
}
211-
212-
@Override
213-
public void close() {
208+
context.forward(record);
214209
}
215210
}, "store-name");
216211

217212
final Properties streamsConfiguration = new Properties();
218213
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
219-
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
220-
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
214+
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
215+
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
221216
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
222217
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L);
223218

0 commit comments

Comments
 (0)