You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "vcrfxia (via GitHub)" <gi...@apache.org> on 2023/02/02 19:32:41 UTC

[GitHub] [kafka] vcrfxia opened a new pull request, #13188: KAFKA-14491: [5/N] Support basic operations for RocksDB versioned store

vcrfxia opened a new pull request, #13188:
URL: https://github.com/apache/kafka/pull/13188

   (This PR is stacked on https://github.com/apache/kafka/pull/13143; the first commit does not need to be reviewed separately.)
   
   Introduces the VersionedKeyValueStore interface proposed in KIP-889, along with the RocksDB-based implementation of the interface. This PR includes fully functional put, get, get-with-timestamp, and delete operations, but does not include the ability to restore records from changelog or surrounding store layers (for metrics or writing to the changelog). Those pieces will come in follow-up PRs.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1097979035


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)

Review Comment:
   Should we specify what "now" is with regard to history retention time?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}

Review Comment:
   ```suggestion
        * This operation is semantically equivalent to {@link #get(Object, long) #get(key, timestamp)}
   ```



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.

Review Comment:
   "latest" -> "current"
   
   I think current is clearer, as it make it more "obvious" that the "latest" version might only be in the history?
   
   



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.

Review Comment:
   Case (1) and (2) sound "similar"... can we rephrase? The "is a tombstone" phrase sounds potentially confusing to users?
   
   maybe -> `or null if there is no current record for this key.` 



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);

Review Comment:
   nit (avoid abbreviations): `lvsName`



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.

