Status

Current state: Under Discussion

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

Author: Arpit Goyal

JIRA KAFKA-16209 - Getting issue details... STATUS

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

The remote log manager currently expects snapshot file to be mandatorily available. Missing snapshot files can happen Kafka is upgraded from any version prior to 2.8.0. When this is the case, The remote log Manager  will break a contract with the RSM API and would result in failure. 

Public Interfaces

Current

Current interface
@InterfaceStability.Evolving

public class LogSegmentData {

    private final Path logSegment;

    private final Path offsetIndex;

    private final Path timeIndex;

    private final Optional<Path> transactionIndex;

    private final Path producerSnapshotIndex;

    private final ByteBuffer leaderEpochIndex;

}


After the proposed change

Proposed interface
@InterfaceStability.Evolving

public class LogSegmentData {

    private final Path logSegment;

    private final Path offsetIndex;

    private final Path timeIndex;

    private final Optional<Path> transactionIndex;

    private final Optional<Path> producerSnapshotIndex;

    private final ByteBuffer leaderEpochIndex;
}

Proposed Changes

Making snapshot file to be an optional parameter similar to transactional index. 

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users -> 
    Without the above change, Existing users who have migrated topic from < 2.8 version will face error(NullPointerException) while using the Tierage Storage feature
    .
  • If we are changing behavior how will we phase out the older behavior?
    This is no change in the older behaviour. The older behaviour will be enhanced to handle the above scenarios.
  • If we need special migration tools, describe them here.
    Not required.
  • When will we remove the existing behavior?
    There is no removal of existing behaviour. But it is an improvement on the  existing behaviour to handle scenarios where ProducerSnapshot can be empty.

Test Plan

Will cover the patch with unit tests.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

  • No labels