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



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)

Review comment:
       I understand the rationale here, but I think rocksDB's lexico comparator 
would take into the length of the bytes into account as well, so if [timestamp, 
lower-range] is a prefix of [timestamp, key] then the former would still be put 
ahead of the latter.
   
   Maybe we can write a simple unit test to verify this? If it is not really 
the case, would putting a single byte be sufficient, than an int?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
Math.max(0, from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
to, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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());
+
+                    // 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 (time > to) {
+                        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) {
+                        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);
+        }
+
+        // for store serdes
+        public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
+            final long timestamp,
+            final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toTimeOrderedStoreKeyBinary(serializedKey, timestamp, 
seqnum);
+        }
+
+        static Bytes toTimeOrderedStoreKeyBinary(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 = 
WindowKeySchema.extractStoreTimestamp(binaryKey);
+            return timeWindowForSize(start, windowSize);
+        }
+    }
+
+    public static class KeyFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                return null;
+            }
+            final byte[] maxSuffix = ByteBuffer.allocate(SUFFIX_SIZE)

Review comment:
       Shouldn't we add the `KEY_FIRST_PREFIX` as well?
   
   And if yes, it seems besides the prefix the logic is the same as the 
existing window key schema, could we consolidate the logic to avoid some 
redundancy?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
Math.max(0, from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
to, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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());
+
+                    // 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 (time > to) {
+                        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) {
+                        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);
+        }
+
+        // for store serdes
+        public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
+            final long timestamp,