Review Comment:
   > with timestamp not exceeding the specified timestamp bound.
   
   Sound complicated... maybe -> `Get the latest record associated with this key and timestamp.`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(

Review Comment:
   Why do we need this helper -- seems it only called once. Can we inline it? The calling method is basically empty.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {
+
+        private static final int TIMESTAMP_SIZE = 8;
+
+        /**
+         * @return the timestamp, from the latest value store value bytes
+         */
+        long getTimestamp(final byte[] latestValue) {
+            return ByteBuffer.wrap(latestValue).getLong();
+        }
+
+        /**
+         * @return the actual record value, from the latest value store value bytes
+         */
+        byte[] getValue(final byte[] latestValue) {
+            final byte[] value = new byte[latestValue.length - TIMESTAMP_SIZE];
+            System.arraycopy(latestValue, TIMESTAMP_SIZE, value, 0, value.length);
+            return value;
+        }
+
+        /**
+         * @return the formatted bytes containing the provided {@code value} and {@code timestamp},
+         *         ready to be stored into the latest value store
+         */
+        byte[] from(final byte[] value, final long timestamp) {
+            if (value == null) {
+                throw new IllegalStateException("Cannot store tombstone in latest value");
+            }
+
+            return ByteBuffer.allocate(TIMESTAMP_SIZE + value.length)
+                .putLong(timestamp)
+                .put(value)
+                .array();
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the segments stores. This is a simple
+     * wrapper class for the static methods provided by {@link RocksDBVersionedStoreSegmentValueFormatter}.
+     * See the javadocs in {@link RocksDBVersionedStoreSegmentValueFormatter} for more.
+     */
+    static class SegmentValueSchema {

Review Comment:
   Not sure what value we get from this class?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete

Review Comment:
   Kafka has an implicit contract that `null` objects must be serialized to `null` bytes[] (for "plain" Kafka the impact of not following this contract is smaller, but for KS we would "fall apart" if the contract is violated.
   
   We should just say, "can be null; null is treated as delete".



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.

Review Comment:
   ```suggestion
        * Add a new record version for the specified timestamp associated with this key.
   ```



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),

Review Comment:
   Why do we pass an optional?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of

Review Comment:
   I would simplify: case (1) and (2) is basically the same -- "is a tombstone" is a weird formulation IMHO.
   
   `or {@code null} if no record for this key exist at the specified timestamp (note: this includes the case if the timestamp is older than history retention time).`



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized

Review Comment:
   This one is missing in `put()` and `delete()` above -- seems we also don't have this documented on every method on `KeyValueStore`...
   
   Maybe we should do a follow up PR to check all existing interfaces if the do document this exception?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);

Review Comment:
   It's hard to keep track of what it what... (everything has "value" in it's name...) this is the full "row" for a segment now, right?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(

Review Comment:
   "fall through" is not very descriptive -- can we find a better name?
   
   I needed to hop between the different method implementation to understand the code. The control flow is not easy (not sure if/how we could improve it right now...)



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.
+     *
+     * @param key           The key to fetch
+     * @param asOfTimestamp The timestamp bound. This bound is inclusive; if a record
+     *                      (for the specified key) exists with this timestamp, then
+     *                      this is the record that will be returned.
+     * @return The value and timestamp of the latest record associated with this key
+     *         satisfying the provided timestamp bound, or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record

Review Comment:
   cf comment on `delete()`



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedRecord.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.streams.KeyValue;
+
+import java.util.Objects;
+
+/**
+ * Combines a value from a {@link KeyValue} with a timestamp, for use as the return type

Review Comment:
   Well, we don't know how `VersionedRecord` is really used, do we? So the reference to `KeyValue` is a little odd?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or

Review Comment:
   latest -> current?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs

Review Comment:
   This comment is misleading, as we still might to put into "latest value store" inside `putFallThrough`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(

Review Comment:
   `putLatestValueStore` -> `maybePutLatestValueStore`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(

Review Comment:
   `maybePutSegments`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp

Review Comment:
   We not just do this optimization right away?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();

Review Comment:
   I think this needs explanation. We use `timestamp` in the line above as "validTo" for existing records (well, think it could only be a single one; if yes, maybe add comment about it), right? That's why we need to move the "oldest" record in the segment we insert into, one segment back, as is validTo could be reduced by the put() ?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {

Review Comment:
   Not sure what value this interface adds? Are you trying to hide other methods from `LogicalKeyValueSegment`? If yes, why? -- We usually only add a new interface if there is clear value to not introduce unnecessary boilerplate. 



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);

Review Comment:
   This is confusing... We pass in a `segmentValue` (which is bytes[] but I needed to double check) to get back a `SegmentValue` object. (Too many thing with the same name)
   
   In other stores, we use `V value` for the deserialized object and `byte[] rawValue` for the serialized bytes. Can we apply the same (or similar) naming pattern to make easier to read (we also run out of name here and you start to be "creative" using `sv`... -- as always, avoid abbreviations)



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);

Review Comment:
   passing `writeToOlderSegmentNeeded` might be unclear (I understand it)... not sure if could make it cleaner? Better variable name? (or add a comment)



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {

Review Comment:
   The control flow is hard to follow due to nesting... What about this:
   
   (1) rename `writeToOlderSegmentNeeded` to `includeValue` (and explain in a comment why we need to include it.
   (2) return directly inside if-blocks to avoid `else` and nesting:
   ```
   if (searchResult.validFrom() == timestamp) {
     ...
     return PutStatus(true, foundTs);
   }
   
   if (includeValue) { // we included a value and thus need to move to older segment plus update found segment
   
     return PutStatus(true, foundTs);
   }
   
   // just plain insert into found segment
   sv.insert(timestamp, value, searchResult.index());
   segment.put(key, sv.serialize());
   return PutStatus(true, foundTs);
   ```
   
   



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {

Review Comment:
   What value does this class add? Seems it's just boilerplate?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.

Review Comment:
   This part is long enough to go into a helper method?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of

Review Comment:
   Also add something like this?
   
   `Note: the record timestamp {@code r.ts} of the returned {@link VersionedRecord} may be smaller than the provided delete timestamp.`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;

Review Comment:
   It seems unnecessary to pass in `prevFoundTs` as it will always be the sentinel.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {

Review Comment:
   For this case `writeToOlderSegmentNeeded` is voided, indicating that we picked a "bad" variable name?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);

Review Comment:
   `latestValue` -> `latestValueAndTimestamp`



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {

Review Comment:
   The control flow is tricky here (I only understood after reading much more code).
   
   Add comment what `segment == null` implies (and why we don't update the "drop record sensor")



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.

Review Comment:
   Why not apply this optimization?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this

Review Comment:
   "earlier segment" -> or "latest value store", right?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);

Review Comment:
   `valueToMove` -> `rawValueToMove` (ie, w/o timestamp)?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(

Review Comment:
   Why is this method static? If forces us to have a long parameter list.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {

Review Comment:
   I am just wondering if this case is actually possible? Above you just say vaguely `depending on the circumstances of the fall-through` -- can you elaborate?
   
   The only case for "latest" is do understand would be the case when we actually insert into the "latest value store" instead of a segment. If we really insert into a segment, it seems we would always insert as earliest?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise

Review Comment:
   This comment about "why not middle" is gold! Thanks.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {
+
+        private static final int TIMESTAMP_SIZE = 8;
+
+        /**
+         * @return the timestamp, from the latest value store value bytes
+         */
+        long getTimestamp(final byte[] latestValue) {
+            return ByteBuffer.wrap(latestValue).getLong();
+        }
+
+        /**
+         * @return the actual record value, from the latest value store value bytes
+         */
+        byte[] getValue(final byte[] latestValue) {

Review Comment:
   `latestValue` -> `latestValueAndTimestamp` ?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(

Review Comment:
   Why `static` ?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {
+
+        private static final int TIMESTAMP_SIZE = 8;
+
+        /**
+         * @return the timestamp, from the latest value store value bytes
+         */
+        long getTimestamp(final byte[] latestValue) {

Review Comment:
   `latestValue` -> `latestValueAndTimestamp` ?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {

Review Comment:
   Why do we need this `if`? Seems we can just do
   ```
   segment.put(
     key,
     segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
   );
   ```
   
   If `value == null`, we still pass `null` as first argument.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {

Review Comment:
   Seems everything in this class could be static methods? If yes, we would not need to have an instance at hand and can eliminate a class member variable.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(

Review Comment:
   "fall through" is not very descriptive -- can we find a better name?
   
   I needed to hop between the different method implementation to understand the code. The control flow is not easy (not sure if/how we could improve it right now...)



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {

Review Comment:
   `LatestValueSchema` -> `LatestValueFormatter` ?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {

Review Comment:
   nit: add comment about `olderSegment == null` case



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102252265


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestamps() {
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 1);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 5);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldPutIntoMultipleSegments() {
+        putToStore("k", null, SEGMENT_INTERVAL - 20);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 15);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentDuringPut() {
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+        putToStore("k", "vn1", SEGMENT_INTERVAL - 1);
+        putToStore("k", "vp1", SEGMENT_INTERVAL + 1);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 5, "vp1", SEGMENT_INTERVAL + 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 1, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentWithNullsDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldFallThroughToExistingOlderSegmentAsLatestDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 12);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+    }
+
+    @Test
+    public void shouldPutNonLatestTombstoneIntoNewSegmentWithValidTo() {
+        putToStore("k", "vp30", SEGMENT_INTERVAL + 30);
+        putToStore("k", null, SEGMENT_INTERVAL - 10); // this put should result in tombstone with validTo=SEGMENT_INTERVAL+30

Review Comment:
   This is the degenerated segment case, right? So `validTo` should be `validfrom == SEGMENT_INTERVAL - 10`? (or do I remember incorrectly how degenerated segments work?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1098139462


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)

Review Comment:
   The line above says "where the provided timestamp bound is within history retention of the current observed stream team." Do you think that needs additional clarification?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of

Review Comment:
   OK. The reason I had separated the cases (1) and (2) was because there was confusion about this case (latest record version is a tombstone) during initial discussion leading up to the KIP and it was requested that the javadocs be explicit about this. But a lot has changed since those initial discussions so let me update it.
   
   Also, I assume it's fine to update javadocs proposed in the KIP after it's been accepted, right? Should I go back and update the KIP too?



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete

Review Comment:
   It makes sense that null object must always be serialized to null bytes, but it's also possible that a non-null object serializes to null bytes, right? My intention with this javadoc was to clarify that it's the serialization that determines whether the put() is treated as a delete -- even if the value itself is not null, as long as its serialization is null then it counts as a delete.
   
   If you think this clarification is more confusing than its worth, I can remove it.



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized

Review Comment:
   Sure, I can make a quick pass for this in a follow-up after this is merged.



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.

Review Comment:
   I worry that "associated with this timestamp" could be confusing because the returned record will often have timestamp smaller than the timestamp bound. (This happens whenever there is no record version with the exact provided timestamp.)
   
   What do you think about 
   
   > Get the record associated with this key, as of the specified timestamp (i.e., existing record with the largest timestamp not exceeding the provided bound). 
   
   ? I'll make the update.



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedRecord.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.streams.KeyValue;
+
+import java.util.Objects;
+
+/**
+ * Combines a value from a {@link KeyValue} with a timestamp, for use as the return type

Review Comment:
   I copied from ValueAndTimestamp ([link](https://github.com/apache/kafka/blob/6d11261d5deaca300e273bebe309f9e4f814f815/streams/src/main/java/org/apache/kafka/streams/state/ValueAndTimestamp.java#L24)) 🤷 
   
   I'll remove it.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {
+
+        private static final int TIMESTAMP_SIZE = 8;
+
+        /**
+         * @return the timestamp, from the latest value store value bytes
+         */
+        long getTimestamp(final byte[] latestValue) {
+            return ByteBuffer.wrap(latestValue).getLong();
+        }
+
+        /**
+         * @return the actual record value, from the latest value store value bytes
+         */
+        byte[] getValue(final byte[] latestValue) {
+            final byte[] value = new byte[latestValue.length - TIMESTAMP_SIZE];
+            System.arraycopy(latestValue, TIMESTAMP_SIZE, value, 0, value.length);
+            return value;
+        }
+
+        /**
+         * @return the formatted bytes containing the provided {@code value} and {@code timestamp},
+         *         ready to be stored into the latest value store
+         */
+        byte[] from(final byte[] value, final long timestamp) {
+            if (value == null) {
+                throw new IllegalStateException("Cannot store tombstone in latest value");
+            }
+
+            return ByteBuffer.allocate(TIMESTAMP_SIZE + value.length)
+                .putLong(timestamp)
+                .put(value)
+                .array();
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the segments stores. This is a simple
+     * wrapper class for the static methods provided by {@link RocksDBVersionedStoreSegmentValueFormatter}.
+     * See the javadocs in {@link RocksDBVersionedStoreSegmentValueFormatter} for more.
+     */
+    static class SegmentValueSchema {

Review Comment:
   I had wanted to abstract details of the schema so that the put() and get() logic is reusable (for example, when we implement an in-memory versioned store) but I think it's a premature optimization. I'll get rid of this for now, and we can cross this bridge again when the time comes.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {

Review Comment:
   Nope. There is a second (completely different) implementation of this interface used for restore -- see https://github.com/apache/kafka/pull/13189.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {

Review Comment:
   The purpose of this interface is not to hide methods. There is a second (completely different) implementation of this interface used for restore -- see https://github.com/apache/kafka/pull/13189.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(

Review Comment:
   That's true for now but it's called again as part of restore -- see https://github.com/apache/kafka/pull/13189.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);

Review Comment:
   Yes, that's correct. Let me go through and try to make this more consistent / clearer.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // insert as latest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(timestamp, foundTs, value);
+                    segment.put(key, sv.serialize());
+                } else {
+                    // insert as earliest
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsEarliest(timestamp, value);
+                    segment.put(key, sv.serialize());
+                }
+            }
+        }
+    }
+
+    /**
+     * Bytes layout for the value portion of rows stored in the latest value store. The layout is
+     * a fixed-size timestamp concatenated with the actual record value.
+     */
+    static class LatestValueSchema {

Review Comment:
   Sure, I'll update it to be static. 



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp

Review Comment:
   I thought the logic in this method was already complicated enough, and didn't want to add another variable for tracking state :) 
   
   Also, this optimization is low on my priority list since I don't expect it to affect performance much in practice. The effect of this optimization is to skip past some empty segments. RocksDB's use of bloom filters means that identifying a segment as empty (i.e., key does not exist in RocksDB) should be faster than a regular key lookup. 



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(

Review Comment:
   See above.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();

Review Comment:
   > That's why we need to move the "oldest" record in the segment we insert into, one segment back, as is validTo could be reduced by the put() ?
   
   Yes, exactly. I will add a comment.
   
   > well, think it could only be a single one
   
   This is correct too. The validTo timestamp is only updated for one existing record version.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);

Review Comment:
   Sure, I like that suggestion. I'll go through and update everything.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(

Review Comment:
   Having `doPut()` static makes it clearer what the method potentially updates -- it's clear from the parameter list that the method potentially writes to the latest value store and segments store, and calls the expired record sensor, but nothing else. (For example, `observedStreamTime` is not updated as part of the `doPut()` method.)
   
   Also, this is admittedly a ways off but I am hoping to reuse this put() (and also the get()) logic for the in-memory versioned store implementation once we get there, which will be easier to do if the logic is in a static method. (We'll probably want to move it into a shared helper class at that point too.) 
   
   I don't feel strongly about this, though. If you think this is harder to read, I'll make `doPut()` and its helper methods non-static. WDYT?



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {

Review Comment:
   Sure, I'll make the changes.
   
   My preference is to name the variable `writeToOlderSegmentMaybeNeeded` rather than `includeValue` since I think it's less natural (/ harder to explain) why we use a variable named `includeValue` to decide whether to write to an older segment or not, than it is to explain why we used a variable named `writeToOlderSegmentMaybeNeeded` to decide whether to include a value while searching. I'll go with that for now but can easily update it if you think `writeToOlderSegmentMaybeNeeded` with generous comments is still confusing.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {

Review Comment:
   Heh, that's very astute of you to notice. Let's continue the conversation in your other comment below.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {

Review Comment:
   Nope, latest is possible for a segment as well. It has to do with the "gaps" which may be formed from degenerate segments. Here's a example:
   * segment interval is 100
   * put tombstone with `ts=50`, so there is a (degenerate) segment with `minTs=nextTs=50` (containing a single tombstone).
   * put record with `ts=150`, so it goes to the latest value store
   * put record with `ts=90`, so now there is a new segment with `minTs=90`, `nextTs=150`, containing the new record.
   * put record with `ts=80`: this put operation will "fall-through" with `foundTs=90`, so the segment it finds will be the one which contains a tombstone (with `minTs=nextTs=50`), so `foundNextTs=50` which is less than the insertion timestamp (80)`.
   
   Do you think this is worth putting into a comment in the code? It's a very long explanation, and requires knowledge of "degenerate segments" which this class currently does not require. I'll leave it out for now.
   
    



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),

Review Comment:
   This method is shared with the logic for restoring from changelog (see https://github.com/apache/kafka/pull/13189). When restoring from changelog, we do not want to record expired records, and so we pass `Optional.empty()`.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;

Review Comment:
   That's correct. I was going for consistency among the three helper methods to `doPut()`, but I'll remove it since it seems it's confusing.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);

Review Comment:
   Will add a comment.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(

Review Comment:
   Would it be better if `putFallThrough()` was simply inlined? Then we would not need to figure out a better name for it. (I trust your opinion as a new reader to this code, since I've been working with it for far too long already, haha)
   
   Originally I had all three methods (`putLatestValue()`, `putSegments()`, and `putFallThrough()`) inlined but thought that having separate methods was easier to read. If that's not the case, we could even inline all of them and do away with `PutStatus` entirely.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.

Review Comment:
   Same as above.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this

Review Comment:
   No, this case is only hit if we've already found a segment which contains data later than the current insertion timestamp, so the new record will never be put into the latest value store.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {

Review Comment:
   You're right! This is leftover from a previous version of the code. I will simplify it.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1101882173


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete

Review Comment:
   Hm... I found this [comment](https://github.com/apache/kafka/blob/083e11a22ca9966ed010acdd5705351ab4300b52/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueAndTimestampSerializer.java#L87-L89) in the code the other day, which gives the impression that it is valid to serialize a non-null value to null bytes:
   > // Serializing non-null values to null can be useful when working with Optional-like values
   > // where the Optional.empty case is serialized to null.
   > // See the discussion here: https://github.com/apache/kafka/pull/7679
   
   Is this no longer true?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1100892271


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {

Review Comment:
   Not sure if I can follow. Maybe some pieces are missing? I don't see code (did I miss it), that uses a `VersionedStoreSegment` and does not care which implementation is it? I only see two implementation of this interface (and two implementations of the `VersionedStoreClient` interface), but they seem to be "isolated / next to each other". What do I miss?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1101902156


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.

Review Comment:
   Hm. Currently "validFrom" and "validTo" are not mentioned anywhere in the public-facing interfaces (VersionedKeyValueStore, VersionedRecord, etc) in javadocs or as parameter names; they are only "officially" introduced as concepts in the javadocs for RocksDBVersionedStore and its implementation helpers. I'm a bit hesitant to introduce these concepts in the public-facing docs at this time if the only purpose is to clarify what's returned from `get(key, asOfTimestamp)`, particularly because during KIP discussion one person mentioned that these terms did not feel intuitive to them. If you feel strongly though, I can move the explanation of "validity interval" including "validFrom" and "validTo" from the javadocs for RocksDBVersionedStore to VersionedKeyValueStore itself, and then update this javadoc.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1099321787


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(

Review Comment:
   Renamed to `finishPut()` for now.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1100893004


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),

Review Comment:
   Nevermind. Found it.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102242415


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);

Review Comment:
   That's an interesting one... User can not also query using "future timestamp" -- nothing wrong with it; just interesting "side effect" :) 



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1105186412


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {

Review Comment:
   > Can we add a test (maybe in a follow up PR, with retention time zero)?
   
   Added in https://github.com/apache/kafka/pull/13243.
   
   > Having no segments and segment_interval = 0 (?) seems to be a corner case we should also support.
   
   Let's discuss in the other PR about whether it makes sense to get rid of the segments store entirely in this case or not.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1100896272


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {

Review Comment:
   Never mind. Found it.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102243061


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);

Review Comment:
   Should we not first do `putToStore("k", "v", BASE_TIMESTAMP);` to ensure that the `put(null)` really deletes `v` ?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102238142


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.

Review Comment:
   Fair point.
   
   > particularly because during KIP discussion one person mentioned that these terms did not feel intuitive to them
   
   I cannot remember this; personally I always found it very intuitive (also because it's use in temporal tables in SQL), but of course this could just be me.
   
   I agree that we should not use it only here, but rather introduce it "globally" or not use it at all. It's "just" JavaDocs and just easy to change in follow up releases if thy cause confusion, so I don't feel strong about it and leave it to your judgement.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1103076551


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete

Review Comment:
   OK, thanks for the additional context. I'll update the javadoc as you suggested.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1098117328


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {

Review Comment:
   I am just wondering if this case is actually possible? Above you just say vaguely `depending on the circumstances of the fall-through` -- can you elaborate?
   
   The only case for "latest" I do understand would be the case when we actually insert into the "latest value store" instead of a segment. If we really insert into a segment, it seems we would always insert as earliest?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1100539790


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of

Review Comment:
   Totally -- JavaDocs are not public API. it's well, _docs_; if we have them in the KIP, it's just a template/placeholder. Not need to go back to the KIP (otherwise we could never change JavaDocs without altering older KIPs 😂)



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)

Review Comment:
   Oh sorry. Seems I missed it... disregard my original comment.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final byte[] latestValue = versionedStoreClient.getLatestValue(key);
+        if (latestValue != null) {
+            foundTs = latestValueSchema.getTimestamp(latestValue);
+            if (timestamp >= foundTs) {
+                // new record belongs in the latest value store
+                if (timestamp > foundTs) {
+                    // move existing latest value into segment.
+                    // it's important that this step happens before the update to the latest value
+                    // store. if there is a partial failure (this step succeeds but the update to
+                    // the latest value store fails), updating the segment first means there will
+                    // not be data loss. (rather, there will be duplicated data which is fine as
+                    // it can/will be reconciled later.)
+                    final long segmentId = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final T segment = versionedStoreClient.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+                    if (segment != null) {
+                        final byte[] valueToMove = latestValueSchema.getValue(latestValue);
+                        final byte[] segmentValueBytes = segment.get(key);
+                        if (segmentValueBytes == null) {
+                            segment.put(
+                                key,
+                                segmentValueSchema
+                                    .newSegmentValueWithRecord(valueToMove, foundTs, timestamp)
+                                    .serialize()
+                            );
+                        } else {
+                            final SegmentValue segmentValue = segmentValueSchema.deserialize(segmentValueBytes);
+                            segmentValue.insertAsLatest(foundTs, timestamp, valueToMove);
+                            segment.put(key, segmentValue.serialize());
+                        }
+                    }
+                }
+
+                // update latest value store
+                if (value != null) {
+                    versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+                } else {
+                    versionedStoreClient.deleteLatestValue(key);
+                }
+                return new PutStatus(true, foundTs);
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putSegments(
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long prevFoundTs
+    ) {
+        // initialize with current foundTs value
+        long foundTs = prevFoundTs;
+
+        final List<T> segments = versionedStoreClient.getReverseSegments(timestamp);
+        for (final T segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {
+                    // this segment (and all earlier segments) does not contain records affected by
+                    // this put. insert into the segment specified by foundTs (i.e., the next
+                    // phase of the put() procedure) and conclude the procedure.
+                    return new PutStatus(false, foundTs);
+                }
+
+                final long foundMinTs = segmentValueSchema.getMinTimestamp(segmentValue);
+                if (foundMinTs <= timestamp) {
+                    // the record being inserted belongs in this segment.
+                    // insert and conclude the procedure.
+                    final long segmentIdForTimestamp = versionedStoreClient.segmentIdForTimestamp(timestamp);
+                    final boolean writeToOlderSegmentNeeded = segmentIdForTimestamp != segment.id();
+
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    final SegmentSearchResult searchResult = sv.find(timestamp, writeToOlderSegmentNeeded);
+
+                    if (searchResult.validFrom() == timestamp) {
+                        // this put() replaces an existing entry, rather than adding a new one
+                        sv.updateRecord(timestamp, value, searchResult.index());
+                        segment.put(key, sv.serialize());
+                    } else {
+                        if (writeToOlderSegmentNeeded) {
+                            // existing record needs to be moved to an older segment.
+                            // it's important that this step happens before updating the current
+                            // segment. if there is a partial failure (this step succeeds but the
+                            // update to the current segment fails), updating the older segment
+                            // first means there will not be data loss. (rather, there will be
+                            // duplicated data which is fine as it can/will be reconciled later.)
+                            final T olderSegment = versionedStoreClient
+                                .getOrCreateSegmentIfLive(segmentIdForTimestamp, context, observedStreamTime);
+                            if (olderSegment != null) {
+                                final byte[] olderSegmentValue = olderSegment.get(key);
+                                if (olderSegmentValue == null) {
+                                    olderSegment.put(
+                                        key,
+                                        segmentValueSchema.newSegmentValueWithRecord(
+                                            searchResult.value(), searchResult.validFrom(), timestamp
+                                        ).serialize()
+                                    );
+                                } else {
+                                    final SegmentValue olderSv = segmentValueSchema.deserialize(olderSegmentValue);
+                                    olderSv.insertAsLatest(searchResult.validFrom(), timestamp, searchResult.value());
+                                    olderSegment.put(key, olderSv.serialize());
+                                }
+                            }
+
+                            // update in newer segment (replace the record that was just moved with the new one)
+                            sv.updateRecord(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        } else {
+                            sv.insert(timestamp, value, searchResult.index());
+                            segment.put(key, sv.serialize());
+                        }
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                if (foundMinTs < observedStreamTime - historyRetention) {
+                    // the record being inserted does not affect version history. discard and return
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return new PutStatus(true, foundTs);
+                }
+
+                // it's possible the record belongs in this segment, but also possible it belongs
+                // in an earlier segment. mark as tentative and continue. as an optimization, this
+                // code could be updated to skip forward to the segment containing foundMinTs.
+                foundTs = foundMinTs;
+            }
+        }
+        return new PutStatus(false, foundTs);
+    }
+
+    private static <T extends VersionedStoreSegment> void putFallThrough(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp,
+        final long foundTs
+    ) {
+        if (foundTs == SENTINEL_TIMESTAMP) {
+            // insert into latest value store
+            if (value != null) {
+                versionedStoreClient.putLatestValue(key, latestValueSchema.from(value, timestamp));
+            } else {
+                // tombstones are not inserted into the latest value store. insert into segment instead.
+                // the specific segment to insert to is determined based on the tombstone's timestamp
+                final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                    versionedStoreClient.segmentIdForTimestamp(timestamp), context, observedStreamTime);
+                if (segment == null) {
+                    if (expiredRecordSensor.isPresent()) {
+                        expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                        LOG.warn("Skipping record for expired put.");
+                    }
+                    return;
+                }
+
+                final byte[] segmentValue = segment.get(key);
+                if (segmentValue == null) {
+                    // in this special case where the latest record version (for a particular key)
+                    // is a tombstone, and the segment that the tombstone belongs in contains no
+                    // record versions for this key, create a new "degenerate" segment with the
+                    // tombstone's timestamp as both validFrom and validTo timestamps for the segment
+                    segment.put(key, segmentValueSchema.newSegmentValueWithRecord(null, timestamp, timestamp).serialize());
+                } else {
+                    // insert as latest, since foundTs = sentinel means nothing later exists
+                    if (segmentValueSchema.getNextTimestamp(segmentValue) == timestamp) {
+                        // next timestamp equal to put() timestamp already represents a tombstone,
+                        // so no additional insertion is needed in this case
+                        return;
+                    }
+                    final SegmentValue sv = segmentValueSchema.deserialize(segmentValue);
+                    sv.insertAsLatest(
+                        segmentValueSchema.getNextTimestamp(segmentValue),
+                        timestamp,
+                        null
+                    );
+                    segment.put(key, sv.serialize());
+                }
+            }
+        } else {
+            // insert into segment corresponding to foundTs, as foundTs represents the validTo
+            // timestamp of the current put.
+            // the new record is either the earliest or the latest in this segment, depending on the
+            // circumstances of the fall-through. (it cannot belong in the middle because otherwise
+            // putSegments() above would have identified a segment for which
+            // minTimestamp <= timestamp < nextTimestamp, and putSegments would've completed the
+            // put procedure without reaching this fall-through case.)
+            final T segment = versionedStoreClient.getOrCreateSegmentIfLive(
+                versionedStoreClient.segmentIdForTimestamp(foundTs), context, observedStreamTime);
+            if (segment == null) {
+                if (expiredRecordSensor.isPresent()) {
+                    expiredRecordSensor.get().record(1.0d, ProcessorContextUtils.currentSystemTime(context));
+                    LOG.warn("Skipping record for expired put.");
+                }
+                return;
+            }
+
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue == null) {
+                if (value != null) {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(value, timestamp, foundTs).serialize()
+                    );
+                } else {
+                    segment.put(
+                        key,
+                        segmentValueSchema.newSegmentValueWithRecord(null, timestamp, foundTs).serialize()
+                    );
+                }
+            } else {
+                final long foundNextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (foundNextTs <= timestamp) {

Review Comment:
   Thanks for the explanation. I think a short comment like `latest is possible if the found segment is a degenerated segment; cf RocksDBVersionedStoreSegmentValueFormatter.java` should be sufficient (it seem for regular segments it's never possible, so just pointing out this corner case without given details should be enough to avoid people scratching their head about the "latest" case).



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(

Review Comment:
   Personally, I find it hard to read if there are long parameter lists.



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.

Review Comment:
   I think the concept of a "validity interval" is easiest to understand?
   
   > Get the record associated with this key, as of the specified timestamp (i.e., existing record with r.validFrom <= timestamp < r.validTo; r.validFrom == r.timestamp and r.validTo is the timestamp of the next record).



##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete

Review Comment:
   > but it's also possible that a non-null object serializes to null bytes, right
   
   That also violations of the contract (it must be null-to-null and non-null-to-non-null), because downstream it would imply that a null-byte array is not deserialized to a null object. (Note: on restore, we actually don't deserialize data and would treat the null-byte array as delete, so stuff breaks if it was not a tombstone...)
   
   I understand your intent, but putting it into the JavaDocs might give the wrong impression that it's ok to do this, while it's not...



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(

Review Comment:
   Cool. Having the helpers is useful to make it easier to understand the control flow... (inlining does not sound like the best idea).



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102246615


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);

Review Comment:
   Same: should we verify that `k` is also not there for `BASE_TIMESTAMP - 1`?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1103079190


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {

Review Comment:
   > Can we add a test (maybe in a follow up PR, with retention time zero)?
   
   Yes, that's a good point. I'll do it in a follow-up PR because I think we'll need a minor code change as well in order to not instantiate the segments store in this scenario.
   
   > it seems we don't test if delete() does return the right "old value"; can we also add some tests for it?
   
   Added a new `shouldDelete()`.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);

Review Comment:
   This test is for validating that the store is able to handle a tombstone as the first thing put to the store (and continue to insert earlier records out-of-order afterwards). There are other tests (e.g., `shouldPutRepeatTimestampAsLatest()` and `shouldPutRepeatTimestamps()`) which test the scenario you've requested.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);

Review Comment:
   > User can not also query using "future timestamp"
   
   "can not" -> "can now". Yup!



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestamps() {
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 1);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 5);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldPutIntoMultipleSegments() {
+        putToStore("k", null, SEGMENT_INTERVAL - 20);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 15);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentDuringPut() {
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+        putToStore("k", "vn1", SEGMENT_INTERVAL - 1);
+        putToStore("k", "vp1", SEGMENT_INTERVAL + 1);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 5, "vp1", SEGMENT_INTERVAL + 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 1, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentWithNullsDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldFallThroughToExistingOlderSegmentAsLatestDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 12);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+    }
+
+    @Test
+    public void shouldPutNonLatestTombstoneIntoNewSegmentWithValidTo() {
+        putToStore("k", "vp30", SEGMENT_INTERVAL + 30);
+        putToStore("k", null, SEGMENT_INTERVAL - 10); // this put should result in tombstone with validTo=SEGMENT_INTERVAL+30

Review Comment:
   No, this is not degenerate. It's not degenerate because there is a well-defined validTo in this case, from the record in the latest value store (`vp30` with timestamp `SEGMENT_INTERVAL + 30`). Degenerate segments only form if the latest value is a tombstone, which means there is no validTo we can use for the tombstone.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestamps() {
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 1);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 5);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldPutIntoMultipleSegments() {
+        putToStore("k", null, SEGMENT_INTERVAL - 20);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 15);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentDuringPut() {
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+        putToStore("k", "vn1", SEGMENT_INTERVAL - 1);
+        putToStore("k", "vp1", SEGMENT_INTERVAL + 1);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 5, "vp1", SEGMENT_INTERVAL + 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 1, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentWithNullsDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldFallThroughToExistingOlderSegmentAsLatestDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 12);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+    }
+
+    @Test
+    public void shouldPutNonLatestTombstoneIntoNewSegmentWithValidTo() {
+        putToStore("k", "vp30", SEGMENT_INTERVAL + 30);
+        putToStore("k", null, SEGMENT_INTERVAL - 10); // this put should result in tombstone with validTo=SEGMENT_INTERVAL+30
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn1", SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 2);
+
+        verifyGetValueFromStore("k", "vp30", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 10, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 1, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 10);
+    }
+
+    @Test
+    public void shouldGetFromOlderSegments() {
+        // use a different key to create three different segments
+        putToStore("ko", null, SEGMENT_INTERVAL - 10);
+        putToStore("ko", null, 2 * SEGMENT_INTERVAL - 10);
+        putToStore("ko", null, 3 * SEGMENT_INTERVAL - 10);
+
+        // return null after visiting all segments
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 20);
+
+        // insert data to create non-empty (first) segment
+        putToStore("k", "v", SEGMENT_INTERVAL - 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 15, "v", SEGMENT_INTERVAL - 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+
+        // insert data to create non-empty (third) segment
+        putToStore("k", "v2", 3 * SEGMENT_INTERVAL - 20);
+        putToStore("k", null, 3 * SEGMENT_INTERVAL - 10);
+
+        // presence of non-empty later segment does not affect results of getting from earlier segment
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 15, "v", SEGMENT_INTERVAL - 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+    }
+
+    @Test
+    public void shouldNotGetExpired() {

Review Comment:
   Hm, I started adding a new test case for this but then realized we can't actually test this without querying the physical segments store itself. Say we try to set up the test like so:
   * put v1 with ts=0
   * put v3 with ts=100
   * advance stream time so ts=50 is expired
   * put v2 with ts=40, which updates the validTo of the first put to something expired
   
   Now what should the validation be? If we query at ts=40, of course we get null because everything prior to ts=50 is expired (the existing test already checks this). If we query at ts=60, of course we get v2 but this happens regardless of whether v1 is expired or not. So the only way to validate that v1 is actually expired is to check the underlying store, but that responsibility seems like it belongs with LogicalKeyValueSegmentsTest instead. 



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);

Review Comment:
   Added (as above).



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);

Review Comment:
   Sure, I've added this check into each of the verifications in this test.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestamps() {
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 1);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 5);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);

Review Comment:
   I added comments to the puts regarding why there are so many lines (they test different combinations of null vs non-null being replaced with null vs non-null, and also with timestamps across segments vs in the same segment vs the latest value store).
   
   I didn't add anything regarding the current state at each step since that feels a bit overkill. The way to figure out the end state is to ignore everything that says "to_be_replace" and then de-dup the few remaining rows. If we want to further improve readability, we can break this up into separate tests but my gut says that's unnecessary at the moment.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax merged pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax merged PR #13188:
URL: https://github.com/apache/kafka/pull/13188


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102246409


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);

Review Comment:
   Should we verify that `"to_be_replaced"` was not move and thus we cannot query it with `BASE_TIMESTAMP - 1`?



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);

Review Comment:
   Should we verify that `"to_be_replaced"` was not move and thus we cannot query it with `BASE_TIMESTAMP - 1`?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102242415


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);

Review Comment:
   That's an interesting one... User can now also query using "future timestamp" -- nothing wrong with it; just interesting "side effect" :) 



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1101897601


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp
+        );
+
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+    }
+
+    @Override
+    public VersionedRecord<byte[]> delete(final Bytes key, final long timestamp) {
+        final VersionedRecord<byte[]> existingRecord = get(key, timestamp);
+        put(key, null, timestamp);
+        return existingRecord;
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key) {
+        // latest value (if present) is guaranteed to be in the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            return new VersionedRecord<>(
+                latestValueSchema.getValue(latestValue),
+                latestValueSchema.getTimestamp(latestValue)
+            );
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public VersionedRecord<byte[]> get(final Bytes key, final long asOfTimestamp) {
+
+        if (asOfTimestamp < observedStreamTime - historyRetention) {
+            // history retention has elapsed. return null for predictability, even if data
+            // is still present in store.
+            return null;
+        }
+
+        // first check the latest value store
+        final byte[] latestValue = latestValueStore.get(key);
+        if (latestValue != null) {
+            final long latestTimestamp = latestValueSchema.getTimestamp(latestValue);
+            if (latestTimestamp <= asOfTimestamp) {
+                return new VersionedRecord<>(latestValueSchema.getValue(latestValue), latestTimestamp);
+            }
+        }
+
+        // check segment stores
+        final List<LogicalKeyValueSegment> segments = segmentStores.segments(asOfTimestamp, Long.MAX_VALUE, false);
+        for (final LogicalKeyValueSegment segment : segments) {
+            final byte[] segmentValue = segment.get(key);
+            if (segmentValue != null) {
+                final long nextTs = segmentValueSchema.getNextTimestamp(segmentValue);
+                if (nextTs <= asOfTimestamp) {
+                    // this segment contains no data for the queried timestamp, so earlier segments
+                    // cannot either
+                    return null;
+                }
+
+                if (segmentValueSchema.getMinTimestamp(segmentValue) > asOfTimestamp) {
+                    // the segment only contains data for after the queried timestamp. skip and
+                    // continue the search to earlier segments. as an optimization, this code
+                    // could be updated to skip forward to the segment containing the minTimestamp
+                    // in the if-condition above.
+                    continue;
+                }
+
+                // the desired result is contained in this segment
+                final SegmentSearchResult searchResult =
+                    segmentValueSchema.deserialize(segmentValue).find(asOfTimestamp, true);
+                if (searchResult.value() != null) {
+                    return new VersionedRecord<>(searchResult.value(), searchResult.validFrom());
+                } else {
+                    return null;
+                }
+            }
+        }
+
+        // checked all segments and no results found
+        return null;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public void flush() {
+        // order shouldn't matter since failure to flush is a fatal exception
+        segmentStores.flush();
+        latestValueStore.flush();
+    }
+
+    @Override
+    public void close() {
+        open = false;
+
+        // close latest value store first so that calls to get() immediately begin to fail with
+        // store not open, as all calls to get() first get() from latest value store
+        latestValueStore.close();
+        segmentStores.close();
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        this.context = context;
+
+        final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context);
+        final String threadId = Thread.currentThread().getName();
+        final String taskName = context.taskId().toString();
+
+        expiredRecordSensor = TaskMetrics.droppedRecordsSensor(
+            threadId,
+            taskName,
+            metrics
+        );
+
+        metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId());
+
+        latestValueStore.openDB(context.appConfigs(), context.stateDir());
+        segmentStores.openExisting(context, observedStreamTime);
+
+        final File positionCheckpointFile = new File(context.stateDir(), name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+
+        // register and possibly restore the state from the logs
+        stateStoreContext.register(
+            root,
+            (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position)
+        );
+
+        open = true;
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    // VisibleForTesting
+    void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        throw new UnsupportedOperationException("not yet implemented");
+    }
+
+    /**
+     * Generic interface for segment stores. See {@link VersionedStoreClient} for use.
+     */
+    interface VersionedStoreSegment {
+
+        /**
+         * @return segment id
+         */
+        long id();
+
+        void put(Bytes key, byte[] value);
+
+        byte[] get(Bytes key);
+    }
+
+    /**
+     * Extracts all operations required for writing to the versioned store (via
+     * {@link #put(Bytes, byte[], long)}) into a generic client interface, so that the same
+     * {@code put(...)} logic can be shared during regular store operation and during restore.
+     *
+     * @param <T> the segment type used by this client
+     */
+    interface VersionedStoreClient<T extends VersionedStoreSegment> {
+
+        /**
+         * @return the contents of the latest value store, for the given key
+         */
+        byte[] getLatestValue(Bytes key);
+
+        /**
+         * Puts the provided key and value into the latest value store.
+         */
+        void putLatestValue(Bytes key, byte[] value);
+
+        /**
+         * Deletes the existing value (if any) from the latest value store, for the given key.
+         */
+        void deleteLatestValue(Bytes key);
+
+        /**
+         * @return the segment with the provided id, or {@code null} if the segment is expired
+         */
+        T getOrCreateSegmentIfLive(long segmentId, ProcessorContext context, long streamTime);
+
+        /**
+         * @return all segments in the store which contain timestamps at least the provided
+         *         timestamp bound, in reverse order by segment id (and time), i.e., such that
+         *         the most recent segment is first
+         */
+        List<T> getReverseSegments(long timestampFrom);
+
+        /**
+         * @return the segment id associated with the provided timestamp
+         */
+        long segmentIdForTimestamp(long timestamp);
+    }
+
+    /**
+     * Client for writing into (and reading from) this persistent {@link RocksDBVersionedStore}.
+     */
+    private class RocksDBVersionedStoreClient implements VersionedStoreClient<LogicalKeyValueSegment> {
+
+        @Override
+        public byte[] getLatestValue(final Bytes key) {
+            return latestValueStore.get(key);
+        }
+
+        @Override
+        public void putLatestValue(final Bytes key, final byte[] value) {
+            latestValueStore.put(key, value);
+        }
+
+        @Override
+        public void deleteLatestValue(final Bytes key) {
+            latestValueStore.delete(key);
+        }
+
+        @Override
+        public LogicalKeyValueSegment getOrCreateSegmentIfLive(final long segmentId, final ProcessorContext context, final long streamTime) {
+            return segmentStores.getOrCreateSegmentIfLive(segmentId, context, streamTime);
+        }
+
+        @Override
+        public List<LogicalKeyValueSegment> getReverseSegments(final long timestampFrom) {
+            return segmentStores.segments(timestampFrom, Long.MAX_VALUE, false);
+        }
+
+        @Override
+        public long segmentIdForTimestamp(final long timestamp) {
+            return segmentStores.segmentId(timestamp);
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> void doPut(
+        final LatestValueSchema latestValueSchema,
+        final SegmentValueSchema segmentValueSchema,
+        final VersionedStoreClient<T> versionedStoreClient,
+        final Optional<Sensor> expiredRecordSensor,
+        final ProcessorContext context,
+        final long observedStreamTime,
+        final long historyRetention,
+        final Bytes key,
+        final byte[] value,
+        final long timestamp
+    ) {
+        // track the smallest timestamp seen so far that is larger than insertion timestamp.
+        // this timestamp determines, based on all segments searched so far, which segment the
+        // new record should be inserted into. the starting "sentinel timestamp" represents
+        // that the segment should be inserted into the latest value store.
+        long foundTs = SENTINEL_TIMESTAMP;
+
+        // check latest value store
+        PutStatus status = putLatestValueStore(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // continue search in segments
+        status = putSegments(
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            historyRetention,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+        if (status.isComplete) {
+            return;
+        } else {
+            foundTs = status.foundTs;
+        }
+
+        // ran out of segments to search. insert into segment as specified by foundTs
+        putFallThrough(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            expiredRecordSensor,
+            context,
+            observedStreamTime,
+            key,
+            value,
+            timestamp,
+            foundTs
+        );
+    }
+
+    /**
+     * Represents the status of an ongoing put() operation.
+     */
+    private static class PutStatus {
+
+        /**
+         * Whether the put() call has been completed.
+         */
+        final boolean isComplete;
+
+        /**
+         * The smallest timestamp seen so far, as part of this current put() operation, that is
+         * larger than insertion timestamp. This timestamp determines, based on all segments
+         * searched so far, which segment the new record should be inserted into.
+         */
+        final long foundTs;
+
+        PutStatus(final boolean isComplete, final long foundTs) {
+            this.isComplete = isComplete;
+            this.foundTs = foundTs;
+        }
+    }
+
+    private static <T extends VersionedStoreSegment> PutStatus putLatestValueStore(

Review Comment:
   OK. I made `doPut()` and all its helper methods non-static in the latest commit.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102256035


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestamps() {
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 1);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 5);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldPutIntoMultipleSegments() {
+        putToStore("k", null, SEGMENT_INTERVAL - 20);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 15);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentDuringPut() {
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", "vp10", SEGMENT_INTERVAL + 10);
+        putToStore("k", "vn10", SEGMENT_INTERVAL - 10);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+        putToStore("k", "vn1", SEGMENT_INTERVAL - 1);
+        putToStore("k", "vp1", SEGMENT_INTERVAL + 1);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 15, "vp10", SEGMENT_INTERVAL + 10);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 5, "vp1", SEGMENT_INTERVAL + 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 1, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn10", SEGMENT_INTERVAL - 10);
+    }
+
+    @Test
+    public void shouldMoveRecordToOlderSegmentWithNullsDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 15);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 6, "vp5", SEGMENT_INTERVAL + 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 8);
+    }
+
+    @Test
+    public void shouldFallThroughToExistingOlderSegmentAsLatestDuringPut() {
+        putToStore("k", null, SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);
+        putToStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", "vn2", SEGMENT_INTERVAL - 2);
+
+        verifyGetValueFromStore("k", "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 30, "vp20", SEGMENT_INTERVAL + 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL + 12);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 2, "vn2", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 6, "vn6", SEGMENT_INTERVAL - 6);
+    }
+
+    @Test
+    public void shouldPutNonLatestTombstoneIntoNewSegmentWithValidTo() {
+        putToStore("k", "vp30", SEGMENT_INTERVAL + 30);
+        putToStore("k", null, SEGMENT_INTERVAL - 10); // this put should result in tombstone with validTo=SEGMENT_INTERVAL+30
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn1", SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 2);
+
+        verifyGetValueFromStore("k", "vp30", SEGMENT_INTERVAL + 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL + 10, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 1, "vn1", SEGMENT_INTERVAL - 1);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 2);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 5, "vn5", SEGMENT_INTERVAL - 5);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 10);
+    }
+
+    @Test
+    public void shouldGetFromOlderSegments() {
+        // use a different key to create three different segments
+        putToStore("ko", null, SEGMENT_INTERVAL - 10);
+        putToStore("ko", null, 2 * SEGMENT_INTERVAL - 10);
+        putToStore("ko", null, 3 * SEGMENT_INTERVAL - 10);
+
+        // return null after visiting all segments
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 20);
+
+        // insert data to create non-empty (first) segment
+        putToStore("k", "v", SEGMENT_INTERVAL - 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 15, "v", SEGMENT_INTERVAL - 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+
+        // insert data to create non-empty (third) segment
+        putToStore("k", "v2", 3 * SEGMENT_INTERVAL - 20);
+        putToStore("k", null, 3 * SEGMENT_INTERVAL - 10);
+
+        // presence of non-empty later segment does not affect results of getting from earlier segment
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 30);
+        verifyTimestampedGetValueFromStore("k", SEGMENT_INTERVAL - 15, "v", SEGMENT_INTERVAL - 20);
+        verifyTimestampedGetNullFromStore("k", SEGMENT_INTERVAL - 5);
+    }
+
+    @Test
+    public void shouldNotGetExpired() {

Review Comment:
   Should we also test the `validTo`-update-case via an "out of order" put (regular record or null) leading to an existing record begin expired?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1171931840


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized

Review Comment:
   Updated in https://github.com/apache/kafka/pull/13615.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1103141124


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete
+     * @param timestamp The timestamp for this record version
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    void put(K key, V value, long timestamp);
+
+    /**
+     * Delete the value associated with this key from the store, at the specified timestamp
+     * (if there is such a value), and return the deleted value.
+     * <p>
+     * This operation is semantically equivalent to {@link #get(Object, long)} #get(key, timestamp))}
+     * followed by {@link #put(Object, Object, long) #put(key, null, timestamp)}.
+     *
+     * @param key       The key
+     * @param timestamp The timestamp for this delete
+     * @return The value and timestamp of the latest record associated with this key
+     *         as of the deletion timestamp (inclusive), or {@code null} if any of
+     *         (1) the store contains no records for this key, (2) the latest record
+     *         for this key as of the deletion timestamp is a tombstone, or
+     *         (3) the deletion timestamp is older than this store's history retention
+     *         (i.e., this store no longer contains data for the provided timestamp).
+     * @throws NullPointerException If {@code null} is used for key.
+     */
+    VersionedRecord<V> delete(K key, long timestamp);
+
+    /**
+     * Get the latest (by timestamp) record associated with this key.
+     *
+     * @param key The key to fetch
+     * @return The value and timestamp of the latest record associated with this key, or
+     *         {@code null} if either (1) the store contains no records for this key or (2) the
+     *         latest record for this key is a tombstone.
+     * @throws NullPointerException       If null is used for key.
+     * @throws InvalidStateStoreException if the store is not initialized
+     */
+    VersionedRecord<V> get(K key);
+
+    /**
+     * Get the latest record associated with this key with timestamp not exceeding the specified
+     * timestamp bound.

Review Comment:
   Makes sense. I'll omit for now, and we can always revisit in the future.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1100890258


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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 java.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
+import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback;
+import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment stores." The latest
+ * record version for each key is stored in the latest value store, while older record versions
+ * are stored in the segment stores. Conceptually, each record version has two associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next record (or deletion)
+ *     associated with the same key, and is implicitly associated with the record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo (exclusive), and
+ * can change as new record versions are inserted into the store (and validTo changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can be dropped from the
+ * store at a time, once the records contained in the segment are no longer relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same key) within a single
+ * segment are stored together using the format specified in {@link RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, byte[]> {
+    private static final Logger LOG = LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) {
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),

Review Comment:
   I am confused. `doPut()` is private and I don't see a call to it on the other PR. What do I miss?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102235180


##########
streams/src/main/java/org/apache/kafka/streams/state/VersionedKeyValueStore.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;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+
+/**
+ * A key-value store that stores multiple record versions per key, and supports timestamp-based
+ * retrieval operations to return the latest record (per key) as of a specified timestamp.
+ * Only one record is stored per key and timestamp, i.e., a second call to
+ * {@link #put(Object, Object, long)} with the same key and timestamp will replace the first.
+ * <p>
+ * Each store instance has an associated, fixed-duration "history retention" which specifies
+ * how long old record versions should be kept for. In particular, a versioned store guarantees
+ * to return accurate results for calls to {@link #get(Object, long)} where the provided timestamp
+ * bound is within history retention of the current observed stream time. (Queries with timestamp
+ * bound older than the specified history retention are considered invalid.)
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public interface VersionedKeyValueStore<K, V> extends StateStore {
+
+    /**
+     * Add a new record version associated with this key.
+     *
+     * @param key       The key
+     * @param value     The value, it can be {@code null};
+     *                  if the serialized bytes are also {@code null} it is interpreted as a delete

Review Comment:
   This PR was controversial... And I am still not a fan of it -> https://github.com/apache/kafka/pull/7679#issuecomment-581695959
   
   It potentially breaks stuff -- of course, if users follow the `null<->null` and `non-null<->non-null` pattern the PR does not do any harm.



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102248187


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {
+
+    private static final String STORE_NAME = "myversionedrocks";
+    private static final String METRICS_SCOPE = "versionedrocksdb";
+    private static final long HISTORY_RETENTION = 300_000L;
+    private static final long SEGMENT_INTERVAL = HISTORY_RETENTION / 3;
+    private static final long BASE_TIMESTAMP = 10L;
+    private static final Serializer<String> STRING_SERIALIZER = new StringSerializer();
+    private static final Deserializer<String> STRING_DESERIALIZER = new StringDeserializer();
+
+    private InternalMockProcessorContext context;
+
+    private RocksDBVersionedStore store;
+
+    @Before
+    public void before() {
+        context = new InternalMockProcessorContext<>(
+            TestUtils.tempDirectory(),
+            Serdes.String(),
+            Serdes.String(),
+            new StreamsConfig(StreamsTestUtils.getStreamsConfig())
+        );
+        context.setTime(BASE_TIMESTAMP);
+
+        store = new RocksDBVersionedStore(STORE_NAME, METRICS_SCOPE, HISTORY_RETENTION, SEGMENT_INTERVAL);
+        store.init((StateStoreContext) context, store);
+    }
+
+    @After
+    public void after() {
+        store.close();
+    }
+
+    @Test
+    public void shouldPutLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP + 1);
+
+        verifyGetValueFromStore("k", "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 1, "v2", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP + 2, "v2", BASE_TIMESTAMP + 1);
+    }
+
+    @Test
+    public void shouldPutNullAsLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP + 1);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP + 2);
+    }
+
+    @Test
+    public void shouldPutOlderWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", "v2", BASE_TIMESTAMP - 2);
+        putToStore("k", "v1", BASE_TIMESTAMP - 1);
+        putToStore("k", "v4", BASE_TIMESTAMP - 4);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 1, "v1", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 2, "v2", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v4", BASE_TIMESTAMP - 4);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNonNullLatest() {
+        putToStore("k", "v", BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetValueFromStore("k", "v", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "v", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutOlderNullWithNullLatest() {
+        putToStore("k", null, BASE_TIMESTAMP);
+        putToStore("k", null, BASE_TIMESTAMP - 2);
+        putToStore("k", null, BASE_TIMESTAMP - 1);
+        putToStore("k", null, BASE_TIMESTAMP - 4);
+        putToStore("k", "v3", BASE_TIMESTAMP - 3);
+        putToStore("k", "v5", BASE_TIMESTAMP - 5);
+        putToStore("k", null, BASE_TIMESTAMP - 6);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 1);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 2);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 3, "v3", BASE_TIMESTAMP - 3);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 4);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP - 5, "v5", BASE_TIMESTAMP - 5);
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP - 6);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestampAsLatest() {
+        putToStore("k", "to_be_replaced", BASE_TIMESTAMP);
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", null, BASE_TIMESTAMP);
+
+        verifyGetNullFromStore("k");
+        verifyTimestampedGetNullFromStore("k", BASE_TIMESTAMP);
+
+        putToStore("k", "b", BASE_TIMESTAMP);
+
+        verifyGetValueFromStore("k", "b", BASE_TIMESTAMP);
+        verifyTimestampedGetValueFromStore("k", BASE_TIMESTAMP, "b", BASE_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutRepeatTimestamps() {
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 10);
+        putToStore("k", null, SEGMENT_INTERVAL - 10);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 1);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL - 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 1);
+        putToStore("k", null, SEGMENT_INTERVAL + 10);
+        putToStore("k", null, SEGMENT_INTERVAL + 5);
+        putToStore("k", "vp5", SEGMENT_INTERVAL + 5);
+        putToStore("k", "to_be_replaced", SEGMENT_INTERVAL - 5);
+        putToStore("k", "vn5", SEGMENT_INTERVAL - 5);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", null, SEGMENT_INTERVAL + 20);
+        putToStore("k", "vn6", SEGMENT_INTERVAL - 6);

Review Comment:
   This is a lot of operations and hard to keep track about it without writing it down... could we add some comments about "expected state" (maybe also in-between the puts)?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mjsax commented on a diff in pull request #13188: KAFKA-14491: [5/N] Basic operations for RocksDB versioned store

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1102245805


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStoreTest.java:
##########
@@ -0,0 +1,435 @@
+/*
+ * 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 static java.nio.charset.StandardCharsets.UTF_8;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RocksDBVersionedStoreTest {

Review Comment:
   Can we add a test (maybe in a follow up PR, with retention time zero)? Having no segments and `segment_interval = 0` (?) seems to be a corner case we should also support.
   
   Btw: it seems we don't test if `delete()` does return the right "old value"; can we also add some tests for it?



-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org