aliehsaeedii commented on code in PR #21513: URL: https://github.com/apache/kafka/pull/21513#discussion_r2838125995
########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreWithHeaders.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.state.AggregationWithHeaders; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.SessionStoreWithHeaders; + +public class MeteredSessionStoreWithHeaders<K, AGG> + extends MeteredSessionStore<K, AggregationWithHeaders<AGG>> + implements SessionStoreWithHeaders<K, AGG> { + + MeteredSessionStoreWithHeaders(final SessionStore<Bytes, byte[]> inner, + final String metricsScope, + final Serde<K> keySerde, + final Serde<AggregationWithHeaders<AGG>> aggSerde, + final Time time) { + super(inner, metricsScope, keySerde, aggSerde, time); + } + + // Factory method to create the wrapped serde + static <AGG> Serde<AggregationWithHeaders<AGG>> createAggregationWithHeadersSerde(final Serde<AGG> aggSerde) { Review Comment: Is this method (`createAggregationWithHeadersSerde`) called any where? ########## streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreWithHeadersTest.java: ########## @@ -0,0 +1,719 @@ +/* + * 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.MetricName; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.KafkaMetricsContext; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.MetricsContext; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.AggregationWithHeaders; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.test.KeyValueIteratorStub; + +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 java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +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 MeteredSessionStoreWithHeadersTest { + + private static final String APPLICATION_ID = "test-app"; + private static final String STORE_TYPE = "scope"; + private static final String STORE_NAME = "mocked-store"; + private static final String STORE_LEVEL_GROUP = "stream-state-metrics"; + private static final String THREAD_ID_TAG_KEY = "thread-id"; + private static final String CHANGELOG_TOPIC = "changelog-topic"; + private static final String KEY = "key"; + private static final Bytes KEY_BYTES = Bytes.wrap(KEY.getBytes()); + private static final Windowed<String> WINDOWED_KEY = new Windowed<>(KEY, new SessionWindow(0, 0)); + private static final Windowed<Bytes> WINDOWED_KEY_BYTES = new Windowed<>(KEY_BYTES, new SessionWindow(0, 0)); + private static final String VALUE = "value"; + private static final long START_TIMESTAMP = 24L; + private static final long END_TIMESTAMP = 42L; + + private final String threadId = Thread.currentThread().getName(); + private final TaskId taskId = new TaskId(0, 0, "My-Topology"); + private final Metrics metrics = new Metrics(); + private MockTime mockTime; + private MeteredSessionStoreWithHeaders<String, String> store; + + @Mock + private SessionStore<Bytes, byte[]> innerStore; + + @Mock + private InternalProcessorContext<?, ?> context; + + private Map<String, String> tags; + + private void setUp() { + mockTime = new MockTime(); + store = new MeteredSessionStoreWithHeaders<>( + innerStore, + STORE_TYPE, + Serdes.String(), + MeteredSessionStoreWithHeaders.createAggregationWithHeadersSerde(Serdes.String()), + mockTime + ); + tags = mkMap( + mkEntry(THREAD_ID_TAG_KEY, threadId), + mkEntry("task-id", taskId.toString()), + mkEntry(STORE_TYPE + "-state-id", STORE_NAME) + ); + + metrics.config().recordLevel(Sensor.RecordingLevel.DEBUG); + when(context.applicationId()).thenReturn(APPLICATION_ID); + when(context.metrics()) + .thenReturn(new StreamsMetricsImpl(metrics, "test", mockTime)); + when(context.taskId()).thenReturn(taskId); + when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC); + when(innerStore.name()).thenReturn(STORE_NAME); + } + + private void init() { + store.init(context, store); + } + + private KafkaMetric metric(final String name) { + return this.metrics.metric(new MetricName(name, STORE_LEVEL_GROUP, "", this.tags)); + } + + private List<MetricName> storeMetrics() { + return metrics.metrics() + .keySet() + .stream() + .filter(name -> name.group().equals(STORE_LEVEL_GROUP) && name.tags().equals(tags)) + .collect(Collectors.toList()); + } + + @Test + public void shouldDelegateInit() { + setUp(); + final MeteredSessionStoreWithHeaders<String, String> outer = new MeteredSessionStoreWithHeaders<>( + innerStore, + STORE_TYPE, + Serdes.String(), + MeteredSessionStoreWithHeaders.createAggregationWithHeadersSerde(Serdes.String()), + new MockTime() + ); + doNothing().when(innerStore).init(context, outer); + outer.init(context, outer); + } + + @Test + public void testMetrics() { + setUp(); + init(); + final JmxReporter reporter = new JmxReporter(); + final MetricsContext metricsContext = new KafkaMetricsContext("kafka.streams"); + reporter.contextChange(metricsContext); + + metrics.addReporter(reporter); + assertTrue(reporter.containsMbean(String.format( + "kafka.streams:type=%s,%s=%s,task-id=%s,%s-state-id=%s", + STORE_LEVEL_GROUP, + THREAD_ID_TAG_KEY, + threadId, + taskId, + STORE_TYPE, + STORE_NAME + ))); + } + + @Test + public void shouldWriteBytesToInnerStoreAndRecordPutMetric() { + setUp(); + init(); + + final Headers headers = new RecordHeaders(); + headers.add("key1", "value1".getBytes()); + final AggregationWithHeaders<String> valueAndHeaders = AggregationWithHeaders.make(VALUE, headers); + + doNothing().when(innerStore).put(any(Windowed.class), any(byte[].class)); + + store.put(WINDOWED_KEY, valueAndHeaders); + + verify(innerStore).put(any(Windowed.class), any(byte[].class)); + + final KafkaMetric metric = metric("put-rate"); + assertTrue((Double) metric.metricValue() > 0); + } + + @Test + public void shouldPutWithHeadersUsingConvenienceMethod() { + setUp(); + init(); + + final Headers headers = new RecordHeaders(); + headers.add("key1", "value1".getBytes()); + + doNothing().when(innerStore).put(any(Windowed.class), any(byte[].class)); + + store.put(WINDOWED_KEY, VALUE, headers); + + verify(innerStore).put(any(Windowed.class), any(byte[].class)); Review Comment: also in the follwing tests if it is possible and if it makes sense. ########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreWithHeaders.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.state.AggregationWithHeaders; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.SessionStoreWithHeaders; + +public class MeteredSessionStoreWithHeaders<K, AGG> + extends MeteredSessionStore<K, AggregationWithHeaders<AGG>> + implements SessionStoreWithHeaders<K, AGG> { + + MeteredSessionStoreWithHeaders(final SessionStore<Bytes, byte[]> inner, + final String metricsScope, + final Serde<K> keySerde, + final Serde<AggregationWithHeaders<AGG>> aggSerde, + final Time time) { + super(inner, metricsScope, keySerde, aggSerde, time); + } + + // Factory method to create the wrapped serde + static <AGG> Serde<AggregationWithHeaders<AGG>> createAggregationWithHeadersSerde(final Serde<AGG> aggSerde) { + return new Serde<>() { + @Override + public Serializer<AggregationWithHeaders<AGG>> serializer() { + return new AggregationWithHeadersSerializer<>(aggSerde.serializer()); + } + + @Override + public Deserializer<AggregationWithHeaders<AGG>> deserializer() { + return new AggregationWithHeadersDeserializer<>(aggSerde.deserializer()); + } + }; + } + + @SuppressWarnings("unchecked") + @Override + public boolean setFlushListener(final CacheFlushListener<Windowed<K>, AggregationWithHeaders<AGG>> listener, + final boolean sendOldValues) { + final SessionStore<Bytes, byte[]> wrapped = wrapped(); + if (wrapped instanceof CachedStateStore) { + return ((CachedStateStore<byte[], byte[]>) wrapped).setFlushListener( + record -> listener.apply( + record.withKey(SessionKeySchema.from(record.key(), serdes.keyDeserializer(), new RecordHeaders(), serdes.topic())) + .withValue(new Change<>( + record.value().newValue != null ? serdes.valueFrom(record.value().newValue, new RecordHeaders()) : null, + record.value().oldValue != null ? serdes.valueFrom(record.value().oldValue, new RecordHeaders()) : null, + record.value().isLatest + )) + ), + sendOldValues); + } + return false; + } +} Review Comment: Please override the put (write to SS) methods. ########## streams/src/main/java/org/apache/kafka/streams/state/SessionStoreWithHeaders.java: ########## @@ -0,0 +1,81 @@ +/* + * 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; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.StateStore; + +/** + * Interface for storing the aggregated values of sessions along with their record headers. + * <p> + * The key is internally represented as {@link Windowed Windowed<K>} that comprises the plain + * key and the {@link Window} that represents window start- and end-timestamp. + * <p> + * If two sessions are merged, a new session with new start- and end-timestamp must be inserted into + * the store while the two old sessions must be deleted. + * + * @param <K> type of the record keys + * @param <AGG> type of the aggregated values + */ +public interface SessionStoreWithHeaders<K, AGG> extends StateStore, ReadOnlySessionStore<K, AggregationWithHeaders<AGG>> { + Review Comment: Could it be? ``` public interface SessionStoreWithHeaders<K, AGG> extends SessionStore<K, AggregationWithHeaders<AGG>> { ########## streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreWithHeadersTest.java: ########## @@ -0,0 +1,719 @@ +/* + * 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.MetricName; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.KafkaMetricsContext; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.MetricsContext; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.AggregationWithHeaders; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.test.KeyValueIteratorStub; + +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 java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +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 MeteredSessionStoreWithHeadersTest { + + private static final String APPLICATION_ID = "test-app"; + private static final String STORE_TYPE = "scope"; + private static final String STORE_NAME = "mocked-store"; + private static final String STORE_LEVEL_GROUP = "stream-state-metrics"; + private static final String THREAD_ID_TAG_KEY = "thread-id"; + private static final String CHANGELOG_TOPIC = "changelog-topic"; + private static final String KEY = "key"; + private static final Bytes KEY_BYTES = Bytes.wrap(KEY.getBytes()); + private static final Windowed<String> WINDOWED_KEY = new Windowed<>(KEY, new SessionWindow(0, 0)); + private static final Windowed<Bytes> WINDOWED_KEY_BYTES = new Windowed<>(KEY_BYTES, new SessionWindow(0, 0)); + private static final String VALUE = "value"; + private static final long START_TIMESTAMP = 24L; + private static final long END_TIMESTAMP = 42L; + + private final String threadId = Thread.currentThread().getName(); + private final TaskId taskId = new TaskId(0, 0, "My-Topology"); + private final Metrics metrics = new Metrics(); + private MockTime mockTime; + private MeteredSessionStoreWithHeaders<String, String> store; + + @Mock + private SessionStore<Bytes, byte[]> innerStore; + + @Mock + private InternalProcessorContext<?, ?> context; + + private Map<String, String> tags; + + private void setUp() { + mockTime = new MockTime(); + store = new MeteredSessionStoreWithHeaders<>( + innerStore, + STORE_TYPE, + Serdes.String(), + MeteredSessionStoreWithHeaders.createAggregationWithHeadersSerde(Serdes.String()), + mockTime + ); + tags = mkMap( + mkEntry(THREAD_ID_TAG_KEY, threadId), + mkEntry("task-id", taskId.toString()), + mkEntry(STORE_TYPE + "-state-id", STORE_NAME) + ); + + metrics.config().recordLevel(Sensor.RecordingLevel.DEBUG); + when(context.applicationId()).thenReturn(APPLICATION_ID); + when(context.metrics()) + .thenReturn(new StreamsMetricsImpl(metrics, "test", mockTime)); + when(context.taskId()).thenReturn(taskId); + when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC); + when(innerStore.name()).thenReturn(STORE_NAME); + } + + private void init() { + store.init(context, store); + } + + private KafkaMetric metric(final String name) { + return this.metrics.metric(new MetricName(name, STORE_LEVEL_GROUP, "", this.tags)); + } + + private List<MetricName> storeMetrics() { + return metrics.metrics() + .keySet() + .stream() + .filter(name -> name.group().equals(STORE_LEVEL_GROUP) && name.tags().equals(tags)) + .collect(Collectors.toList()); + } + + @Test + public void shouldDelegateInit() { + setUp(); + final MeteredSessionStoreWithHeaders<String, String> outer = new MeteredSessionStoreWithHeaders<>( + innerStore, + STORE_TYPE, + Serdes.String(), + MeteredSessionStoreWithHeaders.createAggregationWithHeadersSerde(Serdes.String()), + new MockTime() + ); + doNothing().when(innerStore).init(context, outer); + outer.init(context, outer); + } + + @Test + public void testMetrics() { + setUp(); + init(); + final JmxReporter reporter = new JmxReporter(); + final MetricsContext metricsContext = new KafkaMetricsContext("kafka.streams"); + reporter.contextChange(metricsContext); + + metrics.addReporter(reporter); + assertTrue(reporter.containsMbean(String.format( + "kafka.streams:type=%s,%s=%s,task-id=%s,%s-state-id=%s", + STORE_LEVEL_GROUP, + THREAD_ID_TAG_KEY, + threadId, + taskId, + STORE_TYPE, + STORE_NAME + ))); + } + + @Test + public void shouldWriteBytesToInnerStoreAndRecordPutMetric() { + setUp(); + init(); + + final Headers headers = new RecordHeaders(); + headers.add("key1", "value1".getBytes()); + final AggregationWithHeaders<String> valueAndHeaders = AggregationWithHeaders.make(VALUE, headers); + + doNothing().when(innerStore).put(any(Windowed.class), any(byte[].class)); + + store.put(WINDOWED_KEY, valueAndHeaders); + + verify(innerStore).put(any(Windowed.class), any(byte[].class)); + + final KafkaMetric metric = metric("put-rate"); + assertTrue((Double) metric.metricValue() > 0); + } + + @Test + public void shouldPutWithHeadersUsingConvenienceMethod() { + setUp(); + init(); + + final Headers headers = new RecordHeaders(); + headers.add("key1", "value1".getBytes()); + + doNothing().when(innerStore).put(any(Windowed.class), any(byte[].class)); + + store.put(WINDOWED_KEY, VALUE, headers); + + verify(innerStore).put(any(Windowed.class), any(byte[].class)); Review Comment: could we verify if `headers` is wrotten to the metered layer? ########## streams/src/main/java/org/apache/kafka/streams/state/SessionStoreWithHeaders.java: ########## @@ -0,0 +1,81 @@ +/* + * 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; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.StateStore; + +/** + * Interface for storing the aggregated values of sessions along with their record headers. + * <p> + * The key is internally represented as {@link Windowed Windowed<K>} that comprises the plain + * key and the {@link Window} that represents window start- and end-timestamp. + * <p> + * If two sessions are merged, a new session with new start- and end-timestamp must be inserted into + * the store while the two old sessions must be deleted. + * + * @param <K> type of the record keys + * @param <AGG> type of the aggregated values + */ +public interface SessionStoreWithHeaders<K, AGG> extends StateStore, ReadOnlySessionStore<K, AggregationWithHeaders<AGG>> { + Review Comment: then maybe no method is needed to be defned here. Sorry I did not see that in the prev round. ########## streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreWithHeaders.java: ########## @@ -0,0 +1,77 @@ +/* + * 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.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.Change; +import org.apache.kafka.streams.state.AggregationWithHeaders; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.SessionStoreWithHeaders; + +public class MeteredSessionStoreWithHeaders<K, AGG> + extends MeteredSessionStore<K, AggregationWithHeaders<AGG>> + implements SessionStoreWithHeaders<K, AGG> { + + MeteredSessionStoreWithHeaders(final SessionStore<Bytes, byte[]> inner, + final String metricsScope, + final Serde<K> keySerde, + final Serde<AggregationWithHeaders<AGG>> aggSerde, + final Time time) { + super(inner, metricsScope, keySerde, aggSerde, time); + } + + // Factory method to create the wrapped serde + static <AGG> Serde<AggregationWithHeaders<AGG>> createAggregationWithHeadersSerde(final Serde<AGG> aggSerde) { + return new Serde<>() { + @Override + public Serializer<AggregationWithHeaders<AGG>> serializer() { + return new AggregationWithHeadersSerializer<>(aggSerde.serializer()); + } + + @Override + public Deserializer<AggregationWithHeaders<AGG>> deserializer() { + return new AggregationWithHeadersDeserializer<>(aggSerde.deserializer()); + } + }; + } + + @SuppressWarnings("unchecked") + @Override + public boolean setFlushListener(final CacheFlushListener<Windowed<K>, AggregationWithHeaders<AGG>> listener, + final boolean sendOldValues) { + final SessionStore<Bytes, byte[]> wrapped = wrapped(); + if (wrapped instanceof CachedStateStore) { + return ((CachedStateStore<byte[], byte[]>) wrapped).setFlushListener( + record -> listener.apply( + record.withKey(SessionKeySchema.from(record.key(), serdes.keyDeserializer(), new RecordHeaders(), serdes.topic())) + .withValue(new Change<>( + record.value().newValue != null ? serdes.valueFrom(record.value().newValue, new RecordHeaders()) : null, Review Comment: if you are just sending `new RecordHeaders()`, could we make it in the parent class and remove the override here? BTW, could it be `record.headers()`?! Isn't it more accurate? -- 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]
