Gerrrr commented on code in PR #12393: URL: https://github.com/apache/kafka/pull/12393#discussion_r998842122
########## 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(); + } + + @Override + public boolean recover(final Long changelogOffset) { + truncateTmpStore(); + return true; + } + + private void truncateTmpStore() { + try { + tmpStore().close(); + tmpStore().destroy(); + tmpStore().openDB(configs, stateDir); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean persistent() { + return mainStore().persistent(); + } + + @Override + public boolean isOpen() { + return tmpStore().isOpen() && mainStore().isOpen(); + } + + @Override + public void put(final Bytes key, final byte[] value) { + StoreQueryUtils.updatePosition(position, context); + tmpStore().put(key, toUncommittedValue(value)); + } + + @Override + public byte[] putIfAbsent(final Bytes key, final byte[] value) { + final byte[] prev = get(key); + if (prev == null) { + StoreQueryUtils.updatePosition(position, context); + tmpStore().put(key, toUncommittedValue(value)); + } + return prev; + } + + @Override + public void putAll(final List<KeyValue<Bytes, byte[]>> entries) { + StoreQueryUtils.updatePosition(position, context); + final List<KeyValue<Bytes, byte[]>> tmpEntries = entries + .stream() + .map(e -> new KeyValue<>(e.key, toUncommittedValue(e.value))) + .collect(Collectors.toList()); + tmpStore().putAll(tmpEntries); + } + + @Override + public byte[] delete(final Bytes key) { + final byte[] value = get(key); + tmpStore().put(key, DELETION_VAL); + return value; + } + + @Override + public byte[] get(final Bytes key) { + final byte[] tmpValue = tmpStore().get(key); + if (tmpValue == null) { + return mainStore().get(key); + } else if (tmpValue[0] == DELETION) { + return null; + } else { + return fromUncommittedValue(tmpValue); + } + } + + @Override + public KeyValueIterator<Bytes, byte[]> range(final Bytes from, final Bytes to) { + final MergeKeyValueIterator iterator = new MergeKeyValueIterator( + tmpStore().range(from, to), mainStore().range(from, to), openIterators); + openIterators.add(iterator); + return iterator; + } + + @Override + public KeyValueIterator<Bytes, byte[]> reverseRange(final Bytes from, final Bytes to) { + final MergeKeyValueIterator iterator = new MergeKeyValueIterator( + tmpStore().reverseRange(from, to), + mainStore().reverseRange(from, to), + true, + openIterators); + openIterators.add(iterator); + return iterator; + } + + @Override + public KeyValueIterator<Bytes, byte[]> all() { + final MergeKeyValueIterator iterator = new MergeKeyValueIterator( + tmpStore().all(), mainStore().all(), openIterators); + openIterators.add(iterator); + return iterator; + } + + @Override + public KeyValueIterator<Bytes, byte[]> reverseAll() { + final MergeKeyValueIterator iterator = new MergeKeyValueIterator( + tmpStore().reverseAll(), mainStore().reverseAll(), true, openIterators); + openIterators.add(iterator); + return iterator; + } + + @Override + public long approximateNumEntries() { + try { + return Math.addExact(tmpStore().approximateNumEntries(), mainStore().approximateNumEntries()); + } catch (final ArithmeticException e) { + return Long.MAX_VALUE; + } + } + + @Override + public Position getPosition() { + return position; + } + + private void doCommit() { + try (final KeyValueIterator<Bytes, byte[]> it = tmpStore().all()) { + while (it.hasNext()) { + final KeyValue<Bytes, byte[]> kv = it.next(); + mainStore().put(kv.key, fromUncommittedValue(kv.value)); + } + } + + truncateTmpStore(); + } + + @Override + public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound, + final QueryConfig config) { + return StoreQueryUtils.handleBasicQueries( + query, + positionBound, + config, + this, + position, + context + ); + } + + @Override + public <PS extends Serializer<P>, P> KeyValueIterator<Bytes, byte[]> prefixScan(final P prefix, + final PS prefixKeySerializer) { + final MergeKeyValueIterator iterator = new MergeKeyValueIterator( + tmpStore().prefixScan(prefix, prefixKeySerializer), + mainStore().prefixScan(prefix, prefixKeySerializer), + openIterators); + openIterators.add(iterator); + return iterator; + } + + private static KeyValue<Bytes, byte[]> fromUncommittedKV(final KeyValue<Bytes, byte[]> kv) { + if (kv.value[0] == DELETION) { + return null; + } else { + final byte[] value = new byte[kv.value.length - 1]; + System.arraycopy(kv.value, 1, value, 0, value.length); + return new KeyValue<>(kv.key, value); + } + } + + private static byte[] fromUncommittedValue(final byte[] value) { Review Comment: Thank you for the suggestion! I see one issue with this approach that I am unsure how to fix. `AbstractTransactionalStore#get` should be able to differentiate between a key that is absent in the `tmpStore` and a key that is shadowed by a tombstone. Consider the following sequence: 1. `txnDB.put("foo", 1);` 2. `txnDB.commit(1L);` 3. `txnDB.delete("foo");` 4. `txnDB.get("foo");` The `get` operation at step 4 should return `null`. I am not sure if it is possible to: 1. Distinguish between an absent key in RocksDB and a key covered by a tombstone. 2. Prevent RocksDB from purging tombstones. This step is required to avoid situations where compaction between steps 2 and 3 purges the tombstone for `foo`. Please let me know if you have ideas to overcome this issue. -- 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