nicktelford commented on code in PR #12393:
URL: https://github.com/apache/kafka/pull/12393#discussion_r1005391821


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractTransactionalStore.java:
##########
@@ -0,0 +1,442 @@
+/*
+ * 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 static 
org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+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.RecordBatchingStateRestoreCallback;
+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.KeyValueStore;
+import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder;
+
+public abstract class AbstractTransactionalStore<T extends 
KeyValueStore<Bytes, byte[]>> implements KeyValueStore<Bytes, byte[]> {
+    private static final byte MODIFICATION = 0x1;
+    private static final byte DELETION = 0x2;
+    private static final byte[] DELETION_VAL = {DELETION};
+
+    private StateStoreContext context;
+
+    static final String PREFIX = "transactional-";
+    //VisibleForTesting
+    public static final String TMP_SUFFIX = ".tmp";
+
+    private final Set<MergeKeyValueIterator> openIterators = 
Collections.synchronizedSet(new HashSet<>());
+
+    Map<String, Object> configs;
+    File stateDir;
+
+    private boolean consistencyEnabled = false;
+    private Position position;
+    protected OffsetCheckpoint positionCheckpoint;
+
+    KeyValueSegment createTmpStore(final String segmentName,
+                                   final String windowName,
+                                   final long segmentId,
+                                   final RocksDBMetricsRecorder 
metricsRecorder) {
+        return new KeyValueSegment(segmentName + TMP_SUFFIX,
+                                    windowName,
+                                    segmentId,
+                                    metricsRecorder);
+    }
+
+    public abstract T mainStore();
+
+    public abstract KeyValueSegment tmpStore();
+
+    @Deprecated
+    @Override
+    public void init(final ProcessorContext context, final StateStore root) {
+        if (context instanceof StateStoreContext) {
+            init((StateStoreContext) context, root);
+        } else {
+            throw new UnsupportedOperationException(
+                "Use TransactionalKeyValueStore#init(StateStoreContext, 
StateStore) instead."
+            );
+        }
+    }
+
+    @Override
+    public void init(final StateStoreContext context, final StateStore root) {
+        this.context = context;
+
+        doInit(context.appConfigs(), context.stateDir());
+        ((RocksDBStore) mainStore()).openDB(configs, stateDir);
+
+        final File positionCheckpointFile = new File(context.stateDir(), 
name() + ".position");
+        this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile);
+        this.position = 
StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint);
+        tmpStore().consistencyEnabled = consistencyEnabled;
+
+        // register and possibly restore the state from the logs
+        context.register(
+            root,
+            (RecordBatchingStateRestoreCallback) this::restoreBatch,
+            () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, 
position)
+        );
+
+        consistencyEnabled = StreamsConfig.InternalConfig.getBoolean(
+            context.appConfigs(),
+            IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED,
+            false);
+    }
+
+    private void restoreBatch(final Collection<ConsumerRecord<byte[], byte[]>> 
records) {
+        final Collection<ConsumerRecord<byte[], byte[]>> changelogRecords = 
records
+            .stream()
+            .map(record -> new ConsumerRecord<>(
+                record.topic(),
+                record.partition(),
+                record.offset(),
+                record.key(),
+                toUncommittedValue(record.value())))
+            .collect(Collectors.toList());
+        tmpStore().restoreBatch(changelogRecords);
+        commit(null);
+    }
+
+    void doInit(final Map<String, Object> configs, final File stateDir) {
+        this.configs = configs;
+        this.stateDir = stateDir;
+        tmpStore().openDB(configs, stateDir);
+    }
+
+    @Override
+    public synchronized void close() {
+        final HashSet<KeyValueIterator<Bytes, byte[]>> iterators;
+        synchronized (openIterators) {
+            iterators = new HashSet<>(openIterators);
+        }
+        for (final KeyValueIterator<Bytes, byte[]> iterator : iterators) {
+            iterator.close();
+        }
+
+        tmpStore().close();
+        mainStore().close();
+    }
+
+    @Override
+    public void commit(final Long changelogOffset) {
+        tmpStore().commit(changelogOffset);
+        doCommit();
+    }

Review Comment:
   Thinking about it, since the goal is to reduce write-amplification from 
flushing the memtables on every commit, we should ideally only flush memtables 
when they become large enough to trigger the usual RocksDB flush criteria. In 
which case, instead of a `checkpoint.interval.ms`, perhaps we should have 
something like `checkpoint.dirty.records.threshold` instead, which sets the 
threshold of data written to `tmpStore` that triggers the checkpointing? This 
would essentially set a (loose) upper-bound on the amount of records 
(per-store-partition), that need to be restored in the event of a failure.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java:
##########
@@ -220,6 +221,32 @@ void registerStateStores(final List<StateStore> allStores, 
final InternalProcess
             }
             log.trace("Registered state store {}", store.name());
         }
+        checkMixedTxnStores();
+    }
+
+    private void checkMixedTxnStores() {
+        final Set<String> nonTxnStores = new HashSet<>();
+        final Set<String> txnStores = new HashSet<>();
+        final Collection<StateStore> stateStores = stores
+            .values()
+            .stream()
+            .map(sm -> sm.stateStore)
+            .collect(Collectors.toList());
+        for (final StateStore store : stateStores) {
+            if (store.transactional()) {
+                txnStores.add(store.name());

Review Comment:
   This could more efficiently be done with boolean flags for 
`hasTransactionalStores` and `hasNonTransactionalStores`.



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