guozhangwang commented on a change in pull request #11802:
URL: https://github.com/apache/kafka/pull/11802#discussion_r822940645



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.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.StreamsConfig;
+import org.apache.kafka.streams.errors.ProcessorStateException;
+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.KeyValueIterator;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED;
+
+public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore<S extends 
Segment> implements SegmentedBytesStore {
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class);
+
+    private final String name;
+    protected final AbstractSegments<S> segments;
+    private final String metricScope;
+    protected final KeySchema baseKeySchema;
+    protected final Optional<KeySchema> indexKeySchema;
+
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private boolean consistencyEnabled = false;
+    private Position position;
+    protected OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    AbstractDualSchemaRocksDBSegmentedBytesStore(final String name,
+                                                 final String metricScope,
+                                                 final KeySchema baseKeySchema,
+                                                 final Optional<KeySchema> 
indexKeySchema,
+                                                 final AbstractSegments<S> 
segments) {
+        this.name = name;
+        this.metricScope = metricScope;
+        this.baseKeySchema = baseKeySchema;
+        this.indexKeySchema = indexKeySchema;
+        this.segments = segments;
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> all() {
+        final List<S> searchSpace = segments.allSegments(true);
+
+        return new SegmentIterator<>(
+                searchSpace.iterator(),
+                baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE),
+                null,
+                null,
+                true);
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> backwardAll() {
+        final List<S> searchSpace = segments.allSegments(false);
+
+        return new SegmentIterator<>(
+                searchSpace.iterator(),
+                baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE),
+                null,
+                null,
+                false);
+    }
+
+    @Override
+    public void remove(final Bytes rawKey) {
+        final long timestamp = baseKeySchema.segmentTimestamp(rawKey);
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+        final S segment = segments.getSegmentForTimestamp(timestamp);
+        if (segment == null) {
+            return;
+        }
+        segment.delete(rawKey);
+    }
+
+    @Override
+    public void put(final Bytes rawKey,
+                    final byte[] value) {
+        final long timestamp = baseKeySchema.segmentTimestamp(rawKey);
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+        final long segmentId = segments.segmentId(timestamp);
+        final S segment = segments.getOrCreateSegmentIfLive(segmentId, 
context, observedStreamTime);
+        if (segment == null) {
+            expiredRecordSensor.record(1.0d, 
ProcessorContextUtils.currentSystemTime(context));
+            LOG.warn("Skipping record for expired segment.");
+        } else {
+            StoreQueryUtils.updatePosition(position, stateStoreContext);
+            segment.put(rawKey, value);
+        }
+    }
+
+    @Override
+    public byte[] get(final Bytes rawKey) {
+        final S segment = 
segments.getSegmentForTimestamp(baseKeySchema.segmentTimestamp(rawKey));
+        if (segment == null) {
+            return null;
+        }
+        return segment.get(rawKey);
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @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
+        );
+
+        openSegments(this.context);
+
+        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) this::restoreAllInternal,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, 
position)
+        );
+
+        open = true;
+
+        consistencyEnabled = StreamsConfig.InternalConfig.getBoolean(
+                context.appConfigs(),

Review comment:
       Nit: indent seems not aligned?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.util.Arrays;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.state.StateSerdes;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize;
+
+public class PrefixedWindowKeySchemas {
+
+    private static final int PREFIX_SIZE = 1;
+    private static final byte TIME_FIRST_PREFIX = 0;
+    private static final byte KEY_FIRST_PREFIX = 1;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+
+    private static byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                // Put next prefix instead of null so that we can start from 
right prefix
+                // when scanning backwards
+                final byte nextPrefix = TIME_FIRST_PREFIX + 1;
+                return 
Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE).put(nextPrefix).array());
+            }
+            byte[] maxKey = new byte[key.get().length];
+            Arrays.fill(maxKey, (byte) 0xFF);
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ maxKey.length + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to)
+                .put(maxKey).putInt(Integer.MAX_VALUE)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 
TIMESTAMP_SIZE)
+                    .put(TIME_FIRST_PREFIX)
+                    .putLong(from)
+                    .array());
+            }
+
+            /*
+             * Larger timestamp or key's byte order can't be smaller than this 
lower range. Reason:
+             *     1. Timestamp is fixed length (with big endian byte order). 
Since we put timestamp
+             *        first, larger timestamp will have larger byte order.
+             *     2. If timestamp is the same but key (k1) is larger than 
this lower range key (k2):
+             *         a. If k2 is not a prefix of k1, then k1 will always 
have larger byte order no
+             *            matter what seqnum k2 has
+             *         b. If k2 is a prefix of k1, since k2's seqnum is 0, 
after k1 appends seqnum,
+             *            it will always be larger than (k1 + seqnum).
+             */
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, Math.max(0, 
from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, to, 
Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return TimeFirstWindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, boolean 
forward) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final long time = 
TimeFirstWindowKeySchema.extractStoreTimestamp(bytes.get());
+
+                    // We can return false directly here since keys are sorted 
by time and if
+                    // we get time larger than `to`, there won't be time 
within range.
+                    if (forward && time > to) {
+                        return false;
+                    }
+                    if (!forward && time < from) {
+                        return false;
+                    }
+
+                    final Bytes keyBytes = Bytes.wrap(
+                        
TimeFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    if ((binaryKeyFrom == null || 
keyBytes.compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
keyBytes.compareTo(binaryKeyTo) <= 0)
+                        && time >= from && time <= to) {
+                        return true;
+                    }
+                    iterator.next();
+                }
+                return false;
+            };
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,

