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



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, final 
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 Windowed<Bytes> windowedKey = from(bytes);
+                    final long endTime = windowedKey.window().end();
+                    final long startTime = windowedKey.window().start();
+
+                    // We can return false directly here since keys are sorted 
by end time and if
+                    // we get time smaller than `from`, there won't be time 
within range.
+                    if (!forward && endTime < from) {
+                        return false;
+                    }
+
+                    if ((binaryKeyFrom == null || 
windowedKey.key().compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
windowedKey.key().compareTo(binaryKeyTo) <= 0)
+                        && endTime >= from && startTime <= 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, Long.MAX_VALUE, forward);
+        }
+
+        static long extractStartTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE + 
TIMESTAMP_SIZE);
+        }
+
+        static long extractEndTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE);
+        }
+
+        private static <K> K extractKey(final byte[] binaryKey,
+                                        final Deserializer<K> deserializer,
+                                        final String topic) {
+            return deserializer.deserialize(topic, extractKeyBytes(binaryKey));
+        }
+
+        static byte[] extractKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - 2 * 
TIMESTAMP_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, PREFIX_SIZE + 2 * TIMESTAMP_SIZE, 
bytes, 0, bytes.length);
+            return bytes;
+        }
+
+        static Window extractWindow(final byte[] binaryKey) {
+            final ByteBuffer buffer = ByteBuffer.wrap(binaryKey);
+            final long start = buffer.getLong(PREFIX_SIZE + TIMESTAMP_SIZE);
+            final long end = buffer.getLong(PREFIX_SIZE);
+            return new SessionWindow(start, end);
+        }
+
+        public static Windowed<Bytes> from(final Bytes bytesKey) {
+            final byte[] binaryKey = bytesKey.get();
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(Bytes.wrap(extractKeyBytes(binaryKey)), 
window);
+        }
+
+        public static <K> Windowed<K> from(final byte[] binaryKey,
+                                           final Deserializer<K> 
keyDeserializer,
+                                           final String topic) {
+            final K key = extractKey(binaryKey, keyDeserializer, topic);
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(key, window);
+        }
+
+        public static <K> byte[] toBinary(final Windowed<K> sessionKey,
+                                          final Serializer<K> serializer,
+                                          final String topic) {
+            final byte[] bytes = serializer.serialize(topic, sessionKey.key());
+            return toBinary(Bytes.wrap(bytes), sessionKey.window().start(), 
sessionKey.window().end()).get();
+        }
+
+        public static Bytes toBinary(final Windowed<Bytes> sessionKey) {

Review comment:
       nit: Some functions of these two schemas like `toBinary` are very alike 
except the first byte has a different value. Could we consolidate them as a 
common func in the `PrefixedSessionKeySchemas` class? 

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, final 
boolean forward) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {

Review comment:
       BTW if you agree with my comments on this class, then we can also do the 
same changes on the merged `PrefixedWindowKeySchemas` as well.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedSessionSegmentedBytesStore.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.Map;
+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.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import 
org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import 
org.apache.kafka.streams.state.internals.PrefixedSessionKeySchemas.KeyFirstSessionKeySchema;
+import 
org.apache.kafka.streams.state.internals.PrefixedSessionKeySchemas.TimeFirstSessionKeySchema;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+
+/**
+ * A RocksDB backed time-ordered segmented bytes store for session key schema.
+ */
+public class RocksDBTimeOrderedSessionSegmentedBytesStore extends 
AbstractRocksDBTimeOrderedSegmentedBytesStore {
+
+    private class SessionKeySchemaIndexToBaseStoreIterator  extends 
IndexToBaseStoreIterator {
+        SessionKeySchemaIndexToBaseStoreIterator(final KeyValueIterator<Bytes, 
byte[]> indexIterator) {
+            super(indexIterator);
+        }
+
+        @Override
+        protected Bytes getBaseKey(final Bytes indexKey) {
+            final Window window = 
KeyFirstSessionKeySchema.extractWindow(indexKey.get());
+            final byte[] key = 
KeyFirstSessionKeySchema.extractKeyBytes(indexKey.get());
+
+            return TimeFirstSessionKeySchema.toBinary(Bytes.wrap(key), 
window.start(), window.end());
+        }
+    }
+
+    RocksDBTimeOrderedSessionSegmentedBytesStore(final String name,
+                                                 final String metricsScope,
+                                                 final long retention,
+                                                 final long segmentInterval,
+                                                 final boolean withIndex) {
+        super(name, metricsScope, retention, segmentInterval, new 
TimeFirstSessionKeySchema(),
+            Optional.ofNullable(withIndex ? new KeyFirstSessionKeySchema() : 
null));
+    }
+
+    public byte[] fetchSession(final Bytes key,
+                               final long earliestSessionEndTime,
+                               final long latestSessionStartTime) {
+        return get(TimeFirstSessionKeySchema.toBinary(
+            key,
+            earliestSessionEndTime,
+            latestSessionStartTime
+        ));
+    }
+
+    public void remove(final Windowed<Bytes> key) {
+        remove(TimeFirstSessionKeySchema.toBinary(key));
+    }
+
+    public void put(final Windowed<Bytes> sessionKey, final byte[] aggregate) {
+        put(TimeFirstSessionKeySchema.toBinary(sessionKey), aggregate);
+    }
+
+    @Override
+    protected KeyValue<Bytes, byte[]> getIndexKeyValue(final Bytes baseKey, 
final byte[] baseValue) {
+        final Window window = 
TimeFirstSessionKeySchema.extractWindow(baseKey.get());
+        final byte[] key = 
TimeFirstSessionKeySchema.extractKeyBytes(baseKey.get());
+        return 
KeyValue.pair(KeyFirstSessionKeySchema.toBinary(Bytes.wrap(key), 
window.start(), window.end()), 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 = 
SessionKeySchema.extractEndTimestamp(record.key());
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+        }
+
+        final Map<KeyValueSegment, WriteBatch> writeBatchMap = new HashMap<>();
+        for (final ConsumerRecord<byte[], byte[]> record : records) {
+            final long timestamp = 
SessionKeySchema.extractEndTimestamp(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 
SessionKeySchema
+                    // from ChangeLoggingSessionBytesStore. Reconstruct 
key/value to restore
+                    if (hasIndex()) {
+                        final byte[] indexKey = 
KeyFirstSessionKeySchema.fromNonPrefixSessionKey(record.key());
+                        // Take care of tombstone
+                        final byte[] value = record.value() == null ? null : 
new byte[0];
+                        segment.addToBatch(new KeyValue<>(indexKey, value), 
batch);
+                    }
+
+                    final byte[] baseKey = 
TimeFirstSessionKeySchema.fromNonPrefixSessionKey(record.key());

Review comment:
       This seems even less efficient than for window stores... I think we 
really need to do some benchmarks to check restoration efficiency asap.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, final 
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 Windowed<Bytes> windowedKey = from(bytes);
+                    final long endTime = windowedKey.window().end();
+                    final long startTime = windowedKey.window().start();
+
+                    // We can return false directly here since keys are sorted 
by end time and if
+                    // we get time smaller than `from`, there won't be time 
within range.
+                    if (!forward && endTime < from) {
+                        return false;
+                    }
+
+                    if ((binaryKeyFrom == null || 
windowedKey.key().compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
windowedKey.key().compareTo(binaryKeyTo) <= 0)
+                        && endTime >= from && startTime <= 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, Long.MAX_VALUE, forward);
+        }
+
+        static long extractStartTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE + 
TIMESTAMP_SIZE);
+        }
+
+        static long extractEndTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE);

Review comment:
       Any reasons we intentionally use different logic for these two 
extracting (the other funcs use `binaryKey.length - TIMESTAMP_SIZE` and 
`binaryKey.length - 2 * TIMESTAMP_SIZE`)?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, final 
boolean forward) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != TIME_FIRST_PREFIX) {

Review comment:
       Should this ever happen? If it does happen should we consider it a bug? 
Ditto for the other `hasNextCondition`.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBTimeOrderedSegmentedBytesStore.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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;

Review comment:
       I'm assuming there's no real logical changes here but just to extract 
the common logic from `RocksDBTimeOrderedSegmentedBytesStore` so I did not make 
a thorough pass. Please let me know if otherwise.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,

Review comment:
       nit: with multiple params that cannot fit in one line, we usually just 
have one param per line, ditto the other place.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, final 
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 Windowed<Bytes> windowedKey = from(bytes);
+                    final long endTime = windowedKey.window().end();
+                    final long startTime = windowedKey.window().start();
+
+                    // We can return false directly here since keys are sorted 
by end time and if
+                    // we get time smaller than `from`, there won't be time 
within range.
+                    if (!forward && endTime < from) {
+                        return false;
+                    }
+
+                    if ((binaryKeyFrom == null || 
windowedKey.key().compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
windowedKey.key().compareTo(binaryKeyTo) <= 0)
+                        && endTime >= from && startTime <= 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, Long.MAX_VALUE, forward);
+        }
+
+        static long extractStartTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE + 
TIMESTAMP_SIZE);
+        }
+
+        static long extractEndTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE);
+        }
+
+        private static <K> K extractKey(final byte[] binaryKey,
+                                        final Deserializer<K> deserializer,
+                                        final String topic) {
+            return deserializer.deserialize(topic, extractKeyBytes(binaryKey));
+        }
+
+        static byte[] extractKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - 2 * 
TIMESTAMP_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, PREFIX_SIZE + 2 * TIMESTAMP_SIZE, 
bytes, 0, bytes.length);
+            return bytes;
+        }
+
+        static Window extractWindow(final byte[] binaryKey) {
+            final ByteBuffer buffer = ByteBuffer.wrap(binaryKey);
+            final long start = buffer.getLong(PREFIX_SIZE + TIMESTAMP_SIZE);
+            final long end = buffer.getLong(PREFIX_SIZE);
+            return new SessionWindow(start, end);
+        }
+
+        public static Windowed<Bytes> from(final Bytes bytesKey) {
+            final byte[] binaryKey = bytesKey.get();
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(Bytes.wrap(extractKeyBytes(binaryKey)), 
window);
+        }
+
+        public static <K> Windowed<K> from(final byte[] binaryKey,
+                                           final Deserializer<K> 
keyDeserializer,
+                                           final String topic) {
+            final K key = extractKey(binaryKey, keyDeserializer, topic);
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(key, window);
+        }
+
+        public static <K> byte[] toBinary(final Windowed<K> sessionKey,
+                                          final Serializer<K> serializer,
+                                          final String topic) {
+            final byte[] bytes = serializer.serialize(topic, sessionKey.key());
+            return toBinary(Bytes.wrap(bytes), sessionKey.window().start(), 
sessionKey.window().end()).get();
+        }
+
+        public static Bytes toBinary(final Windowed<Bytes> sessionKey) {
+            return toBinary(sessionKey.key(), sessionKey.window().start(), 
sessionKey.window().end());
+        }
+
+        public static Bytes toBinary(final Bytes key,
+                                     final long startTime,
+                                     final long endTime) {
+            final byte[] bytes = key.get();
+            final ByteBuffer buf = ByteBuffer.allocate(PREFIX_SIZE + 
bytes.length + 2 * TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(endTime)
+                .putLong(startTime)
+                .put(bytes);
+            return Bytes.wrap(buf.array());
+        }
+
+        public static byte[] fromNonPrefixSessionKey(final byte[] binaryKey) {

Review comment:
       I think I understand the semantic differences now.. maybe we should just 
name these two functions differently, e.g. rename this function as 
`extractWindowBytesFromNonPrefixSessionKey`, and the other function as 
`prefixNonPrefixSessionKey`. WDYT?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedSessionKeySchemas.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.kafka.streams.kstream.internals.SessionWindow;
+import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
+
+import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE;
+
+public class PrefixedSessionKeySchemas {
+
+    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 byte extractPrefix(final byte[] binaryBytes) {
+        return binaryBytes[0];
+    }
+
+    public static class TimeFirstSessionKeySchema implements 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());
+            }
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                // the end timestamp can be as large as possible as long as 
it's larger than start time
+                .putLong(Long.MAX_VALUE)
+                // this is the start timestamp
+                .putLong(to)
+                .put(key.get())
+                .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());
+            }
+
+            return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + 2 * 
TIMESTAMP_SIZE + key.get().length)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(from)
+                .putLong(0L)
+                .put(key.get())
+                .array());
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param to the latest start time
+         * @return
+         */
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            return toBinary(key, to, Long.MAX_VALUE);
+        }
+
+        /**
+         *
+         * @param key the key in the range
+         * @param from the earliest end timestamp in the range
+         * @return
+         */
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            return toBinary(key, 0, Math.max(0, from));
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+            final Bytes binaryKeyTo, final long from, final long to, final 
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 Windowed<Bytes> windowedKey = from(bytes);
+                    final long endTime = windowedKey.window().end();
+                    final long startTime = windowedKey.window().start();
+
+                    // We can return false directly here since keys are sorted 
by end time and if
+                    // we get time smaller than `from`, there won't be time 
within range.
+                    if (!forward && endTime < from) {
+                        return false;
+                    }
+
+                    if ((binaryKeyFrom == null || 
windowedKey.key().compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
windowedKey.key().compareTo(binaryKeyTo) <= 0)
+                        && endTime >= from && startTime <= 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, Long.MAX_VALUE, forward);
+        }
+
+        static long extractStartTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE + 
TIMESTAMP_SIZE);
+        }
+
+        static long extractEndTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(PREFIX_SIZE);
+        }
+
+        private static <K> K extractKey(final byte[] binaryKey,
+                                        final Deserializer<K> deserializer,
+                                        final String topic) {
+            return deserializer.deserialize(topic, extractKeyBytes(binaryKey));
+        }
+
+        static byte[] extractKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - 2 * 
TIMESTAMP_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, PREFIX_SIZE + 2 * TIMESTAMP_SIZE, 
bytes, 0, bytes.length);
+            return bytes;
+        }
+
+        static Window extractWindow(final byte[] binaryKey) {
+            final ByteBuffer buffer = ByteBuffer.wrap(binaryKey);
+            final long start = buffer.getLong(PREFIX_SIZE + TIMESTAMP_SIZE);
+            final long end = buffer.getLong(PREFIX_SIZE);
+            return new SessionWindow(start, end);
+        }
+
+        public static Windowed<Bytes> from(final Bytes bytesKey) {
+            final byte[] binaryKey = bytesKey.get();
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(Bytes.wrap(extractKeyBytes(binaryKey)), 
window);
+        }
+
+        public static <K> Windowed<K> from(final byte[] binaryKey,
+                                           final Deserializer<K> 
keyDeserializer,
+                                           final String topic) {
+            final K key = extractKey(binaryKey, keyDeserializer, topic);
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(key, window);
+        }
+
+        public static <K> byte[] toBinary(final Windowed<K> sessionKey,
+                                          final Serializer<K> serializer,
+                                          final String topic) {
+            final byte[] bytes = serializer.serialize(topic, sessionKey.key());
+            return toBinary(Bytes.wrap(bytes), sessionKey.window().start(), 
sessionKey.window().end()).get();
+        }
+
+        public static Bytes toBinary(final Windowed<Bytes> sessionKey) {
+            return toBinary(sessionKey.key(), sessionKey.window().start(), 
sessionKey.window().end());
+        }
+
+        public static Bytes toBinary(final Bytes key,
+                                     final long startTime,
+                                     final long endTime) {
+            final byte[] bytes = key.get();
+            final ByteBuffer buf = ByteBuffer.allocate(PREFIX_SIZE + 
bytes.length + 2 * TIMESTAMP_SIZE)
+                .put(TIME_FIRST_PREFIX)
+                .putLong(endTime)
+                .putLong(startTime)
+                .put(bytes);
+            return Bytes.wrap(buf.array());
+        }
+
+        public static byte[] fromNonPrefixSessionKey(final byte[] binaryKey) {
+            final ByteBuffer buffer = ByteBuffer.allocate(PREFIX_SIZE + 
binaryKey.length).put(TIME_FIRST_PREFIX);
+            // Put timestamp
+            buffer.put(binaryKey, binaryKey.length - 2 * TIMESTAMP_SIZE, 2 * 
TIMESTAMP_SIZE);
+            buffer.put(binaryKey, 0, binaryKey.length - 2 * TIMESTAMP_SIZE);
+
+            return buffer.array();
+        }
+    }
+
+    public static class KeyFirstSessionKeySchema implements KeySchema {
+
+        @Override
+        public Bytes upperRange(final Bytes key, final long to) {
+            final Bytes noPrefixBytes = new SessionKeySchema().upperRange(key, 
to);
+            return wrapPrefix(noPrefixBytes, true);
+        }
+
+        @Override
+        public Bytes lowerRange(final Bytes key, final long from) {
+            final Bytes noPrefixBytes = new SessionKeySchema().lowerRange(key, 
from);
+            // Wrap at least prefix even key is null
+            return wrapPrefix(noPrefixBytes, false);
+        }
+
+        @Override
+        public Bytes lowerRangeFixedSize(final Bytes key, final long from) {
+            final Bytes noPrefixBytes = new 
SessionKeySchema().lowerRangeFixedSize(key, from);
+            return wrapPrefix(noPrefixBytes, false);
+        }
+
+        @Override
+        public Bytes upperRangeFixedSize(final Bytes key, final long to) {
+            final Bytes noPrefixBytes = new 
SessionKeySchema().upperRangeFixedSize(key, to);
+            return wrapPrefix(noPrefixBytes, true);
+        }
+
+        @Override
+        public long segmentTimestamp(final Bytes key) {
+            return extractEndTimestamp(key.get());
+        }
+
+        @Override
+        public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom,
+                                                 final Bytes binaryKeyTo,
+                                                 final long from,
+                                                 final long to,
+                                                 final boolean forward) {
+            return iterator -> {
+                while (iterator.hasNext()) {
+                    final Bytes bytes = iterator.peekNextKey();
+                    final byte prefix = extractPrefix(bytes.get());
+
+                    if (prefix != KEY_FIRST_PREFIX) {
+                        return false;
+                    }
+
+                    final Windowed<Bytes> windowedKey = from(bytes);
+                    final long endTime = windowedKey.window().end();
+                    final long startTime = windowedKey.window().start();
+
+                    if ((binaryKeyFrom == null || 
windowedKey.key().compareTo(binaryKeyFrom) >= 0)
+                        && (binaryKeyTo == null || 
windowedKey.key().compareTo(binaryKeyTo) <= 0)
+                        && endTime >= from
+                        && startTime <= 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, Long.MAX_VALUE, forward);
+        }
+
+        static Window extractWindow(final byte[] binaryKey) {
+            final ByteBuffer buffer = ByteBuffer.wrap(binaryKey);
+            final long start = buffer.getLong(binaryKey.length - 
TIMESTAMP_SIZE);
+            final long end = buffer.getLong(binaryKey.length - 2 * 
TIMESTAMP_SIZE);
+            return new SessionWindow(start, end);
+        }
+
+        static byte[] extractKeyBytes(final byte[] binaryKey) {
+            final byte[] bytes = new byte[binaryKey.length - 2 * 
TIMESTAMP_SIZE - PREFIX_SIZE];
+            System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length);
+            return bytes;
+        }
+
+        public static Windowed<Bytes> from(final Bytes bytesKey) {
+            final byte[] binaryKey = bytesKey.get();
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(Bytes.wrap(extractKeyBytes(binaryKey)), 
window);
+        }
+
+        private static <K> K extractKey(final byte[] binaryKey,
+                                        final Deserializer<K> deserializer,
+                                        final String topic) {
+            return deserializer.deserialize(topic, extractKeyBytes(binaryKey));
+        }
+
+        public static <K> Windowed<K> from(final byte[] binaryKey,
+                                           final Deserializer<K> 
keyDeserializer,
+                                           final String topic) {
+            final K key = extractKey(binaryKey, keyDeserializer, topic);
+            final Window window = extractWindow(binaryKey);
+            return new Windowed<>(key, window);
+        }
+
+        static long extractStartTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - 
TIMESTAMP_SIZE);
+        }
+
+        static long extractEndTimestamp(final byte[] binaryKey) {
+            return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - 2 * 
TIMESTAMP_SIZE);
+        }
+
+        public static Bytes toBinary(final Windowed<Bytes> sessionKey) {
+            return toBinary(sessionKey.key(), sessionKey.window().start(), 
sessionKey.window().end());
+        }
+
+        public static <K> byte[] toBinary(final Windowed<K> sessionKey,
+                                          final Serializer<K> serializer,
+                                          final String topic) {
+            final byte[] bytes = serializer.serialize(topic, sessionKey.key());
+            return toBinary(Bytes.wrap(bytes), sessionKey.window().start(), 
sessionKey.window().end()).get();
+        }
+
+        public static Bytes toBinary(final Bytes key,
+                                     final long startTime,
+                                     final long endTime) {
+            final byte[] bytes = key.get();
+            final ByteBuffer buf = ByteBuffer.allocate(PREFIX_SIZE + 
bytes.length + 2 * TIMESTAMP_SIZE);
+            buf.put(KEY_FIRST_PREFIX);
+            buf.put(bytes);
+            buf.putLong(endTime);
+            buf.putLong(startTime);
+            return Bytes.wrap(buf.array());
+        }
+
+        private static Bytes wrapPrefix(final Bytes noPrefixKey, final boolean 
upperRange) {
+            // Need to scan from prefix even key is null
+            if (noPrefixKey == null) {
+                final byte prefix = upperRange ? KEY_FIRST_PREFIX + 1 : 
KEY_FIRST_PREFIX;
+                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(KEY_FIRST_PREFIX)
+                .put(noPrefixKey.get())
+                .array();
+            return Bytes.wrap(ret);
+        }
+
+        public static byte[] fromNonPrefixSessionKey(final byte[] binaryKey) {

Review comment:
       Not sure I understand why the two `fromNonPrefixSessionKey` are so 
different.. could you elaborate a bit?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to