Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Rename getCommittedOffset to committedOffset

...

Code Block
languagejava
firstline106
titleorg.apache.kafka.streams.processor.StateStore
linenumberstrue
    
    /**
     * Determines if this StateStore manages its own offsets.
     * <p>
     * If this method returns {@code true}, then offsets provided to {@link #commit(Map)} will be retrievable using
     * {@link #getCommittedOffset#committedOffset(TopicPartition)}, even if the store is {@link #close() closed} and later re-opened.
     * <p>
     * If this method returns {@code false}, offsets provided to {@link #commit(Map)} will be ignored, and {@link
     * #getCommittedOffset#committedOffset(TopicPartition)} will be expected to always return {@code null}.
     * <p>
     * This method is provided to enable custom StateStores to opt-in to managing their own offsets. This is highly
     * recommended, if possible, to ensure that custom StateStores provide the consistency guarantees that Kafka Streams
     * expects when operating under the {@code exactly-once} {@code processing.mode}.
     * 
     * @return Whether this StateStore manages its own offsets.
     */
    default boolean managesOffsets() {
        return false;
    }

    /**
     * Flush any cached data
     * 
     * @deprecated Use {@link org.apache.kafka.streams.processor.api.ProcessingContext#commit() ProcessorContext#commit()}
     *             instead.
     */
    @Deprecated
    default void flush() {
        // no-op
    }

    /**
     * Commit all written records to this StateStore.
     * <p>
     * This method <b>MUST NOT<b> be called by users from {@link org.apache.kafka.streams.processor.api.Processor
     * processors}, as doing so may violate the consistency guarantees provided by this store, and expected by Kafka
     * Streams.
     * <p>
     * Instead, users should call {@link org.apache.kafka.streams.processor.api.ProcessingContext#commit() 
     * ProcessorContext#commit()} to request a Task commit.
     * <p>
     * If {@link #persistent()} returns {@code true}, all writes since the last {@link #commit(Map)}, or since {@link
     * #init(StateStore)} <em>MUST</em> be available to readers, even after a restart.
     * <p>
     * If {@link #managesOffsets()} also returns {@code true}, the given {@code changelogOffsets} will be guaranteed
     * to be persisted to disk along with the written records.
     * <p>
     * {@code changelogOffsets} will usually contain a single partition, in the case of a regular StateStore. However,
     * they may contain multiple partitions in the case of a Global StateStore with multiple partitions. All provided
     * partitions <em>MUST</em> be persisted to disk.
     * <p>
     * Implementations <em>SHOULD</em> ensure that {@code changelogOffsets} are committed to disk atomically with the
     * records they represent, if possible.
     * 
     * @param changelogOffsets The changelog offset(s) corresponding to the most recently written records.
     */
    default void commit(final Map<TopicPartition, Long> changelogOffsets) {
        flush();
    }

    /**
     * Returns the most recently {@link #commit(Map) committed} offset for the given {@link TopicPartition}.
     * <p>
     * If {@link #managesOffsets()} and {@link #persistent()} both return {@code true}, this method will return the
     * offset that corresponds to the changelog record most recently written to this store, for the given {@code
     * partition}.
     * <p>
     * This method <i>MUST</i> return the correct offset even if the store is not currently {@link #isOpen() open}.
     * 
     * @param partition The partition to get the committed offset for.
     * @return The last {@link #commit(Map) committed} offset for the {@code partition}; or {@code null} if no offset
     *         has been committed for the partition, or if either {@link #persistent()} or {@link #managesOffsets()}
     *         return {@code false}.
     */
    default Long getCommittedOffsetcommittedOffset(final TopicPartition partition) {
        return null;
    }

...

To ensure that offsets can be returned by getCommittedOffsetcommittedOffset, even with the store is closed (i.e. after close() or before init()), we will write out the latest value for all changelog offsets to a separate file on-disk in the close() method, which will be read from when isOpen() returns false. This file will be deleted during init, and the offsets will be read directly from the column family while the database is open.

...

Input partition "Position" offsets, introduced by KIP-796: Interactive Query v2, are currently stored in a .position file directly by the RocksDBStore implementation. To ensure consistency with the committed data and changelog offsets, these position offsets will be stored in RocksDB, in the same column family as the changelog offsets, instead of the .position file. When a StateStore that manages its own offsets is first initialized, if a .position file exists in the store directory, its offsets will be automatically migrated into the store, and the file will be deleted.

...

With our new API, this file will not contain the offsets for stores that manage changelog offsets themselves. So we need to modify the TaskManager to read not from the.checkpoint file, but from each StateStore by calling getCommittedOffsetcommittedOffset. For this reason, getCommittedOffset committedOffset is required to provide the committed offset even if the store has not been initialized via initStateStore  implementations MUST ensure they can provide offsets when the store is closed/not yet initialized without incurring a significant penalty (e.g. by having to open then close the store).

...

The existing tests that depend on .checkpoint files will either be removed (if they are no longer relevant), or modified to instead call StateStore#getCommittedOffsetStateStore#committedOffset().

Much of the existing behaviour, and all new behaviour, will be provided by specific StateStore implementations/wrappers, instead of implemented directly in the Streams engine. This should simplify testing, as much more of this behaviour can be tested with unit tests, with much less mocking required.

...