-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-20127: Add new methods to StateSerdes #21411
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
90aa5a5
706b001
bbc7472
2327d0e
edeeb62
59bc4e0
2d680be
e56590e
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 |
|---|---|---|
|
|
@@ -16,6 +16,8 @@ | |
| */ | ||
| package org.apache.kafka.streams.state; | ||
|
|
||
| import org.apache.kafka.common.header.Headers; | ||
| import org.apache.kafka.common.header.internals.RecordHeaders; | ||
| import org.apache.kafka.common.serialization.Deserializer; | ||
| import org.apache.kafka.common.serialization.Serde; | ||
| import org.apache.kafka.common.serialization.Serdes; | ||
|
|
@@ -148,39 +150,75 @@ public String topic() { | |
| * | ||
| * @param rawKey the key as raw bytes | ||
| * @return the key as typed object | ||
| * @deprecated Since 4.3. Use {@link #keyFrom(byte[], Headers)} instead. | ||
| */ | ||
| @Deprecated | ||
|
Contributor
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. Does that make sense to replace this deprecated method with the newly introduced one?
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. I suppose that we should add |
||
| public K keyFrom(final byte[] rawKey) { | ||
| return keySerde.deserializer().deserialize(topic, rawKey); | ||
| return keyFrom(rawKey, new RecordHeaders()); | ||
| } | ||
|
|
||
| /** | ||
| * Deserialize the key from raw bytes. | ||
| * | ||
| * @param rawKey the key as raw bytes | ||
| * @return the key as typed object | ||
| */ | ||
| public K keyFrom(final byte[] rawKey, final Headers headers) { | ||
| return keySerde.deserializer().deserialize(topic, headers, rawKey); | ||
| } | ||
|
|
||
| /** | ||
| * Deserialize the value from raw bytes. | ||
| * | ||
| * @param rawValue the value as raw bytes | ||
| * @return the value as typed object | ||
| * @deprecated Since 4.3. Use {@link #valueFrom(byte[], Headers)} instead. | ||
| */ | ||
| @Deprecated | ||
| public V valueFrom(final byte[] rawValue) { | ||
| return valueSerde.deserializer().deserialize(topic, rawValue); | ||
| return valueFrom(rawValue, new RecordHeaders()); | ||
| } | ||
|
|
||
| /** | ||
| * Deserialize the value from raw bytes. | ||
| * | ||
| * @param rawValue the value as raw bytes | ||
| * @return the value as typed object | ||
| */ | ||
| public V valueFrom(final byte[] rawValue, final Headers headers) { | ||
| return valueSerde.deserializer().deserialize(topic, headers, rawValue); | ||
| } | ||
|
|
||
| /** | ||
| * Serialize the given key. | ||
| * | ||
| * @param key the key to be serialized | ||
| * @return the serialized key | ||
| * @deprecated Since 4.3. Use {@link #rawKey(Object, Headers)} instead. | ||
| */ | ||
| @Deprecated | ||
| public byte[] rawKey(final K key) { | ||
| return rawKey(key, new RecordHeaders()); | ||
| } | ||
|
|
||
| /** | ||
| * Serialize the given key. | ||
| * | ||
| * @param key the key to be serialized | ||
| * @return the serialized key | ||
| */ | ||
| public byte[] rawKey(final K key, final Headers headers) { | ||
| try { | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return keySerde.serializer().serialize(topic, key); | ||
| return keySerde.serializer().serialize(topic, headers, key); | ||
| } catch (final ClassCastException e) { | ||
| final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName(); | ||
| throw new StreamsException( | ||
| String.format("A serializer (%s) is not compatible to the actual key type " + | ||
| "(key type: %s). Change the default Serdes in StreamConfig or " + | ||
| "provide correct Serdes via method parameters.", | ||
| keySerializer().getClass().getName(), | ||
| keyClass), | ||
| e); | ||
| String.format("A serializer (%s) is not compatible to the actual key type " + | ||
| "(key type: %s). Change the default Serdes in StreamConfig or " + | ||
| "provide correct Serdes via method parameters.", | ||
| keySerializer().getClass().getName(), | ||
| keyClass), | ||
| e); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -189,11 +227,24 @@ public byte[] rawKey(final K key) { | |
| * | ||
| * @param value the value to be serialized | ||
| * @return the serialized value | ||
| * @deprecated Since 4.3. Use {@link #rawValue(Object, Headers)} instead. | ||
| */ | ||
| @Deprecated | ||
| @SuppressWarnings("rawtypes") | ||
| public byte[] rawValue(final V value) { | ||
| return rawValue(value, new RecordHeaders()); | ||
| } | ||
|
|
||
| /** | ||
| * Serialize the given value. | ||
| * | ||
| * @param value the value to be serialized | ||
| * @return the serialized value | ||
| */ | ||
| @SuppressWarnings("rawtypes") | ||
| public byte[] rawValue(final V value, final Headers headers) { | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| try { | ||
| return valueSerde.serializer().serialize(topic, value); | ||
| return valueSerde.serializer().serialize(topic, headers, value); | ||
| } catch (final ClassCastException e) { | ||
| final String valueClass; | ||
| final Class<? extends Serializer> serializerClass; | ||
|
|
@@ -205,12 +256,12 @@ public byte[] rawValue(final V value) { | |
| valueClass = value == null ? "unknown because value is null" : value.getClass().getName(); | ||
| } | ||
| throw new StreamsException( | ||
| String.format("A serializer (%s) is not compatible to the actual value type " + | ||
| "(value type: %s). Change the default Serdes in StreamConfig or " + | ||
| "provide correct Serdes via method parameters.", | ||
| serializerClass.getName(), | ||
| valueClass), | ||
| e); | ||
| String.format("A serializer (%s) is not compatible to the actual value type " + | ||
| "(value type: %s). Change the default Serdes in StreamConfig or " + | ||
| "provide correct Serdes via method parameters.", | ||
| serializerClass.getName(), | ||
| valueClass), | ||
| e); | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.