Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Changed the status of KIP

Table of Contents

Status

Current state:  Under DiscussionAccepted

Discussion thread: here

Voting Discussion thread: here (full history)

JIRA:

Jira
serverASF JIRA
columnIdsissuekey,summary,issuetype,created,updated,duedate,assignee,reporter,customfield_12311032,customfield_12311037,customfield_12311022,customfield_12311027,priority,status,resolution
columnskey,summary,type,created,updated,due,assignee,reporter,Priority,Priority,Priority,Priority,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-14722

...

Code Block
titlemethod: serdeFrom, exception message
throw new IllegalArgumentException("Unknown class for built-in serializer. Supported types are: " +
    "String, Short, Integer, Long, Float, Double, ByteArray, ByteBuffer, Bytes, UUID, Boolean");


Code Block
titleBooleanSerializer class
public class BooleanSerializer implements Serializer<Boolean> {
	[...]
    public byte[] serialize(final String topic, final Boolean data) { [...]	}
}


Code Block
titleBooleanDeserializer class
public class BooleanDeserializer implements Deserializer<Boolean> {
	[...]
    public Boolean deserialize(final String topic, final byte[] data) { [...] }
}

Proposed Changes

  1. Add Boolean Serde to a org/apache/kafka/common/serialization package
    1. Add static public final class to the Serdes.java, update serdeFrom method, add Boolean() method that returns BooleanSerde();
    2. Create BooleanSerializer class in BooleanSerializer.java file, based on what is in NullableValueAndTimestampSerde.java
    3. Create BooleanDeserializer class in BooleanDeserializer.java file, based on what is in NullableValueAndTimestampSerde.java
  2. Rewrite NullableValueAndTimestampSerde.java file, package: org.apache.kafka.streams.state.internals so that it uses BooleanSerde from common/serialization instead of having own private class.

...