-
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 all commits
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 |
|---|---|---|
|
|
@@ -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) { | ||
|
|
@@ -169,18 +168,15 @@ public void init(final StateStoreContext stateStoreContext, | |
| // open the DB dir | ||
| 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); | ||
| StoreQueryUtils.maybeMigrateExistingPositionFile(stateStoreContext.stateDir(), name(), position); | ||
|
|
||
| // 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 +248,13 @@ void openDB(final Map<String, Object> configs, final File stateDir) { | |
| openRocksDB(dbOptions, columnFamilyOptions); | ||
| dbAccessor = new DirectDBAccessor(db, fOptions, wOptions); | ||
| try { | ||
| cfAccessor.open(dbAccessor, !eosEnabled); | ||
| final Position existingPositionOrEmpty = cfAccessor.open(dbAccessor, !eosEnabled); | ||
| if (position == null) { | ||
| position = existingPositionOrEmpty; | ||
| } else { | ||
| // For segmented stores, the overall position is composed of multiple underlying stores, so merge this store's position into it. | ||
| position.merge(existingPositionOrEmpty); | ||
| } | ||
| } 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 +398,15 @@ private List<ColumnFamilyHandle> mergeColumnFamilyHandleLists(final List<ColumnF | |
| return columnFamilies; | ||
| } | ||
|
|
||
| public final void writePosition() { | ||
| validateStoreOpen(); | ||
| try { | ||
| cfAccessor.commit(dbAccessor, position); | ||
| } catch (final RocksDBException e) { | ||
| log.warn("Error while committing position for store {}", name, e); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public String name() { | ||
| return name; | ||
|
|
@@ -906,6 +917,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; | ||
|
|
@@ -914,9 +927,10 @@ void addToBatch(final byte[] key, | |
|
|
||
| /** | ||
| * Initializes the ColumnFamily. | ||
| * @return the position of the store based on the data in the ColumnFamily. If no offset position is found, an empty position is returned. | ||
| * @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 |
|---|---|---|
|
|
@@ -48,6 +48,7 @@ | |
| import org.apache.kafka.streams.state.WindowStore; | ||
| import org.apache.kafka.streams.state.WindowStoreIterator; | ||
|
|
||
| import java.io.File; | ||
| import java.io.IOException; | ||
| import java.io.PrintWriter; | ||
| import java.io.StringWriter; | ||
|
|
@@ -164,6 +165,14 @@ public static void updatePosition( | |
| } | ||
| } | ||
|
|
||
| public static void maybeMigrateExistingPositionFile(final File stateDir, final String storeName, final Position position) { | ||
| final File positionCheckpointFile = new File(stateDir, storeName + ".position"); | ||
| if (positionCheckpointFile.exists()) { | ||
| final Position existingPosition = readPositionFromCheckpoint(new OffsetCheckpoint(positionCheckpointFile)); | ||
| position.merge(existingPosition); | ||
|
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. Should we delete the position file here after merging?
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. Yes, I think we should delete the position file here, but we will need to recreate it when executing the downgrade process. I didn't do it on this PR because I thought it was part of KAFKA-19710 |
||
| } | ||
| } | ||
|
|
||
| public static boolean isPermitted( | ||
| final Position position, | ||
| final PositionBound positionBound, | ||
|
|
||
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.
Why remove
setPosition? I think we need to add this back asLogicalKeyValueSegments.physicalStoreis a standalone RocksDBStore whose position is set independently duringopenDB().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.
The reason why I'm removing this is because we are already merging the position from the
physicalStoreto the segments position. So basically,physicalStorereads the position offsets duringopenDB()andAbstractSegmentsmerge that position into its own position. The position defined atAbstractSementsis shared among all theLogicalKeyValueSegments. See line 97 on this class.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.
ack thanks for explaining