Review comment:
       nit: we usually align the parameters, like this:
   
   ```
   public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
                                                                                
        final long timestamp,
                                                                                
        final int seqnum) {
   ```

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.NoSuchElementException;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Windowed;
+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.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema;
+
+/**
+ * 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 RocksDBTimeOrderedWindowStore implements WindowStore<Bytes, 
byte[]> {
+
+
+
+    private final boolean retainDuplicates;
+    private final long windowSize;
+    private final String name;
+    private int seqnum = 0;
+    private RocksDBTimeOrderedSegmentedBytesStore baseStore;
+    private RocksDBSegmentedBytesStore indexStore;
+
+    private StateStoreContext stateStoreContext;
+    private Position position;
+    private boolean open;
+
+
+    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 indexIterator.peekNextKey();
+        }
+
+        @Override
+        public boolean hasNext() {
+            while (indexIterator.hasNext()) {
+                final Bytes key = indexIterator.peekNextKey();
+                final Bytes keyBytes = 
Bytes.wrap(KeyFirstWindowKeySchema.extractStoreKeyBytes(key.get()));
+                final long timestamp = 
KeyFirstWindowKeySchema.extractStoreTimestamp(key.get());
+                final int seqnum = 
KeyFirstWindowKeySchema.extractStoreSeqnum(key.get());
+
+                cachedValue = 
baseStore.get(TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
timestamp, seqnum));
+                if (cachedValue == null) {
+                    // Key not in base store, inconsistency happened. Skip 
this key and reply on
+                    // segment store to clean this.
+                    indexIterator.next();
+                } else {
+                    return true;
+                }
+            }
+            return false;
+        }
+
+        @Override
+        public KeyValue<Bytes, byte[]> next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            KeyValue<Bytes, byte[]> ret = indexIterator.next();
+            return KeyValue.pair(ret.key, cachedValue);
+        }
+    }
+
+    RocksDBTimeOrderedWindowStore(
+        final RocksDBTimeOrderedSegmentedBytesStore baseStore,
+        final String name,
+        final boolean retainDuplicates,
+        final long windowSize
+    ) {
+        this(baseStore, null, name, retainDuplicates, windowSize);
+    }
+
+    RocksDBTimeOrderedWindowStore(
+        final RocksDBTimeOrderedSegmentedBytesStore baseStore,
+        final RocksDBSegmentedBytesStore indexStore,
+        final String name,
+        final boolean retainDuplicates,
+        final long windowSize
+    ) {
+        Objects.requireNonNull(baseStore, "baseStore is null");
+        Objects.requireNonNull(name, "name is null");
+        this.baseStore = baseStore;
+        this.indexStore = indexStore;
+        this.name = name;
+        this.retainDuplicates = retainDuplicates;
+        this.windowSize = windowSize;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        baseStore.openSegments(context);
+        if (hasIndexStore()) {
+            indexStore.openSegments(context);
+        }
+        open = true;
+
+        // TODO: register changelog callback and populate base and index store
+        // register callback here since only window store knows how the keys 
are serialized in changelog
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.stateStoreContext = context;
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }
+
+    @Override
+    public void flush() {
+        baseStore.flush();
+        if (hasIndexStore()) {
+            indexStore.flush();
+        }
+    }
+
+    @Override
+    public void close() {
+        open = false;
+        baseStore.close();
+        if (hasIndexStore()) {
+            indexStore.close();
+        }
+    }
+
+    @Override
+    public Position getPosition() {
+        return position;
+    }
+
+    @Override
+    public boolean persistent() {
+        return true;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return open;
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long 
windowStartTimestamp) {
+        // Skip if value is null and duplicates are allowed since this delete 
is a no-op
+        if (!(value == null && retainDuplicates)) {
+            maybeUpdateSeqnumForDups();
+            if (hasIndexStore()) {
+                indexStore.put(
+                    KeyFirstWindowKeySchema.toStoreKeyBinary(key, 
windowStartTimestamp, seqnum), new byte[0]);
+            }
+            baseStore.put(
+                TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
windowStartTimestamp, seqnum), value);
+        }
+    }
+
+    @Override
+    public byte[] fetch(final Bytes key, final long timestamp) {
+        // TODO: check if some segments in index store can be purged
+        return 
baseStore.get(TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
timestamp, seqnum));
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> fetch(final Bytes key, final long 
timeFrom, final long timeTo) {
+        if (hasIndexStore()) {
+            // If index store exists, we still prefer to fetch from index 
store since it's ordered
+            // by key. The number of invalid keys we fetched should be much 
less than fetching
+            // from base store.
+            final KeyValueIterator<Bytes, byte[]> bytesIterator = 
indexStore.fetch(key, timeFrom,
+                timeTo);
+            return new WindowStoreIteratorWrapper(new 
IndexToBaseStoreIterator(bytesIterator),
+                windowSize,
+                KeyFirstWindowKeySchema::extractStoreTimestamp,
+                KeyFirstWindowKeySchema::fromStoreBytesKey).valuesIterator();
+        }
+
+        final KeyValueIterator<Bytes, byte[]> bytesIterator = 
baseStore.fetch(key, timeFrom, timeTo);
+        return new WindowStoreIteratorWrapper(bytesIterator,
+            windowSize,
+            TimeFirstWindowKeySchema::extractStoreTimestamp,
+            TimeFirstWindowKeySchema::fromStoreBytesKey).valuesIterator();
+    }
+
+    @Override
+    public WindowStoreIterator<byte[]> backwardFetch(final Bytes key, final 
long timeFrom, final long timeTo) {
+        if (hasIndexStore()) {
+            // If index store exists, we still prefer to fetch from index 
store since it's ordered
+            // by key. The number of invalid keys we fetched should be much 
less than fetching
+            // from base store.
+            final KeyValueIterator<Bytes, byte[]> bytesIterator = 
indexStore.backwardFetch(key, timeFrom,
+                timeTo);
+            return new WindowStoreIteratorWrapper(new 
IndexToBaseStoreIterator(bytesIterator),
+                windowSize,
+                KeyFirstWindowKeySchema::extractStoreTimestamp,
+                KeyFirstWindowKeySchema::fromStoreBytesKey).valuesIterator();
+        }
+
+        final KeyValueIterator<Bytes, byte[]> bytesIterator = 
baseStore.backwardFetch(key, timeFrom, timeTo);
+        return new WindowStoreIteratorWrapper(bytesIterator,
+            windowSize,
+            TimeFirstWindowKeySchema::extractStoreTimestamp,
+            TimeFirstWindowKeySchema::fromStoreBytesKey).valuesIterator();
+    }
+
+    @Override
+    public KeyValueIterator<Windowed<Bytes>, byte[]> fetch(final Bytes keyFrom,
+                                                           final Bytes keyTo,
+                                                           final long timeFrom,
+                                                           final long timeTo) {
+        final KeyValueIterator<Bytes, byte[]> bytesIterator = 
baseStore.fetch(keyFrom, keyTo, timeFrom, timeTo);

Review comment:
       WDYT whether we should access the index store for this fetch range as 
well? I guess for the emit use cases this does not matter since it is not 
called at all. But in general if the key space is denser compared to the 
time-range space, maybe it's still better to rely on the index store.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
Math.max(0, from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
to, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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());
+
+                    // 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 (time > to) {
+                        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) {
+                        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);
+        }
+
+        // for store serdes
+        public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
+            final long timestamp,
+            final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toTimeOrderedStoreKeyBinary(serializedKey, timestamp, 
seqnum);
+        }
+
+        static Bytes toTimeOrderedStoreKeyBinary(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 = 
WindowKeySchema.extractStoreTimestamp(binaryKey);
+            return timeWindowForSize(start, windowSize);
+        }
+    }
+
+    public static class KeyFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                return null;
+            }
+            final byte[] maxSuffix = ByteBuffer.allocate(SUFFIX_SIZE)
+                .putLong(to)
+                .putInt(Integer.MAX_VALUE)
+                .array();
+
+            return OrderedBytes.upperRange(key, maxSuffix);
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return null;
+            }
+            return OrderedBytes.lowerRange(key, MIN_SUFFIX);
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return WindowKeySchema.toStoreKeyBinary(key, Math.max(0, from), 0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return WindowKeySchema.toStoreKeyBinary(key, to, 
Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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 != KEY_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final Bytes keyBytes = 
Bytes.wrap(KeyFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    final long time = 
KeyFirstWindowKeySchema.extractStoreTimestamp(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);
+        }
+
+        public static Bytes toStoreKeyBinary(final Bytes key,
+                                             final long timestamp,
+                                             final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toStoreKeyBinary(serializedKey, timestamp, seqnum);
+        }
+
+        static Bytes toStoreKeyBinary(final byte[] serializedKey,

Review comment:
       Just curious, could these package private functions be private actually?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
Math.max(0, from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
to, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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());
+
+                    // 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 (time > to) {
+                        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) {
+                        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);
+        }
+
+        // for store serdes
+        public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
+            final long timestamp,
+            final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toTimeOrderedStoreKeyBinary(serializedKey, timestamp, 
seqnum);
+        }
+
+        static Bytes toTimeOrderedStoreKeyBinary(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 = 
WindowKeySchema.extractStoreTimestamp(binaryKey);
+            return timeWindowForSize(start, windowSize);
+        }
+    }
+
+    public static class KeyFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                return null;
+            }
+            final byte[] maxSuffix = ByteBuffer.allocate(SUFFIX_SIZE)
+                .putLong(to)
+                .putInt(Integer.MAX_VALUE)
+                .array();
+
+            return OrderedBytes.upperRange(key, maxSuffix);
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return null;
+            }
+            return OrderedBytes.lowerRange(key, MIN_SUFFIX);
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return WindowKeySchema.toStoreKeyBinary(key, Math.max(0, from), 0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return WindowKeySchema.toStoreKeyBinary(key, to, 
Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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 != KEY_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final Bytes keyBytes = 
Bytes.wrap(KeyFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    final long time = 
KeyFirstWindowKeySchema.extractStoreTimestamp(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);
+        }
+
+        public static Bytes toStoreKeyBinary(final Bytes key,
+                                             final long timestamp,
+                                             final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toStoreKeyBinary(serializedKey, timestamp, seqnum);
+        }
+
+        static Bytes toStoreKeyBinary(final byte[] serializedKey,
+                                      final long timestamp,
+                                      final int seqnum) {
+            final ByteBuffer buf = ByteBuffer.allocate(PREFIX_SIZE + 
serializedKey.length + TIMESTAMP_SIZE + SEQNUM_SIZE);
+            buf.put(KEY_FIRST_PREFIX);
+            buf.put(serializedKey);
+            buf.putLong(timestamp);
+            buf.putInt(seqnum);
+
+            return Bytes.wrap(buf.array());
+        }
+
+        static byte[] extractStoreKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - 
SEQNUM_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length);
+            return bytes;
+        }
+
+        static long extractStoreTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - 
TIMESTAMP_SIZE - SEQNUM_SIZE);
+        }
+
+        static int extractStoreSeqnum(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getInt(binaryKey.length - 
SEQNUM_SIZE);
+        }
+
+        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 = 
WindowKeySchema.extractStoreTimestamp(binaryKey);

Review comment:
       Also for other usage of static functions of `WindowKeySchema`, could we 
double check if they are okay to be re-used without taking the first byte into 
account?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
Math.max(0, from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
to, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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());
+
+                    // 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 (time > to) {
+                        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) {
+                        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);
+        }
+
+        // for store serdes
+        public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
+            final long timestamp,
+            final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toTimeOrderedStoreKeyBinary(serializedKey, timestamp, 
seqnum);
+        }
+
+        static Bytes toTimeOrderedStoreKeyBinary(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 = 
WindowKeySchema.extractStoreTimestamp(binaryKey);
+            return timeWindowForSize(start, windowSize);
+        }
+    }
+
+    public static class KeyFirstWindowKeySchema implements 
RocksDBSegmentedBytesStore.KeySchema {
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            if (key == null) {
+                return null;
+            }
+            final byte[] maxSuffix = ByteBuffer.allocate(SUFFIX_SIZE)
+                .putLong(to)
+                .putInt(Integer.MAX_VALUE)
+                .array();
+
+            return OrderedBytes.upperRange(key, maxSuffix);
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            if (key == null) {
+                return null;
+            }
+            return OrderedBytes.lowerRange(key, MIN_SUFFIX);
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return WindowKeySchema.toStoreKeyBinary(key, Math.max(0, from), 0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return WindowKeySchema.toStoreKeyBinary(key, to, 
Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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 != KEY_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final Bytes keyBytes = 
Bytes.wrap(KeyFirstWindowKeySchema.extractStoreKeyBytes(bytes.get()));
+                    final long time = 
KeyFirstWindowKeySchema.extractStoreTimestamp(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);
+        }
+
+        public static Bytes toStoreKeyBinary(final Bytes key,
+                                             final long timestamp,
+                                             final int seqnum) {
+            final byte[] serializedKey = key.get();
+            return toStoreKeyBinary(serializedKey, timestamp, seqnum);
+        }
+
+        static Bytes toStoreKeyBinary(final byte[] serializedKey,
+                                      final long timestamp,
+                                      final int seqnum) {
+            final ByteBuffer buf = ByteBuffer.allocate(PREFIX_SIZE + 
serializedKey.length + TIMESTAMP_SIZE + SEQNUM_SIZE);
+            buf.put(KEY_FIRST_PREFIX);
+            buf.put(serializedKey);
+            buf.putLong(timestamp);
+            buf.putInt(seqnum);
+
+            return Bytes.wrap(buf.array());
+        }
+
+        static byte[] extractStoreKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - 
SEQNUM_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length);
+            return bytes;
+        }
+
+        static long extractStoreTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - 
TIMESTAMP_SIZE - SEQNUM_SIZE);
+        }
+
+        static int extractStoreSeqnum(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getInt(binaryKey.length - 
SEQNUM_SIZE);
+        }
+
+        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 = 
WindowKeySchema.extractStoreTimestamp(binaryKey);

Review comment:
       `WindowKeySchema.extractStoreTimestamp` does not take the first prefix 
byte into account, is that okay?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java
##########
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.TimeWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+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 = 1;
+    private static final byte KEY_FIRST_PREFIX = 2;
+    private static final int SEQNUM_SIZE = 4;
+    private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE;
+    private static final byte[] MIN_SUFFIX = new byte[SUFFIX_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 (to == Long.MAX_VALUE) {
+                return null;
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(to + 1)
+                .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 + SEQNUM_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .put(key.get())
+                .putInt(0)
+                .array());
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
Math.max(0, from),
+                0);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
to, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return WindowKeySchema.extractStoreTimestamp(key.get());
+        }
+
+        @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());
+
+                    // 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 (time > to) {
+                        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) {
+                        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);
+        }
+
+        // for store serdes
+        public static Bytes toTimeOrderedStoreKeyBinary(final Bytes key,
+            final long timestamp,

Review comment:
       Ditto elsewhere.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java
##########
@@ -0,0 +1,359 @@
+/*
+ * 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.NoSuchElementException;
+import java.util.Objects;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Windowed;
+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.StoreToProcessorContextAdapter;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.WindowStore;
+import org.apache.kafka.streams.state.WindowStoreIterator;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.KeyFirstWindowKeySchema;
+import 
org.apache.kafka.streams.state.internals.PrefixedWindowKeySchemas.TimeFirstWindowKeySchema;
+
+/**
+ * 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 RocksDBTimeOrderedWindowStore implements WindowStore<Bytes, 
byte[]> {
+
+
+
+    private final boolean retainDuplicates;
+    private final long windowSize;
+    private final String name;
+    private int seqnum = 0;
+    private RocksDBTimeOrderedSegmentedBytesStore baseStore;
+    private RocksDBSegmentedBytesStore indexStore;
+
+    private StateStoreContext stateStoreContext;
+    private Position position;
+    private boolean open;
+
+
+    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 indexIterator.peekNextKey();
+        }
+
+        @Override
+        public boolean hasNext() {
+            while (indexIterator.hasNext()) {
+                final Bytes key = indexIterator.peekNextKey();
+                final Bytes keyBytes = 
Bytes.wrap(KeyFirstWindowKeySchema.extractStoreKeyBytes(key.get()));
+                final long timestamp = 
KeyFirstWindowKeySchema.extractStoreTimestamp(key.get());
+                final int seqnum = 
KeyFirstWindowKeySchema.extractStoreSeqnum(key.get());
+
+                cachedValue = 
baseStore.get(TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, 
timestamp, seqnum));
+                if (cachedValue == null) {
+                    // Key not in base store, inconsistency happened. Skip 
this key and reply on
+                    // segment store to clean this.

Review comment:
       Not sure if I follow clearly on the comment here: `Skip this key and 
reply on segment store to clean this.` How the segment store will clean this 
itself? Should we explicitly delete from the index store?




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