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
KAFKA-7223: In-Memory Suppression Buffering #5693
Changes from 3 commits
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,8 +16,6 @@ | |
*/ | ||
package org.apache.kafka.streams.kstream.internals; | ||
|
||
import org.apache.kafka.common.serialization.ByteBufferDeserializer; | ||
import org.apache.kafka.common.serialization.ByteBufferSerializer; | ||
import org.apache.kafka.common.serialization.Deserializer; | ||
import org.apache.kafka.common.serialization.Serde; | ||
import org.apache.kafka.common.serialization.Serializer; | ||
|
@@ -30,6 +28,17 @@ | |
public class FullChangeSerde<T> implements Serde<Change<T>> { | ||
private final Serde<T> inner; | ||
|
||
@SuppressWarnings("unchecked") | ||
public static <T> FullChangeSerde<T> castOrWrap(final Serde<?> serde) { | ||
if (serde == null) { | ||
return null; | ||
} else if (serde instanceof FullChangeSerde) { | ||
return (FullChangeSerde<T>) serde; | ||
} else { | ||
return new FullChangeSerde<T>((Serde<T>) serde); | ||
} | ||
} | ||
|
||
public FullChangeSerde(final Serde<T> inner) { | ||
this.inner = requireNonNull(inner); | ||
} | ||
|
@@ -47,7 +56,6 @@ public void close() { | |
@Override | ||
public Serializer<Change<T>> serializer() { | ||
final Serializer<T> innerSerializer = inner.serializer(); | ||
final ByteBufferSerializer byteBufferSerializer = new ByteBufferSerializer(); | ||
|
||
return new Serializer<Change<T>>() { | ||
@Override | ||
|
@@ -65,8 +73,8 @@ public byte[] serialize(final String topic, final Change<T> data) { | |
final byte[] newBytes = data.newValue == null ? null : innerSerializer.serialize(topic, data.newValue); | ||
final int newSize = newBytes == null ? -1 : newBytes.length; | ||
|
||
final ByteBuffer buffer = ByteBuffer.allocate( | ||
4 + (oldSize == -1 ? 0 : oldSize) + 4 + (newSize == -1 ? 0 : newSize) | ||
final ByteBuffer buffer = ByteBuffer.wrap( | ||
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. Why this change? (Just for my own education.) 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. It's just evidence of my mental slowness... In the prior PR, Guozhang pointed out that my calling Later on I realized that there is a more efficient solution available. By pre-creating the backing array and wrapping it, we know that |
||
new byte[4 + (oldSize == -1 ? 0 : oldSize) + 4 + (newSize == -1 ? 0 : newSize)] | ||
); | ||
buffer.putInt(oldSize); | ||
if (oldBytes != null) { | ||
|
@@ -76,7 +84,7 @@ public byte[] serialize(final String topic, final Change<T> data) { | |
if (newBytes != null) { | ||
buffer.put(newBytes); | ||
} | ||
return byteBufferSerializer.serialize(null, buffer); | ||
return buffer.array(); | ||
} | ||
|
||
@Override | ||
|
@@ -89,7 +97,6 @@ public void close() { | |
@Override | ||
public Deserializer<Change<T>> deserializer() { | ||
final Deserializer<T> innerDeserializer = inner.deserializer(); | ||
final ByteBufferDeserializer byteBufferDeserializer = new ByteBufferDeserializer(); | ||
return new Deserializer<Change<T>>() { | ||
@Override | ||
public void configure(final Map<String, ?> configs, final boolean isKey) { | ||
|
@@ -101,7 +108,7 @@ public Change<T> deserialize(final String topic, final byte[] data) { | |
if (data == null) { | ||
return null; | ||
} | ||
final ByteBuffer buffer = byteBufferDeserializer.deserialize(null, data); | ||
final ByteBuffer buffer = ByteBuffer.wrap(data); | ||
|
||
final int oldSize = buffer.getInt(); | ||
final byte[] oldBytes = oldSize == -1 ? null : new byte[oldSize]; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -39,7 +39,7 @@ | |
import org.apache.kafka.streams.kstream.internals.graph.TableProcessorNode; | ||
import org.apache.kafka.streams.kstream.internals.suppress.FinalResultsSuppressionBuilder; | ||
import org.apache.kafka.streams.kstream.internals.suppress.KTableSuppressProcessor; | ||
import org.apache.kafka.streams.kstream.internals.suppress.SuppressedImpl; | ||
import org.apache.kafka.streams.kstream.internals.suppress.SuppressedInternal; | ||
import org.apache.kafka.streams.processor.ProcessorSupplier; | ||
import org.apache.kafka.streams.state.KeyValueStore; | ||
|
||
|
@@ -356,12 +356,11 @@ public <K1> KStream<K1, V> toStream(final KeyValueMapper<? super K, ? super V, ? | |
public KTable<K, V> suppress(final Suppressed<K> suppressed) { | ||
final String name = builder.newProcessorName(SUPPRESS_NAME); | ||
|
||
// TODO: follow-up pr to forward the k/v serdes | ||
final ProcessorSupplier<K, Change<V>> suppressionSupplier = | ||
() -> new KTableSuppressProcessor<>( | ||
buildSuppress(suppressed), | ||
null, | ||
null | ||
keySerde, | ||
valSerde == null ? null : new FullChangeSerde<>(valSerde) | ||
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. Could we extend 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 can and will. 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've added that check because If it's ok with you, though, I prefer the current code right here. This code ensures that |
||
); | ||
|
||
final ProcessorParameters<K, Change<V>> processorParameters = new ProcessorParameters<>( | ||
|
@@ -387,18 +386,18 @@ public KTable<K, V> suppress(final Suppressed<K> suppressed) { | |
} | ||
|
||
@SuppressWarnings("unchecked") | ||
private SuppressedImpl<K> buildSuppress(final Suppressed<K> suppress) { | ||
private SuppressedInternal<K> buildSuppress(final Suppressed<K> suppress) { | ||
if (suppress instanceof FinalResultsSuppressionBuilder) { | ||
final long grace = findAndVerifyWindowGrace(streamsGraphNode); | ||
|
||
final FinalResultsSuppressionBuilder<?> builder = (FinalResultsSuppressionBuilder) suppress; | ||
|
||
final SuppressedImpl<? extends Windowed> finalResultsSuppression = | ||
final SuppressedInternal<? extends Windowed> finalResultsSuppression = | ||
builder.buildFinalResultsSuppression(Duration.ofMillis(grace)); | ||
|
||
return (SuppressedImpl<K>) finalResultsSuppression; | ||
} else if (suppress instanceof SuppressedImpl) { | ||
return (SuppressedImpl<K>) suppress; | ||
return (SuppressedInternal<K>) finalResultsSuppression; | ||
} else if (suppress instanceof SuppressedInternal) { | ||
return (SuppressedInternal<K>) suppress; | ||
} else { | ||
throw new IllegalArgumentException("Custom subclasses of Suppressed are not allowed."); | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,7 +27,7 @@ | |
import org.apache.kafka.streams.kstream.TimeWindowedKStream; | ||
import org.apache.kafka.streams.kstream.Window; | ||
import org.apache.kafka.streams.kstream.Windowed; | ||
import org.apache.kafka.streams.kstream.WindowedSerdes; | ||
import org.apache.kafka.streams.kstream.WindowedSerdes.TimeWindowedSerde; | ||
import org.apache.kafka.streams.kstream.Windows; | ||
import org.apache.kafka.streams.kstream.internals.graph.StreamsGraphNode; | ||
import org.apache.kafka.streams.state.StoreBuilder; | ||
|
@@ -93,7 +93,7 @@ private KTable<Windowed<K>, Long> doCount(final Materialized<K, Long, WindowStor | |
materialize(materializedInternal), | ||
new KStreamWindowAggregate<>(windows, materializedInternal.storeName(), aggregateBuilder.countInitializer, aggregateBuilder.countAggregator), | ||
materializedInternal.isQueryable(), | ||
materializedInternal.keySerde() != null ? new WindowedSerdes.TimeWindowedSerde<>(materializedInternal.keySerde()) : null, | ||
materializedInternal.keySerde() != null ? new TimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null, | ||
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 had to add these so that suppress doesn't "forget" the window end time when it round-trips the record. 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. Not sure if this is the best way to tack it? Requires public API change. |
||
materializedInternal.valueSerde()); | ||
} | ||
|
||
|
@@ -120,7 +120,7 @@ public <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer, | |
materialize(materializedInternal), | ||
new KStreamWindowAggregate<>(windows, materializedInternal.storeName(), initializer, aggregator), | ||
materializedInternal.isQueryable(), | ||
materializedInternal.keySerde() != null ? new WindowedSerdes.TimeWindowedSerde<>(materializedInternal.keySerde()) : null, | ||
materializedInternal.keySerde() != null ? new TimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null, | ||
materializedInternal.valueSerde()); | ||
} | ||
|
||
|
@@ -149,7 +149,7 @@ public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer, final Materialize | |
materialize(materializedInternal), | ||
new KStreamWindowReduce<>(windows, materializedInternal.storeName(), reducer), | ||
materializedInternal.isQueryable(), | ||
materializedInternal.keySerde() != null ? new WindowedSerdes.TimeWindowedSerde<>(materializedInternal.keySerde()) : null, | ||
materializedInternal.keySerde() != null ? new TimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null, | ||
materializedInternal.valueSerde()); | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this mentioned in the KIP? It's a public API change.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, no. When I did this before, I did it differently to keep it private. I thought this was a better way, but overlooked the public-ness of it.
I'll go back to private mode.