mjsax commented on code in PR #13188:
URL: https://github.com/apache/kafka/pull/13188#discussion_r1100890258


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBVersionedStore.java:
##########
@@ -0,0 +1,792 @@
+/*
+ * 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.io.File;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+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.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.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import 
org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue;
+import 
org.apache.kafka.streams.state.internals.RocksDBVersionedStoreSegmentValueFormatter.SegmentValue.SegmentSearchResult;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A persistent, versioned key-value store based on RocksDB.
+ * <p>
+ * This store implementation consists of a "latest value store" and "segment 
stores." The latest
+ * record version for each key is stored in the latest value store, while 
older record versions
+ * are stored in the segment stores. Conceptually, each record version has two 
associated
+ * timestamps:
+ * <ul>
+ *     <li>a {@code validFrom} timestamp. This timestamp is explicitly 
associated with the record
+ *     as part of the {@link VersionedKeyValueStore#put(Object, Object, 
long)}} call to the store;
+ *     i.e., this is the record's timestamp.</li>
+ *     <li>a {@code validTo} timestamp. This is the timestamp of the next 
record (or deletion)
+ *     associated with the same key, and is implicitly associated with the 
record. This timestamp
+ *     can change as new records are inserted into the store.</li>
+ * </ul>
+ * The validity interval of a record is from validFrom (inclusive) to validTo 
(exclusive), and
+ * can change as new record versions are inserted into the store (and validTo 
changes as a result).
+ * <p>
+ * Old record versions are stored in segment stores according to their validTo 
timestamps. This
+ * allows for efficient expiry of old record versions, as entire segments can 
be dropped from the
+ * store at a time, once the records contained in the segment are no longer 
relevant based on the
+ * store's history retention (for an explanation of "history retention", see
+ * {@link VersionedKeyValueStore}). Multiple record versions (for the same 
key) within a single
+ * segment are stored together using the format specified in {@link 
RocksDBVersionedStoreSegmentValueFormatter}.
+ */
+public class RocksDBVersionedStore implements VersionedKeyValueStore<Bytes, 
byte[]> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBVersionedStore.class);
+    // a marker to indicate that no record version has yet been found as part 
of an ongoing
+    // put() procedure. any value which is not a valid record timestamp will 
do.
+    private static final long SENTINEL_TIMESTAMP = Long.MIN_VALUE;
+
+    private final String name;
+    private final long historyRetention;
+    private final RocksDBMetricsRecorder metricsRecorder;
+
+    private final RocksDBStore latestValueStore;
+    private final LogicalKeyValueSegments segmentStores;
+    private final LatestValueSchema latestValueSchema;
+    private final SegmentValueSchema segmentValueSchema;
+    private final VersionedStoreClient<LogicalKeyValueSegment> 
versionedStoreClient;
+
+    private ProcessorContext context;
+    private StateStoreContext stateStoreContext;
+    private Sensor expiredRecordSensor;
+    private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+    private Position position;
+    private OffsetCheckpoint positionCheckpoint;
+    private volatile boolean open;
+
+    RocksDBVersionedStore(final String name, final String metricsScope, final 
long historyRetention, final long segmentInterval) {
+        this.name = name;
+        this.historyRetention = historyRetention;
+        this.metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name);
+        this.latestValueStore = new RocksDBStore(lvsName(name), name, 
metricsRecorder);
+        this.segmentStores = new LogicalKeyValueSegments(segmentsName(name), 
name, historyRetention, segmentInterval, metricsRecorder);
+        this.latestValueSchema = new LatestValueSchema();
+        this.segmentValueSchema = new SegmentValueSchema();
+        this.versionedStoreClient = new RocksDBVersionedStoreClient();
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value, final long timestamp) 
{
+
+        doPut(
+            latestValueSchema,
+            segmentValueSchema,
+            versionedStoreClient,
+            Optional.of(expiredRecordSensor),

Review Comment:
   I am confused. `doPut()` is private and I don't see a call to it on the 
other PR. What do I miss?



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

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to