mjsax commented on code in PR #21486:
URL: https://github.com/apache/kafka/pull/21486#discussion_r2834880173
##########
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java:
##########
@@ -181,8 +182,8 @@ public boolean setFlushListener(final
CacheFlushListener<Windowed<K>, V> listene
record -> listener.apply(
record.withKey(WindowKeySchema.fromStoreKey(record.key(),
windowSizeMs, serdes.keyDeserializer(), serdes.topic()))
.withValue(new Change<>(
- record.value().newValue != null ?
serdes.valueFrom(record.value().newValue) : null,
- record.value().oldValue != null ?
serdes.valueFrom(record.value().oldValue) : null,
+ record.value().newValue != null ?
serdes.valueFrom(record.value().newValue, new RecordHeaders()) : null,
+ record.value().oldValue != null ?
serdes.valueFrom(record.value().oldValue, new RecordHeaders()) : null,
Review Comment:
I think this must be `record.headers()` in both cases. -- The flush listener
is a DSL feature though, so we cannot test it yet) -- I would also be ok to
just not pass headers for now (we did not update this part for kv-case in
https://github.com/apache/kafka/pull/21451 either), and fix later across all
stores in on PR?
##########
streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreWithHeadersTest.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.processor.TaskId;
+import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.MockRecordCollector;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class MeteredTimestampedWindowStoreWithHeadersTest {
+ private static final String STORE_NAME = "mocked-store";
+ private static final String STORE_TYPE = "scope";
+ private static final String CHANGELOG_TOPIC = "changelog-topic";
+ private static final String KEY = "key";
+ private static final Bytes KEY_BYTES = Bytes.wrap(KEY.getBytes());
+ // timestamp is 97 what is ASCII of 'a'
+ private static final long TIMESTAMP = 97L;
+ private static final RecordHeaders HEADERS = makeHeaders();
+ private static final ValueTimestampHeaders<String> VALUE_TIMESTAMP_HEADERS
=
+ ValueTimestampHeaders.make("value", TIMESTAMP, HEADERS);
+ private static final byte[] VALUE_TIMESTAMP_HEADERS_BYTES =
serializeValueTimestampHeaders();
+ private static final int WINDOW_SIZE_MS = 10;
+
+ private InternalMockProcessorContext<String, Long> context;
+ private final TaskId taskId = new TaskId(0, 0, "My-Topology");
+ @Mock
+ private WindowStore<Bytes, byte[]> innerStoreMock;
+ private final Metrics metrics = new Metrics(new
MetricConfig().recordLevel(Sensor.RecordingLevel.DEBUG));
+ private MeteredTimestampedWindowStoreWithHeaders<String, String> store;
+
+ public void setUp() {
+ final StreamsMetricsImpl streamsMetrics =
+ new StreamsMetricsImpl(metrics, "test", new MockTime());
+
+ context = new InternalMockProcessorContext<>(
+ TestUtils.tempDirectory(),
+ Serdes.String(),
+ Serdes.Long(),
+ streamsMetrics,
+ new StreamsConfig(StreamsTestUtils.getStreamsConfig()),
+ MockRecordCollector::new,
+ new ThreadCache(new LogContext("testCache "), 0, streamsMetrics),
+ Time.SYSTEM,
+ taskId
+ );
+
+ when(innerStoreMock.name()).thenReturn(STORE_NAME);
+
+ store = new MeteredTimestampedWindowStoreWithHeaders<>(
+ innerStoreMock,
+ WINDOW_SIZE_MS, // any size
+ STORE_TYPE,
+ new MockTime(),
+ Serdes.String(),
+ new ValueTimestampHeadersSerde<>(new SerdeThatDoesntHandleNull())
+ );
+ }
+
+ public void setUpWithoutContextName() {
+ final StreamsMetricsImpl streamsMetrics =
+ new StreamsMetricsImpl(metrics, "test", new MockTime());
+
+ context = new InternalMockProcessorContext<>(
+ TestUtils.tempDirectory(),
+ Serdes.String(),
+ Serdes.Long(),
+ streamsMetrics,
+ new StreamsConfig(StreamsTestUtils.getStreamsConfig()),
+ MockRecordCollector::new,
+ new ThreadCache(new LogContext("testCache "), 0, streamsMetrics),
+ Time.SYSTEM,
+ taskId
+ );
+
+ store = new MeteredTimestampedWindowStoreWithHeaders<>(
+ innerStoreMock,
+ WINDOW_SIZE_MS, // any size
+ STORE_TYPE,
+ new MockTime(),
+ Serdes.String(),
+ new ValueTimestampHeadersSerde<>(new SerdeThatDoesntHandleNull())
+ );
+ }
+
+ @Test
+ public void shouldDelegateInit() {
+ setUpWithoutContextName();
+ @SuppressWarnings("unchecked")
+ final WindowStore<Bytes, byte[]> inner = mock(WindowStore.class);
+ final MeteredTimestampedWindowStoreWithHeaders<String, String> outer =
new MeteredTimestampedWindowStoreWithHeaders<>(
+ inner,
+ WINDOW_SIZE_MS, // any size
+ STORE_TYPE,
+ new MockTime(),
+ Serdes.String(),
+ new ValueTimestampHeadersSerde<>(new SerdeThatDoesntHandleNull())
+ );
+ when(inner.name()).thenReturn("store");
+
+ outer.init(context, outer);
+
+ verify(inner).init(context, outer);
+ }
+
+ @Test
+ public void shouldPassChangelogTopicNameToStateStoreSerde() {
+ setUp();
+ context.addChangelogForStore(STORE_NAME, CHANGELOG_TOPIC);
+ doShouldPassChangelogTopicNameToStateStoreSerde(CHANGELOG_TOPIC);
+ }
+
+ @Test
+ public void
shouldPassDefaultChangelogTopicNameToStateStoreSerdeIfLoggingDisabled() {
+ setUp();
+ final String defaultChangelogTopicName =
+ ProcessorStateManager.storeChangelogTopic(context.applicationId(),
STORE_NAME, taskId.topologyName());
+
doShouldPassChangelogTopicNameToStateStoreSerde(defaultChangelogTopicName);
+ }
+
+ @Test
+ public void shouldCloseUnderlyingStore() {
+ setUp();
+ store.init(context, store);
+ store.close();
+
+ verify(innerStoreMock).close();
+ }
+
+ @Test
+ public void shouldNotExceptionIfFetchReturnsNull() {
+ setUp();
+ when(innerStoreMock.fetch(Bytes.wrap("a".getBytes()),
0)).thenReturn(null);
+
+ store.init(context, store);
+ assertNull(store.fetch("a", 0));
+ }
+
+ @Test
+ public void
shouldNotThrowExceptionIfSerdesCorrectlySetFromConstructorParameters() {
Review Comment:
The test `MeteredTimestampedWindowStoreTest` also has
`shouldNotThrowExceptionIfSerdesCorrectlySetFromProcessorContext` -- why did we
not mimic it, too?
##########
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreWithHeaders.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.errors.ProcessorStateException;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.TimestampedWindowStoreWithHeaders;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+import org.apache.kafka.streams.state.WindowStore;
+
+import java.util.Objects;
+
+import static
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+
+/**
+ * A Metered {@link TimestampedWindowStoreWithHeaders} wrapper that is used
for recording operation metrics,
+ * and hence its inner WindowStore implementation does not need to provide its
own metrics collecting functionality.
+ * The inner {@link WindowStore} of this class is of type
<Bytes,byte[]>, hence we use {@link Serde}s
+ * to convert from <K,ValueTimestampHeaders<V>> to
<Bytes,byte[]>.
+ *
+ * @param <K> key type
+ * @param <V> value type
+ */
+class MeteredTimestampedWindowStoreWithHeaders<K, V>
Review Comment:
Should we override `query()` and `getPosition()` similar to
https://github.com/apache/kafka/pull/21451 ?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]