[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
guozhangwang commented on a change in pull request #11802: URL: https://github.com/apache/kafka/pull/11802#discussion_r823215870 ## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/PrefixedWindowKeySchemas.java ## @@ -0,0 +1,405 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import java.util.Arrays; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windowed; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.kafka.streams.state.StateSerdes; + +import static org.apache.kafka.streams.state.StateSerdes.TIMESTAMP_SIZE; +import static org.apache.kafka.streams.state.internals.WindowKeySchema.timeWindowForSize; + +public class PrefixedWindowKeySchemas { + +private static final int PREFIX_SIZE = 1; +private static final byte TIME_FIRST_PREFIX = 0; +private static final byte KEY_FIRST_PREFIX = 1; +private static final int SEQNUM_SIZE = 4; +private static final int SUFFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE; + +private static byte extractPrefix(final byte[] binaryBytes) { +return binaryBytes[0]; +} + +public static class TimeFirstWindowKeySchema implements RocksDBSegmentedBytesStore.KeySchema { + +@Override +public Bytes upperRange(final Bytes key, final long to) { +if (key == null) { +// Put next prefix instead of null so that we can start from right prefix +// when scanning backwards +final byte nextPrefix = TIME_FIRST_PREFIX + 1; +return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE).put(nextPrefix).array()); +} +byte[] maxKey = new byte[key.get().length]; +Arrays.fill(maxKey, (byte) 0xFF); +return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE + maxKey.length + SEQNUM_SIZE) +.put(TIME_FIRST_PREFIX) +.putLong(to) +.put(maxKey).putInt(Integer.MAX_VALUE) +.array()); +} + +@Override +public Bytes lowerRange(final Bytes key, final long from) { +if (key == null) { +return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE) +.put(TIME_FIRST_PREFIX) +.putLong(from) +.array()); +} + +/* + * Larger timestamp or key's byte order can't be smaller than this lower range. Reason: + * 1. Timestamp is fixed length (with big endian byte order). Since we put timestamp + *first, larger timestamp will have larger byte order. + * 2. If timestamp is the same but key (k1) is larger than this lower range key (k2): + * a. If k2 is not a prefix of k1, then k1 will always have larger byte order no + *matter what seqnum k2 has + * b. If k2 is a prefix of k1, since k2's seqnum is 0, after k1 appends seqnum, + *it will always be larger than (k1 + seqnum). + */ +return Bytes.wrap(ByteBuffer.allocate(PREFIX_SIZE + TIMESTAMP_SIZE + key.get().length) +.put(TIME_FIRST_PREFIX) +.putLong(from) +.put(key.get()) +.array()); +} + +@Override +public Bytes lowerRangeFixedSize(final Bytes key, final long from) { +return TimeFirstWindowKeySchema.toStoreKeyBinary(key, Math.max(0, from), +0); +} + +@Override +public Bytes upperRangeFixedSize(final Bytes key, final long to) { +return TimeFirstWindowKeySchema.toStoreKeyBinary(key, to, Integer.MAX_VALUE); +} + +@Override +public long segmentTimestamp(final Bytes key) { +return TimeFirstWindowKeySchema.extractStoreTimestamp(key.get()); +} + +@Override +public HasNextCondition hasNextCondition(final Bytes bi
[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
guozhangwang commented on a change in pull request #11802: URL: https://github.com/apache/kafka/pull/11802#discussion_r823200925 ## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java ## @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import java.util.Optional; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.ProcessorContextUtils; +import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback; +import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.rocksdb.RocksDBException; +import org.rocksdb.WriteBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED; + +public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore implements SegmentedBytesStore { +private static final Logger LOG = LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class); + +private final String name; +protected final AbstractSegments segments; +private final String metricScope; +protected final KeySchema baseKeySchema; +protected final Optional indexKeySchema; + + +private ProcessorContext context; +private StateStoreContext stateStoreContext; Review comment: Filed https://issues.apache.org/jira/browse/KAFKA-13722 -- 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
[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
guozhangwang commented on a change in pull request #11802: URL: https://github.com/apache/kafka/pull/11802#discussion_r822940645 ## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java ## @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import java.util.Optional; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.ProcessorStateException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.ProcessorContextUtils; +import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback; +import org.apache.kafka.streams.processor.internals.StoreToProcessorContextAdapter; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; +import org.apache.kafka.streams.query.Position; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.rocksdb.RocksDBException; +import org.rocksdb.WriteBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED; + +public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore implements SegmentedBytesStore { +private static final Logger LOG = LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class); + +private final String name; +protected final AbstractSegments segments; +private final String metricScope; +protected final KeySchema baseKeySchema; +protected final Optional indexKeySchema; + + +private ProcessorContext context; +private StateStoreContext stateStoreContext; +private Sensor expiredRecordSensor; +private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; +private boolean consistencyEnabled = false; +private Position position; +protected OffsetCheckpoint positionCheckpoint; +private volatile boolean open; + +AbstractDualSchemaRocksDBSegmentedBytesStore(final String name, + final String metricScope, + final KeySchema baseKeySchema, + final Optional indexKeySchema, + final AbstractSegments segments) { +this.name = name; +this.metricScope = metricScope; +this.baseKeySchema = baseKeySchema; +this.indexKeySchema = indexKeySchema; +this.segments = segments; +} + +@Override +public KeyValueIterator all() { +final List searchSpace = segments.allSegments(true); + +return new SegmentIterator<>( +searchSpace.iterator(), +baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), +null, +null, +true); +} + +@Override +public KeyValueIterator backwardAll() { +final List searchSpace = segments.allSegments(false); + +return new SegmentIterator<>( +searchSpace.iterator(), +baseKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE), +null, +null, +false); +} + +@Override +public void remove(final Bytes rawKey) { +final long timestamp = baseKeySchema.segmentTimestamp(rawKey); +observedStreamTime = Math.max(observedStreamTime, timestamp); +final S segment = segments.getSegmentForTimestamp(timestamp); +if (segment == null) { +return; +} +segment.delete(rawKey); +} + +
[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
guozhangwang commented on a change in pull request #11802: URL: https://github.com/apache/kafka/pull/11802#discussion_r818348258 ## 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: Basically I was wondering if it is really necessary to pad four 0 bytes on the key schema. We use `seqnum` for window stores that are used in stream-stream joins, since we need to maintain values of the same `[key, timestamp]`, but for aggregation windowed key we would not need to maintain uniqueness, and hence not necessary to use `seqnum`. I'm basically asking if just `[from, key]` is good enough as the lower range. -- 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
[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
guozhangwang commented on a change in pull request #11802: URL: https://github.com/apache/kafka/pull/11802#discussion_r818345900 ## 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 { + + + +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 { +private final KeyValueIterator indexIterator; +private byte[] cachedValue; + + +IndexToBaseStoreIterator(final KeyValueIterator 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.
[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
guozhangwang commented on a change in pull request #11802: URL: https://github.com/apache/kafka/pull/11802#discussion_r817311291 ## 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; Review comment: Another nitpick: to use 0 as the base store prefix, and 1 as indices and so on; the main thinking is that in the future we may extend it to have multiple indices with a single base. -- 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
[GitHub] [kafka] guozhangwang commented on a change in pull request #11802: [RFC][1/N]add new RocksDBTimeOrderedWindowStore
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