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<Bytes, byte[]> { + + + + private final boolean retainDuplicates; + private final long windowSize; + private final String name; + private int seqnum = 0; + private RocksDBTimeOrderedSegmentedBytesStore baseStore; + private RocksDBSegmentedBytesStore indexStore; + + private StateStoreContext stateStoreContext; + private Position position; + private boolean open; + + + private class IndexToBaseStoreIterator implements KeyValueIterator<Bytes, byte[]> { + private final KeyValueIterator<Bytes, byte[]> indexIterator; + private byte[] cachedValue; + + + IndexToBaseStoreIterator(final KeyValueIterator<Bytes, byte[]> indexIterator) { + this.indexIterator = indexIterator; + } + + @Override + public void close() { + indexIterator.close(); + } + + @Override + public Bytes peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return indexIterator.peekNextKey(); + } + + @Override + public boolean hasNext() { + while (indexIterator.hasNext()) { + final Bytes key = indexIterator.peekNextKey(); + final Bytes keyBytes = Bytes.wrap(KeyFirstWindowKeySchema.extractStoreKeyBytes(key.get())); + final long timestamp = KeyFirstWindowKeySchema.extractStoreTimestamp(key.get()); + final int seqnum = KeyFirstWindowKeySchema.extractStoreSeqnum(key.get()); + + cachedValue = baseStore.get(TimeFirstWindowKeySchema.toTimeOrderedStoreKeyBinary(key, timestamp, seqnum)); + if (cachedValue == null) { + // Key not in base store, inconsistency happened. Skip this key and reply on + // segment store to clean this. Review comment: I see, thanks. -- 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