-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-19713: Persist Position into offsets CF #21750
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
82458ed
1e8467c
285f8f2
ecdfbc5
ea0162a
543845a
6180727
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,7 +40,6 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.io.File; | ||
| import java.util.Collection; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
|
|
@@ -296,17 +295,14 @@ public void init(final StateStoreContext stateStoreContext, final StateStore roo | |
| metrics | ||
| ); | ||
|
|
||
| final File positionCheckpointFile = new File(stateStoreContext.stateDir(), name() + ".position"); | ||
| this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile); | ||
| this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint); | ||
| segments.setPosition(position); | ||
| segments.openExisting(internalProcessorContext, observedStreamTime); | ||
| this.position = segments.position; | ||
|
|
||
| // register and possibly restore the state from the logs | ||
| stateStoreContext.register( | ||
| root, | ||
| (RecordBatchingStateRestoreCallback) this::restoreAllInternal, | ||
| () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position) | ||
| () -> { } // Nothing to do? | ||
|
||
| ); | ||
|
|
||
| open = true; | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,6 @@ | |
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.kafka.streams.processor.StateStoreContext; | ||
| import org.apache.kafka.streams.processor.internals.ProcessorContextUtils; | ||
| import org.apache.kafka.streams.query.Position; | ||
| import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; | ||
|
|
||
| import java.util.HashMap; | ||
|
|
@@ -56,11 +55,6 @@ public class LogicalKeyValueSegments extends AbstractSegments<LogicalKeyValueSeg | |
| this.physicalStore = new RocksDBStore(name, parentDir, metricsRecorder, false); | ||
| } | ||
|
|
||
| @Override | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why remove
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The reason why I'm removing this is because we are already merging the position from the public void openExisting(final StateStoreContext context, final long streamTime) {
metricsRecorder.init(ProcessorContextUtils.metricsImpl(context), context.taskId());
physicalStore.openDB(context.appConfigs(), context.stateDir());
position.merge(physicalStore.getPosition());
}
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ack thanks for explaining |
||
| public void setPosition(final Position position) { | ||
| this.physicalStore.position = position; | ||
| } | ||
|
|
||
| @Override | ||
| protected LogicalKeyValueSegment createSegment(final long segmentId, final String segmentName) { | ||
| if (segmentId < 0) { | ||
|
|
@@ -100,6 +94,7 @@ LogicalKeyValueSegment getReservedSegment(final long segmentId) { | |
| public void openExisting(final StateStoreContext context, final long streamTime) { | ||
| metricsRecorder.init(ProcessorContextUtils.metricsImpl(context), context.taskId()); | ||
| physicalStore.openDB(context.appConfigs(), context.stateDir()); | ||
| position.merge(physicalStore.getPosition()); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -140,7 +140,6 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BatchWritingS | |
|
|
||
| protected StateStoreContext context; | ||
| protected Position position; | ||
| private OffsetCheckpoint positionCheckpoint; | ||
|
|
||
| public RocksDBStore(final String name, | ||
| final String metricsScope) { | ||
|
|
@@ -170,17 +169,13 @@ public void init(final StateStoreContext stateStoreContext, | |
| metricsRecorder.init(metricsImpl(stateStoreContext), stateStoreContext.taskId()); | ||
| openDB(stateStoreContext.appConfigs(), stateStoreContext.stateDir()); | ||
|
|
||
| final File positionCheckpointFile = new File(stateStoreContext.stateDir(), name() + ".position"); | ||
| this.positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile); | ||
| this.position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint); | ||
|
|
||
| // value getter should always read directly from rocksDB | ||
| // since it is only for values that are already flushed | ||
| this.context = stateStoreContext; | ||
| stateStoreContext.register( | ||
| root, | ||
| (RecordBatchingStateRestoreCallback) this::restoreBatch, | ||
| () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position) | ||
| this::writePosition | ||
| ); | ||
| consistencyEnabled = StreamsConfig.InternalConfig.getBoolean( | ||
| stateStoreContext.appConfigs(), | ||
|
|
@@ -252,7 +247,7 @@ void openDB(final Map<String, Object> configs, final File stateDir) { | |
| openRocksDB(dbOptions, columnFamilyOptions); | ||
| dbAccessor = new DirectDBAccessor(db, fOptions, wOptions); | ||
| try { | ||
| cfAccessor.open(dbAccessor, !eosEnabled); | ||
| position = cfAccessor.open(dbAccessor, !eosEnabled); | ||
| } catch (final StreamsException fatal) { | ||
| final String fatalMessage = "State store " + name + " didn't find a valid state, since under EOS it has the risk of getting uncommitted data in stores"; | ||
| throw new ProcessorStateException(fatalMessage, fatal); | ||
|
|
@@ -396,6 +391,16 @@ private List<ColumnFamilyHandle> mergeColumnFamilyHandleLists(final List<ColumnF | |
| return columnFamilies; | ||
| } | ||
|
|
||
| private void writePosition() { | ||
| validateStoreOpen(); | ||
| try { | ||
| cfAccessor.commit(dbAccessor, position); | ||
| } catch (final RocksDBException e) { | ||
| // TODO: fatal error? | ||
| throw new RuntimeException(e); | ||
|
||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public String name() { | ||
| return name; | ||
|
|
@@ -906,6 +911,8 @@ ManagedKeyValueIterator<Bytes, byte[]> range(final DBAccessor accessor, | |
|
|
||
| void commit(final DBAccessor accessor, final Map<TopicPartition, Long> changelogOffsets) throws RocksDBException; | ||
|
|
||
| void commit(final DBAccessor accessor, final Position storePosition) throws RocksDBException; | ||
|
|
||
| void addToBatch(final byte[] key, | ||
| final byte[] value, | ||
| final WriteBatchInterface batch) throws RocksDBException; | ||
|
|
@@ -916,7 +923,7 @@ void addToBatch(final byte[] key, | |
| * Initializes the ColumnFamily. | ||
| * @throws StreamsException if an invalid state is found and ignoreInvalidState is false | ||
| */ | ||
| void open(final RocksDBStore.DBAccessor accessor, final boolean ignoreInvalidState) throws RocksDBException, StreamsException; | ||
| Position open(final RocksDBStore.DBAccessor accessor, final boolean ignoreInvalidState) throws RocksDBException, StreamsException; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Need to update JavaDoc that |
||
|
|
||
| Long getCommittedOffset(final RocksDBStore.DBAccessor accessor, final TopicPartition partition) throws RocksDBException; | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -50,7 +50,6 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.io.File; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.ArrayList; | ||
| import java.util.Collection; | ||
|
|
@@ -366,17 +365,14 @@ public void init(final StateStoreContext stateStoreContext, final StateStore roo | |
|
|
||
| metricsRecorder.init(ProcessorContextUtils.metricsImpl(stateStoreContext), stateStoreContext.taskId()); | ||
|
|
||
| final File positionCheckpointFile = new File(stateStoreContext.stateDir(), name() + ".position"); | ||
| positionCheckpoint = new OffsetCheckpoint(positionCheckpointFile); | ||
| position = StoreQueryUtils.readPositionFromCheckpoint(positionCheckpoint); | ||
| segmentStores.setPosition(position); | ||
| segmentStores.openExisting(internalProcessorContext, observedStreamTime); | ||
| this.position = segmentStores.position; | ||
|
|
||
| // register and possibly restore the state from the logs | ||
| stateStoreContext.register( | ||
| root, | ||
| (RecordBatchingStateRestoreCallback) RocksDBVersionedStore.this::restoreBatch, | ||
| () -> StoreQueryUtils.checkpointPosition(positionCheckpoint, position) | ||
| () -> { } // Nothing to do? | ||
|
||
| ); | ||
|
|
||
| open = true; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Write the position here, since on open/read we merge positions for all segments. Plus the cost is small since it's small entry. Same for the other callbacks
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @bbejeck I addressed this.
At first, I was confused about this because the underlying store should be supposed to persist the position after commit, but that's not true for segmented stores.