Skip to content

Commit

Permalink
KAFKA-16573: Specify node and store where serdes are needed (apache#1…
Browse files Browse the repository at this point in the history
…5790)

Reviewers: Matthias J. Sax <[email protected]>, Bruno Cadonna <[email protected]>, Anna Sophie Blee-Goldman <[email protected]>
  • Loading branch information
AyoubOm authored Jun 5, 2024
1 parent 896af1b commit 1134520
Show file tree
Hide file tree
Showing 14 changed files with 433 additions and 67 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1875,7 +1875,7 @@ public KafkaClientSupplier getKafkaClientSupplier() {
@SuppressWarnings("WeakerAccess")
public Serde<?> defaultKeySerde() {
final Object keySerdeConfigSetting = get(DEFAULT_KEY_SERDE_CLASS_CONFIG);
if (keySerdeConfigSetting == null) {
if (keySerdeConfigSetting == null) {
throw new ConfigException("Please specify a key serde or set one through StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG");
}
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,22 +33,19 @@ private static <T> Deserializer<T> prepareDeserializer(final Deserializer<T> spe
final Deserializer<T> deserializerToUse;

if (specificDeserializer == null) {
final Deserializer<?> contextKeyDeserializer = context.keySerde().deserializer();
final Deserializer<?> contextValueDeserializer = context.valueSerde().deserializer();
deserializerToUse = (Deserializer<T>) (isKey ? contextKeyDeserializer : contextValueDeserializer);
deserializerToUse = (Deserializer<T>) (isKey ? context.keySerde().deserializer() : context.valueSerde().deserializer());
} else {
deserializerToUse = specificDeserializer;
initNullableDeserializer(deserializerToUse, new SerdeGetter(context));
}
return deserializerToUse;
}

@SuppressWarnings("unchecked")
private static <T> Serializer<T> prepareSerializer(final Serializer<T> specificSerializer, final ProcessorContext context, final boolean isKey, final String name) {
final Serializer<T> serializerToUse;
if (specificSerializer == null) {
final Serializer<?> contextKeySerializer = context.keySerde().serializer();
final Serializer<?> contextValueSerializer = context.valueSerde().serializer();
serializerToUse = (Serializer<T>) (isKey ? contextKeySerializer : contextValueSerializer);
serializerToUse = (Serializer<T>) (isKey ? context.keySerde().serializer() : context.valueSerde().serializer());
} else {
serializerToUse = specificSerializer;
initNullableSerializer(serializerToUse, new SerdeGetter(context));
Expand All @@ -60,7 +57,7 @@ private static <T> Serializer<T> prepareSerializer(final Serializer<T> specificS
private static <T> Serde<T> prepareSerde(final Serde<T> specificSerde, final SerdeGetter getter, final boolean isKey) {
final Serde<T> serdeToUse;
if (specificSerde == null) {
serdeToUse = (Serde<T>) (isKey ? getter.keySerde() : getter.valueSerde());
serdeToUse = (Serde<T>) (isKey ? getter.keySerde() : getter.valueSerde());
} else {
serdeToUse = specificSerde;
}
Expand Down Expand Up @@ -93,12 +90,14 @@ public static <K> Serde<K> prepareKeySerde(final Serde<K> specificSerde, final S
public static <V> Serde<V> prepareValueSerde(final Serde<V> specificSerde, final SerdeGetter getter) {
return prepareSerde(specificSerde, getter, false);
}

@SuppressWarnings({"rawtypes", "unchecked"})
public static <T> void initNullableSerializer(final Serializer<T> specificSerializer, final SerdeGetter getter) {
if (specificSerializer instanceof WrappingNullableSerializer) {
((WrappingNullableSerializer) specificSerializer).setIfUnset(getter);
}
}

@SuppressWarnings({"rawtypes", "unchecked"})
public static <T> void initNullableDeserializer(final Deserializer<T> specificDeserializer, final SerdeGetter getter) {
if (specificDeserializer instanceof WrappingNullableDeserializer) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,9 @@
*/
package org.apache.kafka.streams.processor.internals;

import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.processor.TopicNameExtractor;
import org.apache.kafka.streams.processor.api.Record;
Expand Down Expand Up @@ -58,8 +60,17 @@ public void addChild(final ProcessorNode<Void, Void, ?, ?> child) {
public void init(final InternalProcessorContext<Void, Void> context) {
super.init(context);
this.context = context;
keySerializer = prepareKeySerializer(keySerializer, context, this.name());
valSerializer = prepareValueSerializer(valSerializer, context, this.name());
try {
keySerializer = prepareKeySerializer(keySerializer, context, this.name());
} catch (ConfigException | StreamsException e) {
throw new StreamsException(String.format("Failed to initialize key serdes for sink node %s", name()), e, context.taskId());
}

try {
valSerializer = prepareValueSerializer(valSerializer, context, this.name());
} catch (final ConfigException | StreamsException e) {
throw new StreamsException(String.format("Failed to initialize value serdes for sink node %s", name()), e, context.taskId());
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,11 @@
*/
package org.apache.kafka.streams.processor.internals;

import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.TimestampExtractor;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics;
Expand Down Expand Up @@ -74,8 +76,17 @@ public void init(final InternalProcessorContext<KIn, VIn> context) {
super.init(context);
this.context = context;

keyDeserializer = prepareKeyDeserializer(keyDeserializer, context, name());
valDeserializer = prepareValueDeserializer(valDeserializer, context, name());
try {
keyDeserializer = prepareKeyDeserializer(keyDeserializer, context, name());
} catch (final ConfigException | StreamsException e) {
throw new StreamsException(String.format("Failed to initialize key serdes for source node %s", name()), e, context.taskId());
}

try {
valDeserializer = prepareValueDeserializer(valDeserializer, context, name());
} catch (final ConfigException | StreamsException e) {
throw new StreamsException(String.format("Failed to initialize value serdes for source node %s", name()), e, context.taskId());
}
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,6 @@

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
import static org.apache.kafka.streams.state.internals.StoreQueryUtils.getDeserializeValue;

Expand Down Expand Up @@ -187,21 +186,15 @@ protected Serde<V> prepareValueSerdeForStore(final Serde<V> valueSerde, final Se
protected void initStoreSerde(final ProcessorContext context) {
final String storeName = name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
serdes = new StateSerdes<>(
changelogTopic,
prepareKeySerde(keySerde, new SerdeGetter(context)),
prepareValueSerdeForStore(valueSerde, new SerdeGetter(context))
);
serdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerdeForStore);
}

protected void initStoreSerde(final StateStoreContext context) {
final String storeName = name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
serdes = new StateSerdes<>(
changelogTopic,
prepareKeySerde(keySerde, new SerdeGetter(context)),
prepareValueSerdeForStore(valueSerde, new SerdeGetter(context))
);
serdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerdeForStore);
}

@SuppressWarnings("unchecked")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
import org.apache.kafka.streams.processor.internals.ProcessorContextUtils;
import org.apache.kafka.streams.processor.internals.SerdeGetter;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.query.FailureReason;
import org.apache.kafka.streams.query.PositionBound;
Expand Down Expand Up @@ -151,21 +150,15 @@ private void registerMetrics() {
private void initStoreSerde(final ProcessorContext context) {
final String storeName = name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
serdes = new StateSerdes<>(
changelogTopic,
WrappingNullableUtils.prepareKeySerde(keySerde, new SerdeGetter(context)),
WrappingNullableUtils.prepareValueSerde(valueSerde, new SerdeGetter(context))
);
serdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde);
}

private void initStoreSerde(final StateStoreContext context) {
final String storeName = name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
serdes = new StateSerdes<>(
changelogTopic,
WrappingNullableUtils.prepareKeySerde(keySerde, new SerdeGetter(context)),
WrappingNullableUtils.prepareValueSerde(valueSerde, new SerdeGetter(context))
);
serdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde);
}

@SuppressWarnings("unchecked")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ public class MeteredTimestampedKeyValueStore<K, V>
(query, positionBound, config, store) -> runTimestampedKeyQuery(query, positionBound, config)
)
);

@SuppressWarnings("unchecked")
@Override
protected Serde<ValueAndTimestamp<V>> prepareValueSerdeForStore(final Serde<ValueAndTimestamp<V>> valueSerde, final SerdeGetter getter) {
Expand All @@ -96,7 +97,6 @@ protected Serde<ValueAndTimestamp<V>> prepareValueSerdeForStore(final Serde<Valu
}
}


public RawAndDeserializedValue<V> getWithBinary(final K key) {
try {
return maybeMeasureLatency(() -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,18 +18,18 @@

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde;
import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareValueSerde;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;


import java.time.Instant;
import java.util.Map;
import java.util.Objects;

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
Expand Down Expand Up @@ -303,11 +303,8 @@ protected void initStoreSerde(final ProcessorContext context) {
// additionally init raw value serde
final String storeName = super.name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
plainValueSerdes = new StateSerdes<>(
changelogTopic,
prepareKeySerde(keySerde, new SerdeGetter(context)),
prepareValueSerde(plainValueSerde, new SerdeGetter(context))
);
plainValueSerdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, plainValueSerde, WrappingNullableUtils::prepareValueSerde);
}

@Override
Expand All @@ -317,11 +314,8 @@ protected void initStoreSerde(final StateStoreContext context) {
// additionally init raw value serde
final String storeName = super.name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
plainValueSerdes = new StateSerdes<>(
changelogTopic,
prepareKeySerde(keySerde, new SerdeGetter(context)),
prepareValueSerde(plainValueSerde, new SerdeGetter(context))
);
plainValueSerdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, plainValueSerde, WrappingNullableUtils::prepareValueSerde);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
import static org.apache.kafka.streams.state.internals.StoreQueryUtils.getDeserializeValue;

Expand Down Expand Up @@ -170,19 +169,15 @@ private void registerMetrics() {
private void initStoreSerde(final ProcessorContext context) {
final String storeName = name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
serdes = new StateSerdes<>(
changelogTopic,
prepareKeySerde(keySerde, new SerdeGetter(context)),
prepareValueSerde(valueSerde, new SerdeGetter(context)));
serdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerde);
}

private void initStoreSerde(final StateStoreContext context) {
final String storeName = name();
final String changelogTopic = ProcessorContextUtils.changelogFor(context, storeName, Boolean.FALSE);
serdes = new StateSerdes<>(
changelogTopic,
prepareKeySerde(keySerde, new SerdeGetter(context)),
prepareValueSerde(valueSerde, new SerdeGetter(context)));
serdes = StoreSerdeInitializer.prepareStoreSerde(
context, storeName, changelogTopic, keySerde, valueSerde, this::prepareValueSerde);
}

@SuppressWarnings("unchecked")
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* 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.state.internals;

import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.SerdeGetter;
import org.apache.kafka.streams.state.StateSerdes;


public class StoreSerdeInitializer {
static <K, V> StateSerdes<K, V> prepareStoreSerde(final StateStoreContext context,
final String storeName,
final String changelogTopic,
final Serde<K> keySerde,
final Serde<V> valueSerde,
final PrepareFunc<V> prepareValueSerdeFunc) {
return new StateSerdes<>(
changelogTopic,
prepareSerde(WrappingNullableUtils::prepareKeySerde, storeName, keySerde, new SerdeGetter(context), true, context.taskId()),
prepareSerde(prepareValueSerdeFunc, storeName, valueSerde, new SerdeGetter(context), false, context.taskId())
);
}

static <K, V> StateSerdes<K, V> prepareStoreSerde(final ProcessorContext context,
final String storeName,
final String changelogTopic,
final Serde<K> keySerde,
final Serde<V> valueSerde,
final PrepareFunc<V> prepareValueSerdeFunc) {
return new StateSerdes<>(
changelogTopic,
prepareSerde(WrappingNullableUtils::prepareKeySerde, storeName, keySerde, new SerdeGetter(context), true, context.taskId()),
prepareSerde(prepareValueSerdeFunc, storeName, valueSerde, new SerdeGetter(context), false, context.taskId())
);
}

private static <T> Serde<T> prepareSerde(final PrepareFunc<T> prepare,
final String storeName,
final Serde<T> serde,
final SerdeGetter getter,
final Boolean isKey,
final TaskId taskId) {

final String serdeType = isKey ? "key" : "value";
try {
return prepare.prepareSerde(serde, getter);
} catch (final ConfigException | StreamsException e) {
throw new StreamsException(String.format("Failed to initialize %s serdes for store %s", serdeType, storeName), e, taskId);
}
}
}

interface PrepareFunc<T> {
Serde<T> prepareSerde(Serde<T> serde, SerdeGetter getter);
}

Loading

0 comments on commit 1134520

Please sign in to comment.