mjsax commented on code in PR #21454:
URL: https://github.com/apache/kafka/pull/21454#discussion_r2796196117


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContext.java:
##########
@@ -122,6 +123,13 @@ void logChange(final String storeName,
                    final long timestamp,
                    final Position position);
 
+    void logChange(final String storeName,

Review Comment:
   Why do we add a new overload? Do we still need the variant w/o header 
parameter? Seems the few callers of the existing `logChange` could just be 
updated to pass in `new RecordHeaders()`?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java:
##########
@@ -164,4 +164,17 @@ static Headers headers(final byte[] 
rawValueTimestampHeaders) {
         final byte[] rawHeaders = readBytes(buffer, headersSize);
         return HEADERS_DESERIALIZER.deserialize("", rawHeaders);
     }
+    /**
+     * Extract raw value from serialized ValueTimestampHeaders.
+     */
+    static byte[] rawValue(final byte[] rawValueTimestampHeaders) {

Review Comment:
   We have `static <T> T value(...)` above -- given that we need `rawValue` 
here, wondering if `value(...)` from above is actually needed or not at all?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.KeyValueStore;
+
+import java.util.List;
+
+import static 
org.apache.kafka.streams.state.internals.ValueTimestampHeadersDeserializer.headers;
+import static 
org.apache.kafka.streams.state.internals.ValueTimestampHeadersDeserializer.rawValue;
+import static 
org.apache.kafka.streams.state.internals.ValueTimestampHeadersDeserializer.timestamp;
+
+/**
+ * Change-logging wrapper for a timestamped key-value bytes store whose values 
also carry headers.
+ * <p>
+ * the header-aware serialized value format produced by {@link 
ValueTimestampHeadersSerializer}.
+ * <p>
+ * Semantics:
+ *  - The inner store value format is:
+ *        [ varint header_length ][ header_bytes ][ 8-byte timestamp ][ 
value_bytes ]
+ *  - The changelog record value logged via {@code log(...)} remains just 
{@code value_bytes}
+ *    (no timestamp, no headers), and the timestamp is logged separately.
+ */
+public class ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders
+    extends ChangeLoggingKeyValueBytesStore {
+
+    ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders(final 
KeyValueStore<Bytes, byte[]> inner) {
+        super(inner);
+    }
+
+    @Override
+    public void put(final Bytes key,
+                    final byte[] valueTimestampHeaders) {
+        wrapped().put(key, valueTimestampHeaders);
+        log(
+            key,
+            rawValue(valueTimestampHeaders),
+            valueTimestampHeaders == null
+                ? internalContext.recordContext().timestamp()
+                : timestamp(valueTimestampHeaders),
+            headers(valueTimestampHeaders)
+        );
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key,
+                              final byte[] valueTimestampHeaders) {
+        final byte[] previous = wrapped().putIfAbsent(key, 
valueTimestampHeaders);
+        if (previous == null) {
+            // then it was absent
+            log(
+                key,
+                rawValue(valueTimestampHeaders),
+                valueTimestampHeaders == null
+                    ? internalContext.recordContext().timestamp()
+                    : timestamp(valueTimestampHeaders),
+                headers(valueTimestampHeaders)
+            );
+        }
+        return previous;
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        wrapped().putAll(entries);
+        for (final KeyValue<Bytes, byte[]> entry : entries) {
+            final byte[] valueTimestampHeaders = entry.value;
+            log(
+                entry.key,
+                rawValue(valueTimestampHeaders),
+                valueTimestampHeaders == null
+                    ? internalContext.recordContext().timestamp()
+                    : timestamp(valueTimestampHeaders),
+                headers(valueTimestampHeaders)
+            );
+        }
+    }
+
+    void log(final Bytes key, final byte[] value, final long timestamp, final 
Headers headers) {

Review Comment:
   If we just extend `InternalProcessorContext.log(...)` with a new `Headers` 
parameter and not add a new overload, and update all existing code accordingly, 
I believe we can remove the method and reuse the existing one from 
`ChangeLoggingKeyValueBytesStore`



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreWithHeadersTest.java:
##########
@@ -0,0 +1,339 @@
+/*
+ * 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.metrics.Metrics;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.MockRecordCollector;
+import org.apache.kafka.test.TestUtils;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Arrays;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class ChangeLoggingTimestampedKeyValueBytesStoreWithHeadersTest {
+
+    private final MockRecordCollector collector = new MockRecordCollector();
+    private final InMemoryKeyValueStore root = new InMemoryKeyValueStore("kv");
+    private final ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders store =
+        new ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders(root);
+    private final Bytes hi = Bytes.wrap("hi".getBytes());
+    private final Bytes hello = Bytes.wrap("hello".getBytes());
+
+    private final RecordHeaders thereHeaders = new RecordHeaders();
+    private final RecordHeaders worldHeaders = new RecordHeaders();
+
+    private final ValueTimestampHeaders<byte[]> there = 
ValueTimestampHeaders.make("there".getBytes(), 97L, thereHeaders);
+    private final ValueTimestampHeaders<byte[]> world = 
ValueTimestampHeaders.make("world".getBytes(), 98L, worldHeaders);
+
+    private byte[] rawThere;
+    private byte[] rawWorld;
+
+    @BeforeEach
+    public void before() {
+        thereHeaders.add("key1", "value1".getBytes());
+        worldHeaders.add("key2", "value2".getBytes());
+
+        final ValueTimestampHeadersSerializer<byte[]> serializer =
+            new 
ValueTimestampHeadersSerializer<>(Serdes.ByteArray().serializer());
+        rawThere = serializer.serialize("topic", there);
+        rawWorld = serializer.serialize("topic", world);
+
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        context.setTime(0);
+        store.init(context, store);
+    }
+
+    private InternalMockProcessorContext<String, Long> mockContext() {
+        return new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.Long(),
+            collector,
+            new ThreadCache(new LogContext("testCache "), 0, new 
MockStreamsMetrics(new Metrics()))
+        );
+    }
+
+    @AfterEach
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldDelegateInit() {
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        final KeyValueStore<Bytes, byte[]> inner = 
mock(InMemoryKeyValueStore.class);
+        final StateStore outer = new 
ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders(inner);
+
+        outer.init(context, outer);
+        verify(inner).init(context, outer);
+    }
+
+    @Test
+    public void shouldWriteKeyValueBytesToInnerStoreOnPut() {
+        store.put(hi, rawThere);
+
+        assertEquals(rawThere, root.get(hi));
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+
+        // Verify headers are logged
+        final Headers loggedHeaders = collector.collected().get(0).headers();
+        assertEquals(1, loggedHeaders.toArray().length);
+        assertEquals("value1", new 
String(loggedHeaders.lastHeader("key1").value()));
+    }
+
+    @Test
+    public void shouldWriteAllKeyValueToInnerStoreOnPutAll() {
+        store.putAll(Arrays.asList(KeyValue.pair(hi, rawThere),
+                                   KeyValue.pair(hello, rawWorld)));
+        assertEquals(rawThere, root.get(hi));
+        assertEquals(rawWorld, root.get(hello));
+    }
+
+    @Test
+    public void shouldLogChangesOnPutAll() {
+        store.putAll(Arrays.asList(KeyValue.pair(hi, rawThere),
+                                   KeyValue.pair(hello, rawWorld)));
+
+        assertEquals(2, collector.collected().size());
+
+        // First entry
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+        final Headers headers0 = collector.collected().get(0).headers();
+        assertEquals(1, headers0.toArray().length);
+        assertEquals("value1", new 
String(headers0.lastHeader("key1").value()));
+
+        // Second entry
+        assertEquals(hello, collector.collected().get(1).key());
+        assertArrayEquals(world.value(), (byte[]) 
collector.collected().get(1).value());
+        assertEquals(98L, collector.collected().get(1).timestamp());
+        final Headers headers1 = collector.collected().get(1).headers();
+        assertEquals(1, headers1.toArray().length);
+        assertEquals("value2", new 
String(headers1.lastHeader("key2").value()));
+    }
+
+    @Test
+    public void shouldPropagateDelete() {
+        store.put(hi, rawThere);
+        store.delete(hi);
+        assertEquals(0L, root.approximateNumEntries());
+        assertNull(root.get(hi));
+    }
+
+    @Test
+    public void shouldReturnOldValueOnDelete() {
+        store.put(hi, rawThere);
+        assertEquals(rawThere, store.delete(hi));
+    }
+
+    @Test
+    public void shouldLogKeyNullOnDelete() {
+        store.put(hi, rawThere);
+        store.delete(hi);
+
+        assertEquals(2, collector.collected().size());
+
+        // First record is the put
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+        final Headers headers0 = collector.collected().get(0).headers();
+        assertEquals(1, headers0.toArray().length);
+        assertEquals("value1", new 
String(headers0.lastHeader("key1").value()));
+
+        // Second record is the delete
+        assertEquals(hi, collector.collected().get(1).key());
+        assertNull(collector.collected().get(1).value());
+        assertEquals(0L, collector.collected().get(1).timestamp());
+        assertEquals(0, 
collector.collected().get(1).headers().toArray().length);
+    }
+
+    @Test
+    public void shouldWriteToInnerOnPutIfAbsentNoPreviousValue() {
+        store.putIfAbsent(hi, rawThere);
+        assertEquals(rawThere, root.get(hi));
+    }
+
+    @Test
+    public void shouldNotWriteToInnerOnPutIfAbsentWhenValueForKeyExists() {
+        store.put(hi, rawThere);
+        store.putIfAbsent(hi, rawWorld);
+        assertEquals(rawThere, root.get(hi));
+    }
+
+    @Test
+    public void shouldWriteToChangelogOnPutIfAbsentWhenNoPreviousValue() {
+        store.putIfAbsent(hi, rawThere);
+
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+
+        final Headers headers = collector.collected().get(0).headers();
+        assertEquals(1, headers.toArray().length);
+        assertEquals("value1", new String(headers.lastHeader("key1").value()));
+    }
+
+    @Test
+    public void shouldNotWriteToChangeLogOnPutIfAbsentWhenValueForKeyExists() {
+        store.put(hi, rawThere);
+        store.putIfAbsent(hi, rawWorld);
+
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+        final Headers headers0 = collector.collected().get(0).headers();
+        assertEquals(1, headers0.toArray().length);
+        assertEquals("value1", new 
String(headers0.lastHeader("key1").value()));
+    }
+
+    @Test
+    public void shouldReturnCurrentValueOnPutIfAbsent() {
+        store.put(hi, rawThere);
+        assertEquals(rawThere, store.putIfAbsent(hi, rawWorld));
+    }
+
+    @Test
+    public void shouldReturnNullOnPutIfAbsentWhenNoPreviousValue() {
+        assertNull(store.putIfAbsent(hi, rawThere));
+    }
+
+    @Test
+    public void shouldReturnValueOnGetWhenExists() {
+        store.put(hello, rawWorld);
+        assertEquals(rawWorld, store.get(hello));
+    }
+
+    @Test
+    public void shouldReturnNullOnGetWhenDoesntExist() {
+        assertNull(store.get(hello));
+    }
+
+    @Test
+    public void shouldHandleNullValueInPut() {
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        context.setTime(42L);
+        store.init(context, store);
+
+        store.put(hi, null);
+
+        // Should log with context timestamp when value is null
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertNull(collector.collected().get(0).value());
+        assertEquals(42L, collector.collected().get(0).timestamp());
+        assertEquals(0, 
collector.collected().get(0).headers().toArray().length);
+    }
+
+    @Test
+    public void shouldHandleNullValueInPutIfAbsent() {
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        context.setTime(50L);

Review Comment:
   As above



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedKeyValueBytesStoreWithHeadersTest.java:
##########
@@ -0,0 +1,339 @@
+/*
+ * 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.metrics.Metrics;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.ValueTimestampHeaders;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.MockRecordCollector;
+import org.apache.kafka.test.TestUtils;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+import java.util.Arrays;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.STRICT_STUBS)
+public class ChangeLoggingTimestampedKeyValueBytesStoreWithHeadersTest {
+
+    private final MockRecordCollector collector = new MockRecordCollector();
+    private final InMemoryKeyValueStore root = new InMemoryKeyValueStore("kv");
+    private final ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders store =
+        new ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders(root);
+    private final Bytes hi = Bytes.wrap("hi".getBytes());
+    private final Bytes hello = Bytes.wrap("hello".getBytes());
+
+    private final RecordHeaders thereHeaders = new RecordHeaders();
+    private final RecordHeaders worldHeaders = new RecordHeaders();
+
+    private final ValueTimestampHeaders<byte[]> there = 
ValueTimestampHeaders.make("there".getBytes(), 97L, thereHeaders);
+    private final ValueTimestampHeaders<byte[]> world = 
ValueTimestampHeaders.make("world".getBytes(), 98L, worldHeaders);
+
+    private byte[] rawThere;
+    private byte[] rawWorld;
+
+    @BeforeEach
+    public void before() {
+        thereHeaders.add("key1", "value1".getBytes());
+        worldHeaders.add("key2", "value2".getBytes());
+
+        final ValueTimestampHeadersSerializer<byte[]> serializer =
+            new 
ValueTimestampHeadersSerializer<>(Serdes.ByteArray().serializer());
+        rawThere = serializer.serialize("topic", there);
+        rawWorld = serializer.serialize("topic", world);
+
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        context.setTime(0);
+        store.init(context, store);
+    }
+
+    private InternalMockProcessorContext<String, Long> mockContext() {
+        return new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.Long(),
+            collector,
+            new ThreadCache(new LogContext("testCache "), 0, new 
MockStreamsMetrics(new Metrics()))
+        );
+    }
+
+    @AfterEach
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldDelegateInit() {
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        final KeyValueStore<Bytes, byte[]> inner = 
mock(InMemoryKeyValueStore.class);
+        final StateStore outer = new 
ChangeLoggingTimestampedKeyValueBytesStoreWithHeaders(inner);
+
+        outer.init(context, outer);
+        verify(inner).init(context, outer);
+    }
+
+    @Test
+    public void shouldWriteKeyValueBytesToInnerStoreOnPut() {
+        store.put(hi, rawThere);
+
+        assertEquals(rawThere, root.get(hi));
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+
+        // Verify headers are logged
+        final Headers loggedHeaders = collector.collected().get(0).headers();
+        assertEquals(1, loggedHeaders.toArray().length);
+        assertEquals("value1", new 
String(loggedHeaders.lastHeader("key1").value()));
+    }
+
+    @Test
+    public void shouldWriteAllKeyValueToInnerStoreOnPutAll() {
+        store.putAll(Arrays.asList(KeyValue.pair(hi, rawThere),
+                                   KeyValue.pair(hello, rawWorld)));
+        assertEquals(rawThere, root.get(hi));
+        assertEquals(rawWorld, root.get(hello));
+    }
+
+    @Test
+    public void shouldLogChangesOnPutAll() {
+        store.putAll(Arrays.asList(KeyValue.pair(hi, rawThere),
+                                   KeyValue.pair(hello, rawWorld)));
+
+        assertEquals(2, collector.collected().size());
+
+        // First entry
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+        final Headers headers0 = collector.collected().get(0).headers();
+        assertEquals(1, headers0.toArray().length);
+        assertEquals("value1", new 
String(headers0.lastHeader("key1").value()));
+
+        // Second entry
+        assertEquals(hello, collector.collected().get(1).key());
+        assertArrayEquals(world.value(), (byte[]) 
collector.collected().get(1).value());
+        assertEquals(98L, collector.collected().get(1).timestamp());
+        final Headers headers1 = collector.collected().get(1).headers();
+        assertEquals(1, headers1.toArray().length);
+        assertEquals("value2", new 
String(headers1.lastHeader("key2").value()));
+    }
+
+    @Test
+    public void shouldPropagateDelete() {
+        store.put(hi, rawThere);
+        store.delete(hi);
+        assertEquals(0L, root.approximateNumEntries());
+        assertNull(root.get(hi));
+    }
+
+    @Test
+    public void shouldReturnOldValueOnDelete() {
+        store.put(hi, rawThere);
+        assertEquals(rawThere, store.delete(hi));
+    }
+
+    @Test
+    public void shouldLogKeyNullOnDelete() {
+        store.put(hi, rawThere);
+        store.delete(hi);
+
+        assertEquals(2, collector.collected().size());
+
+        // First record is the put
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+        final Headers headers0 = collector.collected().get(0).headers();
+        assertEquals(1, headers0.toArray().length);
+        assertEquals("value1", new 
String(headers0.lastHeader("key1").value()));
+
+        // Second record is the delete
+        assertEquals(hi, collector.collected().get(1).key());
+        assertNull(collector.collected().get(1).value());
+        assertEquals(0L, collector.collected().get(1).timestamp());
+        assertEquals(0, 
collector.collected().get(1).headers().toArray().length);
+    }
+
+    @Test
+    public void shouldWriteToInnerOnPutIfAbsentNoPreviousValue() {
+        store.putIfAbsent(hi, rawThere);
+        assertEquals(rawThere, root.get(hi));
+    }
+
+    @Test
+    public void shouldNotWriteToInnerOnPutIfAbsentWhenValueForKeyExists() {
+        store.put(hi, rawThere);
+        store.putIfAbsent(hi, rawWorld);
+        assertEquals(rawThere, root.get(hi));
+    }
+
+    @Test
+    public void shouldWriteToChangelogOnPutIfAbsentWhenNoPreviousValue() {
+        store.putIfAbsent(hi, rawThere);
+
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+
+        final Headers headers = collector.collected().get(0).headers();
+        assertEquals(1, headers.toArray().length);
+        assertEquals("value1", new String(headers.lastHeader("key1").value()));
+    }
+
+    @Test
+    public void shouldNotWriteToChangeLogOnPutIfAbsentWhenValueForKeyExists() {
+        store.put(hi, rawThere);
+        store.putIfAbsent(hi, rawWorld);
+
+        assertEquals(1, collector.collected().size());
+        assertEquals(hi, collector.collected().get(0).key());
+        assertArrayEquals(there.value(), (byte[]) 
collector.collected().get(0).value());
+        assertEquals(97L, collector.collected().get(0).timestamp());
+        final Headers headers0 = collector.collected().get(0).headers();
+        assertEquals(1, headers0.toArray().length);
+        assertEquals("value1", new 
String(headers0.lastHeader("key1").value()));
+    }
+
+    @Test
+    public void shouldReturnCurrentValueOnPutIfAbsent() {
+        store.put(hi, rawThere);
+        assertEquals(rawThere, store.putIfAbsent(hi, rawWorld));
+    }
+
+    @Test
+    public void shouldReturnNullOnPutIfAbsentWhenNoPreviousValue() {
+        assertNull(store.putIfAbsent(hi, rawThere));
+    }
+
+    @Test
+    public void shouldReturnValueOnGetWhenExists() {
+        store.put(hello, rawWorld);
+        assertEquals(rawWorld, store.get(hello));
+    }
+
+    @Test
+    public void shouldReturnNullOnGetWhenDoesntExist() {
+        assertNull(store.get(hello));
+    }
+
+    @Test
+    public void shouldHandleNullValueInPut() {
+        final InternalMockProcessorContext<String, Long> context = 
mockContext();
+        context.setTime(42L);

Review Comment:
   Would we also need to set a header, and use the context header when writing 
into the changelog?



-- 
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]

Reply via email to