Review comment:
       Could we just call `hasNextCondition(...., true)` to avoid duplicate 
code?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedSegmentedBytesStore.java
##########
@@ -0,0 +1,337 @@
+/*
+ * 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.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.errors.ProcessorStateException;
+import 
org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * RocksDB store backed by two SegmentedBytesStores which can optimize scan by 
time as well as window
+ * lookup for a specific key.
+ *
+ * Schema for first SegmentedBytesStore (base store) is as below:
+ *     Key schema: | timestamp + recordkey |
+ *     Value schema: | value |. Value here is determined by caller.
+ *
+ * Schema for second SegmentedBytesStore (index store) is as below:
+ *     Key schema: | record + timestamp |
+ *     Value schema: ||
+ *
+ * Operations:
+ *     Put: 1. Put to index store. 2. Put to base store.
+ *     Delete: 1. Delete from base store. 2. Delete from index store.
+ * Since we need to update two stores, failure can happen in the middle. We 
put in index store first
+ * to make sure if a failure happens in second step and the view is 
inconsistent, we can't get the
+ * value for the key. We delete from base store first to make sure if a 
failure happens in second step
+ * and the view is inconsistent, we can't get the value for the key.
+ *
+ * Note:
+ *     Index store can be optional if we can construct the timestamp in base 
store instead of looking
+ *     them up from index store.
+ *
+ */
+public class RocksDBTimeOrderedSegmentedBytesStore extends 
AbstractDualSchemaRocksDBSegmentedBytesStore<KeyValueSegment> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class);
+
+    private class IndexToBaseStoreIterator implements KeyValueIterator<Bytes, 
byte[]> {
+        private final KeyValueIterator<Bytes, byte[]> indexIterator;
+        private byte[] cachedValue;
+
+
+        IndexToBaseStoreIterator(final KeyValueIterator<Bytes, byte[]> 
indexIterator) {
+            this.indexIterator = indexIterator;
+        }
+
+        @Override
+        public void close() {
+            indexIterator.close();
+        }
+
+        @Override
+        public Bytes peekNextKey() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            return getBaseKey(indexIterator.peekNextKey());
+        }
+
+        @Override
+        public boolean hasNext() {
+            while (indexIterator.hasNext()) {
+                final Bytes key = indexIterator.peekNextKey();
+                final Bytes baseKey = getBaseKey(key);
+
+                cachedValue = get(baseKey);
+                if (cachedValue == null) {
+                    // Key not in base store, inconsistency happened and 
remove from index.
+                    indexIterator.next();
+                    
RocksDBTimeOrderedSegmentedBytesStore.this.removeIndex(key);
+                } else {
+                    return true;
+                }
+            }
+            return false;
+        }
+
+        @Override
+        public KeyValue<Bytes, byte[]> next() {
+            if (cachedValue == null && !hasNext()) {
+                throw new NoSuchElementException();
+            }
+            final KeyValue<Bytes, byte[]> ret = indexIterator.next();
+            final byte[] value = cachedValue;
+            cachedValue = null;
+            return KeyValue.pair(getBaseKey(ret.key), value);
+        }
+
+        private Bytes getBaseKey(final Bytes indexKey) {
+            final byte[] keyBytes = 
KeyFirstWindowKeySchema.extractStoreKeyBytes(indexKey.get());
+            final long timestamp = 
KeyFirstWindowKeySchema.extractStoreTimestamp(indexKey.get());
+            final int seqnum = 
KeyFirstWindowKeySchema.extractStoreSequence(indexKey.get());
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(keyBytes, 
timestamp, seqnum);
+        }
+    }
+
+    RocksDBTimeOrderedSegmentedBytesStore(final String name,
+                                          final String metricsScope,
+                                          final long retention,
+                                          final long segmentInterval,
+                                          final boolean withIndex) {
+        super(name, metricsScope, new TimeFirstWindowKeySchema(),
+            Optional.ofNullable(withIndex ? new KeyFirstWindowKeySchema() : 
null),
+            new KeyValueSegments(name, metricsScope, retention, 
segmentInterval));
+    }
+
+    public void put(final Bytes key, final long timestamp, final int seqnum, 
final byte[] value) {
+        final Bytes baseKey = TimeFirstWindowKeySchema.toStoreKeyBinary(key, 
timestamp, seqnum);
+        put(baseKey, value);
+    }
+
+    byte[] fetch(final Bytes key, final long timestamp, final int seqnum) {
+        return get(TimeFirstWindowKeySchema.toStoreKeyBinary(key, timestamp, 
seqnum));
+    }
+
+    @Override
+    protected KeyValue<Bytes, byte[]> getIndexKeyValue(final Bytes baseKey, 
final byte[] baseValue) {
+        final byte[] key = 
TimeFirstWindowKeySchema.extractStoreKeyBytes(baseKey.get());
+        final long timestamp = 
TimeFirstWindowKeySchema.extractStoreTimestamp(baseKey.get());
+        final int seqnum = 
TimeFirstWindowKeySchema.extractStoreSequence(baseKey.get());
+
+        return KeyValue.pair(KeyFirstWindowKeySchema.toStoreKeyBinary(key, 
timestamp, seqnum), new byte[0]);
+    }
+
+    @Override
+    Map<KeyValueSegment, WriteBatch> getWriteBatches(
+        final Collection<ConsumerRecord<byte[], byte[]>> records) {
+        // advance stream time to the max timestamp in the batch
+        for (final ConsumerRecord<byte[], byte[]> record : records) {
+            final long timestamp = 
WindowKeySchema.extractStoreTimestamp(record.key());
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+        }
+
+        final Map<KeyValueSegment, WriteBatch> writeBatchMap = new HashMap<>();
+        for (final ConsumerRecord<byte[], byte[]> record : records) {
+            final long timestamp = 
WindowKeySchema.extractStoreTimestamp(record.key());
+            final long segmentId = segments.segmentId(timestamp);
+            final KeyValueSegment segment = 
segments.getOrCreateSegmentIfLive(segmentId, context, observedStreamTime);
+            if (segment != null) {
+                
ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition(
+                    record,
+                    consistencyEnabled,
+                    position
+                );
+                try {
+                    final WriteBatch batch = 
writeBatchMap.computeIfAbsent(segment, s -> new WriteBatch());
+
+                    // Assuming changelog record is serialized using 
WindowKeySchema
+                    // from ChangeLoggingTimestampedWindowBytesStore. 
Reconstruct key/value to restore
+                    final byte[] key = 
WindowKeySchema.extractStoreKeyBytes(record.key());

Review comment:
       I concern that by stripping off just the first byte for base key and 
index key, we would be paying too much byte array copies which would be much 
slower than other stores (for which we just blindly copy-paste the bytes). 
Maybe we can have some optimization in-place for the first draft? E.g.:
   
   1) for index key, we know we just need to add the first byte and strip the 
last four byte of seq (? is that right, need to double check).
   
   2) for base key, we can just do a swap-copy from the index key to switch the 
position of timestamp and key.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.util.Arrays;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.state.StateSerdes;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize;
+
+public class PrefixedWindowKeySchemas {
+
+    private static final int PREFIX_SIZE = 1;
+    private static final byte TIME_FIRST_PREFIX = 0;
+    private static final byte KEY_FIRST_PREFIX = 1;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+
+    private static byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                // Put next prefix instead of null so that we can start from 
right prefix
+                // when scanning backwards
+                final byte nextPrefix = TIME_FIRST_PREFIX + 1;
+                return 
Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE).put(nextPrefix).array());
+            }
+            byte[] maxKey = new byte[key.get().length];
+            Arrays.fill(maxKey, (byte) 0xFF);
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ maxKey.length + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to)
+                .put(maxKey).putInt(Integer.MAX_VALUE)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 
TIMESTAMP_SIZE)
+                    .put(TIME_FIRST_PREFIX)
+                    .putLong(from)
+                    .array());
+            }
+
+            /*
+             * Larger timestamp or key's byte order can't be smaller than this 
lower range. Reason:
+             *     1. Timestamp is fixed length (with big endian byte order). 
Since we put timestamp
+             *        first, larger timestamp will have larger byte order.
+             *     2. If timestamp is the same but key (k1) is larger than 
this lower range key (k2):
+             *         a. If k2 is not a prefix of k1, then k1 will always 
have larger byte order no
+             *            matter what seqnum k2 has
+             *         b. If k2 is a prefix of k1, since k2's seqnum is 0, 
after k1 appends seqnum,
+             *            it will always be larger than (k1 + seqnum).
+             */
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, Math.max(0, 
from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, to, 
Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return TimeFirstWindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, boolean 
forward) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final long time = 
TimeFirstWindowKeySchema.extractStoreTimestamp(bytes.get());
+
+                    // We can return false directly here since keys are sorted 
by time and if
+                    // we get time larger than `to`, there won't be time 
within range.
+                    if (forward && time > to) {
+                        return false;
+                    }
+                    if (!forward && time < from) {
+                        return false;
+                    }
+
+                    final Bytes keyBytes = Bytes.wrap(
+                        
TimeFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    if ((binaryKeyFrom == null || 
keyBytes.compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
keyBytes.compareTo(binaryKeyTo) <= 0)
+                        && time >= from && time <= to) {
+                        return true;
+                    }
+                    iterator.next();
+                }
+                return false;
+            };
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo,
+            final long from,
+            final long to) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final long time = 
TimeFirstWindowKeySchema.extractStoreTimestamp(bytes.get());
+
+                    final Bytes keyBytes = Bytes.wrap(
+                        
TimeFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    if ((binaryKeyFrom == null || 
keyBytes.compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
keyBytes.compareTo(binaryKeyTo) <= 0)
+                        && time >= from && time <= to) {
+                        return true;
+                    }
+                    iterator.next();
+                }
+                return false;
+            };
+        }
+
+        @Override
+        public <S extends Segment> List<S> segmentsToSearch(final Segments<S> 
segments,
+            final long from,
+            final long to,
+            final boolean forward) {
+            return segments.segments(from, to, forward);
+        }
+
+        static byte[] extractStoreKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - 
SEQNUM_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, TIMESTAMP_SIZE + PREFIX_SIZE, bytes, 
0, bytes.length);
+            return bytes;
+        }
+
+        static long extractStoreTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE);
+        }
+
+        public static Bytes toStoreKeyBinary(final Windowed<Bytes> timeKey,
+                                             final int seqnum) {
+            return toStoreKeyBinary(timeKey.key().get(), 
timeKey.window().start(), seqnum);
+        }
+
+        public static <K> Windowed<K> fromStoreKey(final byte[] binaryKey,
+                                                   final long windowSize,
+                                                   final Deserializer<K> 
deserializer,
+                                                   final String topic) {
+            final K key = deserializer.deserialize(topic, 
extractStoreKeyBytes(binaryKey));
+            final Window window = extractStoreWindow(binaryKey, windowSize);
+            return new Windowed<>(key, window);
+        }
+
+        public static <K> Bytes toStoreKeyBinary(final Windowed<K> timeKey,
+                                                 final int seqnum,
+                                                 final StateSerdes<K, ?> 
serdes) {
+            final byte[] serializedKey = serdes.rawKey(timeKey.key());
+            return toStoreKeyBinary(serializedKey, timeKey.window().start(), 
seqnum);
+        }
+
+        // for store serdes
+        public static Bytes toStoreKeyBinary(final Bytes key,
+                                             final long timestamp,
+                                             final int seqnum) {
+            return toStoreKeyBinary(key.get(), timestamp, seqnum);
+        }
+
+        static Bytes toStoreKeyBinary(final byte[] serializedKey,
+                                                 final long timestamp,
+                                                 final int seqnum) {
+            final ByteBuffer buf = ByteBuffer.allocate(
+                PREFIX_SIZE + TIMESTAMP_SIZE + serializedKey.length + 
SEQNUM_SIZE);
+            buf.put(TIME_FIRST_PREFIX);
+            buf.putLong(timestamp);
+            buf.put(serializedKey);
+            buf.putInt(seqnum);
+
+            return Bytes.wrap(buf.array());
+        }
+
+        public static Windowed<Bytes> fromStoreBytesKey(final byte[] binaryKey,
+                                                        final long windowSize) 
{
+            final Bytes key = Bytes.wrap(extractStoreKeyBytes(binaryKey));
+            final Window window = extractStoreWindow(binaryKey, windowSize);
+            return new Windowed<>(key, window);
+        }
+
+        static Window extractStoreWindow(final byte[] binaryKey,
+                                         final long windowSize) {
+            final long start = extractStoreTimestamp(binaryKey);
+            return timeWindowForSize(start, windowSize);
+        }
+
+        static int extractStoreSequence(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getInt(binaryKey.length - 
SEQNUM_SIZE);
+        }
+    }
+
+    public static class KeyFirstWindowKeySchema extends WindowKeySchema {
+
+        private Bytes wrapPrefix(final Bytes noPrefixKey) {
+            return wrapPrefix(noPrefixKey, KEY_FIRST_PREFIX);
+        }
+
+        private Bytes wrapPrefix(final Bytes noPrefixKey, final byte prefix) {
+            // Need to scan from prefix even key is null
+            if (noPrefixKey == null) {
+                final byte[] ret = ByteBuffer.allocate(PREFIX_SIZE)
+                    .put(prefix)
+                    .array();
+                return Bytes.wrap(ret);
+            }
+            final byte[] ret = ByteBuffer.allocate(PREFIX_SIZE + 
noPrefixKey.get().length)
+                .put(prefix)
+                .put(noPrefixKey.get())
+                .array();
+            return Bytes.wrap(ret);
+        }
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            final Bytes noPrefixBytes = super.upperRange(key, to);
+            if (noPrefixBytes == null) {

Review comment:
       I feel it's a bit error-vulnerable to split the `if (noPrefixBytes == 
null)` condition between the caller and callee of `wrapPrefix`. Could we just 
have this condition in a single place inside the callee only?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.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.StreamsConfig;
+import org.apache.kafka.streams.errors.ProcessorStateException;
+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.KeyValueIterator;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED;
+
+public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore<S extends 
Segment> implements SegmentedBytesStore {
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class);
+
+    private final String name;
+    protected final AbstractSegments<S> segments;
+    private final String metricScope;
+    protected final KeySchema baseKeySchema;
+    protected final Optional<KeySchema> indexKeySchema;
+
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;

Review comment:
       This reminds me that we should change the internal interfaces of 
`Segments` etc to use the new StateStoreContext, not the ProcessorContext 
anymore, will create a new JIRA.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.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.StreamsConfig;
+import org.apache.kafka.streams.errors.ProcessorStateException;
+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.KeyValueIterator;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED;
+
+public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore<S extends 
Segment> implements SegmentedBytesStore {
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class);
+
+    private final String name;
+    protected final AbstractSegments<S> segments;
+    private final String metricScope;
+    protected final KeySchema baseKeySchema;
+    protected final Optional<KeySchema> indexKeySchema;
+
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private boolean consistencyEnabled = false;
+    private Position position;
+    protected OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    AbstractDualSchemaRocksDBSegmentedBytesStore(final String name,
+                                                 final String metricScope,
+                                                 final KeySchema baseKeySchema,
+                                                 final Optional<KeySchema> 
indexKeySchema,
+                                                 final AbstractSegments<S> 
segments) {
+        this.name = name;
+        this.metricScope = metricScope;
+        this.baseKeySchema = baseKeySchema;
+        this.indexKeySchema = indexKeySchema;
+        this.segments = segments;
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> all() {
+        final List<S> searchSpace = segments.allSegments(true);
+
+        return new SegmentIterator<>(
+                searchSpace.iterator(),
+                baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE),
+                null,
+                null,
+                true);
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> backwardAll() {
+        final List<S> searchSpace = segments.allSegments(false);
+
+        return new SegmentIterator<>(
+                searchSpace.iterator(),
+                baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE),
+                null,
+                null,
+                false);
+    }
+
+    @Override
+    public void remove(final Bytes rawKey) {
+        final long timestamp = baseKeySchema.segmentTimestamp(rawKey);
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+        final S segment = segments.getSegmentForTimestamp(timestamp);
+        if (segment == null) {
+            return;
+        }
+        segment.delete(rawKey);
+    }
+
+    @Override
+    public void put(final Bytes rawKey,
+                    final byte[] value) {
+        final long timestamp = baseKeySchema.segmentTimestamp(rawKey);
+        observedStreamTime = Math.max(observedStreamTime, timestamp);
+        final long segmentId = segments.segmentId(timestamp);
+        final S segment = segments.getOrCreateSegmentIfLive(segmentId, 
context, observedStreamTime);
+        if (segment == null) {
+            expiredRecordSensor.record(1.0d, 
ProcessorContextUtils.currentSystemTime(context));
+            LOG.warn("Skipping record for expired segment.");
+        } else {
+            StoreQueryUtils.updatePosition(position, stateStoreContext);
+            segment.put(rawKey, value);
+        }
+    }
+
+    @Override
+    public byte[] get(final Bytes rawKey) {
+        final S segment = 
segments.getSegmentForTimestamp(baseKeySchema.segmentTimestamp(rawKey));
+        if (segment == null) {
+            return null;
+        }
+        return segment.get(rawKey);
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @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,

Review comment:
       Nit: indent seems not aligned?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.util.Arrays;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.state.StateSerdes;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize;
+
+public class PrefixedWindowKeySchemas {
+
+    private static final int PREFIX_SIZE = 1;
+    private static final byte TIME_FIRST_PREFIX = 0;
+    private static final byte KEY_FIRST_PREFIX = 1;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+
+    private static byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                // Put next prefix instead of null so that we can start from 
right prefix
+                // when scanning backwards
+                final byte nextPrefix = TIME_FIRST_PREFIX + 1;
+                return 
Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE).put(nextPrefix).array());
+            }
+            byte[] maxKey = new byte[key.get().length];
+            Arrays.fill(maxKey, (byte) 0xFF);
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ maxKey.length + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to)
+                .put(maxKey).putInt(Integer.MAX_VALUE)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 
TIMESTAMP_SIZE)
+                    .put(TIME_FIRST_PREFIX)
+                    .putLong(from)
+                    .array());
+            }
+
+            /*
+             * Larger timestamp or key's byte order can't be smaller than this 
lower range. Reason:
+             *     1. Timestamp is fixed length (with big endian byte order). 
Since we put timestamp
+             *        first, larger timestamp will have larger byte order.
+             *     2. If timestamp is the same but key (k1) is larger than 
this lower range key (k2):
+             *         a. If k2 is not a prefix of k1, then k1 will always 
have larger byte order no
+             *            matter what seqnum k2 has
+             *         b. If k2 is a prefix of k1, since k2's seqnum is 0, 
after k1 appends seqnum,
+             *            it will always be larger than (k1 + seqnum).
+             */
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, Math.max(0, 
from),
+                0);

Review comment:
       Nit: no need newline.
   
   Also I think we do not need the `TimeFirstWindowKeySchema.` since it is 
within this class right? Ditto below.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.util.Arrays;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.state.StateSerdes;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize;
+
+public class PrefixedWindowKeySchemas {
+
+    private static final int PREFIX_SIZE = 1;
+    private static final byte TIME_FIRST_PREFIX = 0;
+    private static final byte KEY_FIRST_PREFIX = 1;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+
+    private static byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                // Put next prefix instead of null so that we can start from 
right prefix
+                // when scanning backwards
+                final byte nextPrefix = TIME_FIRST_PREFIX + 1;
+                return 
Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE).put(nextPrefix).array());

Review comment:
       I think we can be stricter here to put [1, to+1] as upper bound, wdyt?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.util.Arrays;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.state.StateSerdes;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize;
+
+public class PrefixedWindowKeySchemas {
+
+    private static final int PREFIX_SIZE = 1;
+    private static final byte TIME_FIRST_PREFIX = 0;
+    private static final byte KEY_FIRST_PREFIX = 1;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+
+    private static byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                // Put next prefix instead of null so that we can start from 
right prefix
+                // when scanning backwards
+                final byte nextPrefix = TIME_FIRST_PREFIX + 1;
+                return 
Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE).put(nextPrefix).array());
+            }
+            byte[] maxKey = new byte[key.get().length];
+            Arrays.fill(maxKey, (byte) 0xFF);
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ maxKey.length + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to)
+                .put(maxKey).putInt(Integer.MAX_VALUE)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 
TIMESTAMP_SIZE)
+                    .put(TIME_FIRST_PREFIX)
+                    .putLong(from)
+                    .array());
+            }
+
+            /*
+             * Larger timestamp or key's byte order can't be smaller than this 
lower range. Reason:
+             *     1. Timestamp is fixed length (with big endian byte order). 
Since we put timestamp
+             *        first, larger timestamp will have larger byte order.
+             *     2. If timestamp is the same but key (k1) is larger than 
this lower range key (k2):
+             *         a. If k2 is not a prefix of k1, then k1 will always 
have larger byte order no
+             *            matter what seqnum k2 has
+             *         b. If k2 is a prefix of k1, since k2's seqnum is 0, 
after k1 appends seqnum,
+             *            it will always be larger than (k1 + seqnum).
+             */
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE 
+ key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, Math.max(0, 
from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toStoreKeyBinary(key, to, 
Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return TimeFirstWindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, boolean 
forward) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final long time = 
TimeFirstWindowKeySchema.extractStoreTimestamp(bytes.get());
+
+                    // We can return false directly here since keys are sorted 
by time and if
+                    // we get time larger than `to`, there won't be time 
within range.
+                    if (forward && time > to) {
+                        return false;
+                    }
+                    if (!forward && time < from) {
+                        return false;
+                    }
+
+                    final Bytes keyBytes = Bytes.wrap(
+                        
TimeFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    if ((binaryKeyFrom == null || 
keyBytes.compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
keyBytes.compareTo(binaryKeyTo) <= 0)
+                        && time >= from && time <= to) {
+                        return true;
+                    }
+                    iterator.next();
+                }
+                return false;
+            };
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo,
+            final long from,
+            final long to) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final long time = 
TimeFirstWindowKeySchema.extractStoreTimestamp(bytes.get());
+
+                    final Bytes keyBytes = Bytes.wrap(
+                        
TimeFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    if ((binaryKeyFrom == null || 
keyBytes.compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
keyBytes.compareTo(binaryKeyTo) <= 0)
+                        && time >= from && time <= to) {
+                        return true;
+                    }
+                    iterator.next();
+                }
+                return false;
+            };
+        }
+
+        @Override
+        public <S extends Segment> List<S> segmentsToSearch(final Segments<S> 
segments,
+            final long from,
+            final long to,
+            final boolean forward) {
+            return segments.segments(from, to, forward);
+        }
+
+        static byte[] extractStoreKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - 
SEQNUM_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, TIMESTAMP_SIZE + PREFIX_SIZE, bytes, 
0, bytes.length);
+            return bytes;
+        }
+
+        static long extractStoreTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE);
+        }
+
+        public static Bytes toStoreKeyBinary(final Windowed<Bytes> timeKey,
+                                             final int seqnum) {
+            return toStoreKeyBinary(timeKey.key().get(), 
timeKey.window().start(), seqnum);
+        }
+
+        public static <K> Windowed<K> fromStoreKey(final byte[] binaryKey,
+                                                   final long windowSize,
+                                                   final Deserializer<K> 
deserializer,
+                                                   final String topic) {
+            final K key = deserializer.deserialize(topic, 
extractStoreKeyBytes(binaryKey));
+            final Window window = extractStoreWindow(binaryKey, windowSize);
+            return new Windowed<>(key, window);
+        }
+
+        public static <K> Bytes toStoreKeyBinary(final Windowed<K> timeKey,
+                                                 final int seqnum,
+                                                 final StateSerdes<K, ?> 
serdes) {
+            final byte[] serializedKey = serdes.rawKey(timeKey.key());
+            return toStoreKeyBinary(serializedKey, timeKey.window().start(), 
seqnum);
+        }
+
+        // for store serdes
+        public static Bytes toStoreKeyBinary(final Bytes key,
+                                             final long timestamp,
+                                             final int seqnum) {
+            return toStoreKeyBinary(key.get(), timestamp, seqnum);
+        }
+
+        static Bytes toStoreKeyBinary(final byte[] serializedKey,
+                                                 final long timestamp,

Review comment:
       Nit: param alignment.

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStoreTest.java
##########
@@ -0,0 +1,968 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.SystemTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.StreamsConfig.InternalConfig;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import 
org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper;
+import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
+import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
+import org.apache.kafka.streams.processor.internals.Task.TaskType;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import 
org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.StateSerdes;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.MockRecordCollector;
+import org.apache.kafka.test.StreamsTestUtils;
+import org.apache.kafka.test.TestUtils;
+import org.hamcrest.Matchers;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.rocksdb.WriteBatch;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.SimpleTimeZone;
+
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest<S 
extends Segment> {

Review comment:
       Thanks for the great coverage! As for the key-value pairs to put, could 
we also have some cases like `AAA` and `AAAB` i.e. some key is other key's 
prefix, etc, similar to what we have in the other unit tests to make sure our 
upper/lower range functions do not have edge case errors?




-- 
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-unsubscr...@kafka.apache.org

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


Reply via email to