Status

Current state: Accepted

Discussion thread: here

Vote thread: here, and here

Vote result: Accepted with 3 binding +1 votes (Matthias J Sax, Bill Bejeck, Chia-Ping Tsai), 2 non-binding +1 votes (Kirk True, Sean Quah)

JIRA: KAFKA-17939

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

Motivation

  • Scenario:

A few classes in org.apache.kafka.common.utils, such as Bytes and Time, are currently exposed through public API interfaces in Kafka Streams and other components, even though they are not part of the public API themselves.

Note: After discussing with the community, this KIP focuses only on Bytes because it is straightforward and uncontroversial. The Time API will be addressed in a separate KIP because it needs a more detailed assessment due to its design considerations and dependencies (such as the Timer class).

The issue was originally identified in KAFKA-17939, where it was noted that:

    • The org.apache.kafka.common.utils package is not part of the public API, according to the Javadoc
    • However, classes like org.apache.kafka.common.util.Bytes from this package are extensively used in public APIs like org.apache.kafka.streams.kstreamorg.apache.kafka.streams.state and org.apache.kafka.common.serialization
    • Reference links in Javadoc are broken when they point to these classes, as noted in the above public APIs documentation
  • Current problems:

This situation creates quite a few issues:

    1. User Confusion: Users cannot easily determine which classes are officially supported vs internal implementation details
    2. Documentation Issues: Javadoc references to these classes do not render as clickable links because the classes are not included in the generated documentation
    3. Maintenance Challenges: Since these classes have been "leaked" into the public APIs over the years, demonstrating the difficulty of maintaining this distinction
  • A Few Examples of Exposure:
      1. Bytes in org.apache.kafka.streams.kstream:
        public interface KTable<K, V> {
        	<VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
                                        final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
                                        final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
        }
      2. Bytes in org.apache.kafka.streams.state:  
        public interface KeyValueBytesStoreSupplier extends StoreSupplier<KeyValueStore<Bytes, byte[]>> {
        }
      3. Bytes in org.apache.kafka.common.serialization:  
        // BytesSerializer - users must provide Bytes objects
        public class BytesSerializer implements Serializer<Bytes> {
            public byte[] serialize(String topic, Bytes data) {
                if (data == null)
                    return null;
                return data.get();
            }
        }
        
        // BytesDeserializer - users receive Bytes objects  
        public class BytesDeserializer implements Deserializer<Bytes> {
            public Bytes deserialize(String topic, byte[] data) {
                if (data == null)
                    return null;
                return new Bytes(data);
            }
        }

Public Interfaces

This KIP proposes to make the core functionality of org.apache.kafka.common.utils.Bytes officially part of the public API.

Methods that will become officially public in the class Bytes are:

static Bytes wrap(byte[] bytes) - Factory method for creating Bytes instances

Bytes(byte[] bytes) - Constructor

byte[] get() - Returns the underlying byte array

int hashCode() - Standard Object method

boolean equals(Object other) - Standard Object method

int compareTo(Bytes that) - Implements Comparable interface

String toString() - Standard Object method

byte[] EMPTY - Empty byte array constant

The Bytes class will:

  • Be included in Javadoc generation
  • Receive the same stability guarantees as other public APIs
  • Follow standard deprecation policies for any future changes


Helper methods/fields that will be deprecated:

The following helpers are currently public but will be marked as @Deprecated and removed in version 5.0 to minimize the public API usage:

static Bytes increment(Bytes input) -> will be deprecated in the next release and removed in version 5.0

ByteArrayComparator BYTES_LEXICO_COMPARATOR -> will be deprecated in the next release and removed in version 5.0

interface ByteArrayComparator -> will be deprecated in the next release and removed in version 5.0

These helpers will be duplicated in org.apache.kafka.common.utils.internals.BytesUtils for internal Kafka use. Internal Kafka code will be migrated to use the internals package version. The deprecated public versions will remain functional until version 5.0 to maintain backward compatibility for any external users who may be relying on them.

Proposed Changes

1. Add Bytes core methods to Javadoc generation

Update the build configuration to include the core Bytes class in the public API documentation:

javadoc {
    include 'org/apache/kafka/common/utils/Bytes.java'
}

2. Deprecate helper methods:

Mark the following as @Deprecated in org.apache.kafka.common.utils.Bytes :

/**
 * @deprecated This method is not part of the public API and will be removed in version 5.0.
 *             Internal Kafka code should use org.apache.kafka.common.utils.internals.BytesUtils instead.
 */
@Deprecated
public static Bytes increment(Bytes input) throws IndexOutOfBoundsException {
    // existing implementation
}

/**
 * @deprecated This field is not part of the public API and will be removed in version 5.0.
 *             Internal Kafka code should use org.apache.kafka.common.utils.internals.BytesUtils instead.
 */
@Deprecated
public static final ByteArrayComparator BYTES_LEXICO_COMPARATOR = new LexicographicByteArrayComparator();

/**
 * @deprecated This interface is not part of the public API and will be removed in version 5.0.
 *             Internal Kafka code should use org.apache.kafka.common.utils.internals.BytesUtils instead.
 */
@Deprecated
public interface ByteArrayComparator extends Comparator<byte[]>, Serializable {
    // existing interface
}

3. Create an internals package with duplicates

Create org.apache.kafka.common.utils.internals.BytesUtils class containing:

  • increment() method
  • ByteArrayComparator
  • BYTES_LEXICO_COMPARATOR field

These will be used by internal Kafka code going forward.

3. Enhance documentation

Add or improve Javadoc for the core Bytes methods to meet public API standards.

4. Update Internal References

Update all internal Kafka code to use the helpers from org.apache.kafka.common.utils.internals.BytesUtils instead of the deprecated versions in the public Bytes class.

5. Remove deprecated methods in version 5.0

In Apache Kafka version 5.0, remove the deprecated helper methods from the public Bytes class entirely. By this time, all internal code will be using the internals package, and external users will have had ample warning.

Compatibility, Deprecation, and Migration Plan

No compatibility impact for external users:

  • The core Bytes class remains in org.apache.kafka.common.utils
  • All public methods that users interact with remain unchanged
  • No user code changes required
  • Deprecated helper methods will continue working until version 5.0
  • Users relying on deprecated helpers (unlikely but possible) will receive deprecation warnings and have multiple release cycles to migrate

For internal Kafka code:

  • Import statements for helper methods will need updating to reference org.apache.kafka.common.utils.internals.BytesUtils
  • This can be automated with IDE refactoring tools
  • No functional changes to the helpers themselves
  • All changes will be made before the release containing these deprecations

Rejected Alternatives

Refactor public APIs to remove Bytes:

Removing Bytes from the public APIs would require extensive breaking changes to the widely used APIs like Kafka Streams, with a significant impact on users.

Keep all helper methods in the public API:

We could make all current public methods (including helpers like increment()) officially part of the public API. However, this would create an unnecessary backward compatibility burden for utility methods that are primarily used internally. By deprecating them, we maintain the flexibility to modify or remove them in the future without long-term public API commitments.

Move helpers to the internals package without deprecation:

We could move the helpers to the internals package without deprecation. However, this risks breaking any external users who might be relying on these methods, even if unlikely. Following a proper deprecation cycle is the safer, more community-friendly approach that aligns with Kafka's compatibility commitments.



  • No labels