Skip to content
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

Merged
merged 5 commits into from Oct 2, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -19,7 +19,7 @@
import org.apache.kafka.streams.kstream.internals.suppress.EagerBufferConfigImpl;
import org.apache.kafka.streams.kstream.internals.suppress.FinalResultsSuppressionBuilder;
import org.apache.kafka.streams.kstream.internals.suppress.StrictBufferConfigImpl;
import org.apache.kafka.streams.kstream.internals.suppress.SuppressedImpl;
import org.apache.kafka.streams.kstream.internals.suppress.SuppressedInternal;

import java.time.Duration;

Expand Down Expand Up @@ -155,6 +155,6 @@ static <K extends Windowed> Suppressed<K> untilWindowCloses(final StrictBufferCo
* @return a suppression configuration
*/
static <K> Suppressed<K> untilTimeLimit(final Duration timeToWaitForMoreEvents, final BufferConfig bufferConfig) {
return new SuppressedImpl<>(timeToWaitForMoreEvents, bufferConfig, null, false);
return new SuppressedInternal<>(timeToWaitForMoreEvents, bufferConfig, null, false);
}
}
Expand Up @@ -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;
Expand All @@ -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);
}
Expand All @@ -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
Expand All @@ -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(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why this change? (Just for my own education.)

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 buffer.array() was incorrect, since the backing array isn't guaranteed to be exactly within the bounds we allocated. I fixed it at the time by delegating to the ByteBufferSerializer, which handles this.

Later on I realized that there is a more efficient solution available. By pre-creating the backing array and wrapping it, we know that buffer.array() returns what we needed. No need for the more general handling logic in ByteBufferSerializer.

new byte[4 + (oldSize == -1 ? 0 : oldSize) + 4 + (newSize == -1 ? 0 : newSize)]
);
buffer.putInt(oldSize);
if (oldBytes != null) {
Expand All @@ -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
Expand All @@ -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) {
Expand All @@ -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];
Expand Down
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.kstream.internals;

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.kstream.TimeWindowedDeserializer;
import org.apache.kafka.streams.kstream.TimeWindowedSerializer;
import org.apache.kafka.streams.kstream.Windowed;

class FullTimeWindowedSerde<T> extends Serdes.WrapperSerde<Windowed<T>> {
FullTimeWindowedSerde(final Serde<T> inner, final long windowSize) {
super(
new TimeWindowedSerializer<>(inner.serializer()),
new TimeWindowedDeserializer<>(inner.deserializer(), windowSize)
);
}
}
Expand Up @@ -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;

Expand Down Expand Up @@ -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)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we extend wrapOrCast to add a null check and return null for this case and use it here to make code more readable?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can and will.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've added that check because context.valueSerde() (called elsewhere) could return null.

If it's ok with you, though, I prefer the current code right here. This code ensures that valSerde is of the correct type (notice that no casting is necessary). In general, I think we should avoid casting unless we actually need it, as it makes regressions harder to catch.

);

final ProcessorParameters<K, Change<V>> processorParameters = new ProcessorParameters<>(
Expand All @@ -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.");
}
Expand Down
Expand Up @@ -27,7 +27,6 @@
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.Windows;
import org.apache.kafka.streams.kstream.internals.graph.StreamsGraphNode;
import org.apache.kafka.streams.state.StoreBuilder;
Expand Down Expand Up @@ -93,7 +92,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 FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null,
materializedInternal.valueSerde());
}

Expand All @@ -120,7 +119,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 FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null,
materializedInternal.valueSerde());
}

Expand Down Expand Up @@ -149,7 +148,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 FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.size()) : null,
materializedInternal.valueSerde());
}

Expand Down
Expand Up @@ -20,8 +20,8 @@

import static org.apache.kafka.streams.kstream.internals.suppress.BufferFullStrategy.SHUT_DOWN;

abstract class BufferConfigImpl<BC extends Suppressed.BufferConfig<BC>> implements Suppressed.BufferConfig<BC> {
public abstract long maxKeys();
abstract class BufferConfigInternal<BC extends Suppressed.BufferConfig<BC>> implements Suppressed.BufferConfig<BC> {
public abstract long maxRecords();

public abstract long maxBytes();

Expand All @@ -39,12 +39,12 @@ public Suppressed.StrictBufferConfig withNoBound() {

@Override
public Suppressed.StrictBufferConfig shutDownWhenFull() {
return new StrictBufferConfigImpl(maxKeys(), maxBytes(), SHUT_DOWN);
return new StrictBufferConfigImpl(maxRecords(), maxBytes(), SHUT_DOWN);
}

@Override
public Suppressed.BufferConfig emitEarlyWhenFull() {
return new EagerBufferConfigImpl(maxKeys(), maxBytes());
return new EagerBufferConfigImpl(maxRecords(), maxBytes());
}

@Override
Expand Down
Expand Up @@ -20,13 +20,13 @@

import java.util.Objects;

public class EagerBufferConfigImpl extends BufferConfigImpl {
public class EagerBufferConfigImpl extends BufferConfigInternal {

private final long maxKeys;
private final long maxRecords;
private final long maxBytes;

public EagerBufferConfigImpl(final long maxKeys, final long maxBytes) {
this.maxKeys = maxKeys;
public EagerBufferConfigImpl(final long maxRecords, final long maxBytes) {
this.maxRecords = maxRecords;
this.maxBytes = maxBytes;
}

Expand All @@ -37,12 +37,12 @@ public Suppressed.BufferConfig withMaxRecords(final long recordLimit) {

@Override
public Suppressed.BufferConfig withMaxBytes(final long byteLimit) {
return new EagerBufferConfigImpl(maxKeys, byteLimit);
return new EagerBufferConfigImpl(maxRecords, byteLimit);
}

@Override
public long maxKeys() {
return maxKeys;
public long maxRecords() {
return maxRecords;
}

@Override
Expand All @@ -60,17 +60,17 @@ public boolean equals(final Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
final EagerBufferConfigImpl that = (EagerBufferConfigImpl) o;
return maxKeys == that.maxKeys &&
return maxRecords == that.maxRecords &&
maxBytes == that.maxBytes;
}

@Override
public int hashCode() {
return Objects.hash(maxKeys, maxBytes);
return Objects.hash(maxRecords, maxBytes);
}

@Override
public String toString() {
return "EagerBufferConfigImpl{maxKeys=" + maxKeys + ", maxBytes=" + maxBytes + '}';
return "EagerBufferConfigImpl{maxKeys=" + maxRecords + ", maxBytes=" + maxBytes + '}';
}
}
Expand Up @@ -18,7 +18,6 @@

import org.apache.kafka.streams.kstream.Suppressed;
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.processor.ProcessorContext;

import java.time.Duration;
import java.util.Objects;
Expand All @@ -30,11 +29,11 @@ public FinalResultsSuppressionBuilder(final Suppressed.StrictBufferConfig buffer
this.bufferConfig = bufferConfig;
}

public SuppressedImpl<K> buildFinalResultsSuppression(final Duration gracePeriod) {
return new SuppressedImpl<>(
public SuppressedInternal<K> buildFinalResultsSuppression(final Duration gracePeriod) {
return new SuppressedInternal<>(
gracePeriod,
bufferConfig,
(ProcessorContext context, K key) -> key.window().end(),
TimeDefinitions.WindowEndTimeDefinition.instance(),
true
);
}
Expand Down