Skip to content
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,37 +18,34 @@

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.api.ProcessorContext;

import java.util.function.Supplier;

/**
* Allows serde access across different context types.
*/
public class SerdeGetter {

private final org.apache.kafka.streams.processor.ProcessorContext oldProcessorContext;
private final org.apache.kafka.streams.processor.api.ProcessorContext newProcessorContext;
private final StateStoreContext stateStorecontext;
public SerdeGetter(final org.apache.kafka.streams.processor.ProcessorContext context) {
oldProcessorContext = context;
newProcessorContext = null;
stateStorecontext = null;
}
public SerdeGetter(final org.apache.kafka.streams.processor.api.ProcessorContext context) {
oldProcessorContext = null;
newProcessorContext = context;
stateStorecontext = null;
private final Supplier<Serde<?>> keySerdeSupplier;
private final Supplier<Serde<?>> valueSerdeSupplier;

public SerdeGetter(final ProcessorContext<?, ?> context) {
keySerdeSupplier = context::keySerde;
valueSerdeSupplier = context::valueSerde;
}

public SerdeGetter(final StateStoreContext context) {
oldProcessorContext = null;
newProcessorContext = null;
stateStorecontext = context;
keySerdeSupplier = context::keySerde;
valueSerdeSupplier = context::valueSerde;
}
public Serde keySerde() {
return oldProcessorContext != null ? oldProcessorContext.keySerde() :
newProcessorContext != null ? newProcessorContext.keySerde() : stateStorecontext.keySerde();

public Serde<?> keySerde() {
return keySerdeSupplier.get();
}
public Serde valueSerde() {
return oldProcessorContext != null ? oldProcessorContext.valueSerde() :
newProcessorContext != null ? newProcessorContext.valueSerde() : stateStorecontext.valueSerde();

public Serde<?> valueSerde() {
return valueSerdeSupplier.get();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public class RocksDBTimeOrderedKeyValueBuffer<K, V> implements TimeOrderedKeyVal
private final boolean loggingEnabled;
private int partition;
private String changelogTopic;
private InternalProcessorContext context;
private InternalProcessorContext<?, ?> context;
private boolean minValid;

public static class Builder<K, V> implements StoreBuilder<TimeOrderedKeyValueBuffer<K, V, V>> {
Expand Down Expand Up @@ -156,7 +156,7 @@ public RocksDBTimeOrderedKeyValueBuffer(final RocksDBTimeOrderedKeyValueBytesSto
@Override
public void setSerdesIfNull(final SerdeGetter getter) {
keySerde = keySerde == null ? (Serde<K>) getter.keySerde() : keySerde;
valueSerde = valueSerde == null ? getter.valueSerde() : valueSerde;
valueSerde = valueSerde == null ? (Serde<V>) getter.valueSerde() : valueSerde;
}

private long observedStreamTime() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,11 @@
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,
Expand All @@ -41,19 +39,6 @@ static <K, V> StateSerdes<K, V> prepareStoreSerde(final StateStoreContext contex
);
}

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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.streams.processor.StateStoreContext;
Expand Down Expand Up @@ -59,10 +60,11 @@ public class RocksDBTimeOrderedKeyValueBufferTest {
public Sensor sensor;
public long offset;

@SuppressWarnings({"rawtypes", "unchecked"})
@BeforeEach
public void setUp() {
when(serdeGetter.keySerde()).thenReturn(new Serdes.StringSerde());
when(serdeGetter.valueSerde()).thenReturn(new Serdes.StringSerde());
when(serdeGetter.keySerde()).thenReturn((Serde) new Serdes.StringSerde());
when(serdeGetter.valueSerde()).thenReturn((Serde) new Serdes.StringSerde());
final Metrics metrics = new Metrics();
offset = 0;
streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", "processId", new MockTime());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@
import org.apache.kafka.common.serialization.Serdes;
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.state.StateSerdes;
import org.apache.kafka.test.MockInternalNewProcessorContext;

Expand Down Expand Up @@ -62,7 +60,7 @@ public void shouldPrepareStoreSerdeForProcessorContext() {
utilsMock.when(() -> WrappingNullableUtils.prepareValueSerde(any(), any())).thenReturn(valueSerde);

final StateSerdes<String, String> result = StoreSerdeInitializer.prepareStoreSerde(
(ProcessorContext) context, "myStore", "topic", keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde);
context, "myStore", "topic", keySerde, valueSerde, WrappingNullableUtils::prepareValueSerde);

assertThat(result.keySerde(), equalTo(keySerde));
assertThat(result.valueSerde(), equalTo(valueSerde));
Expand All @@ -77,7 +75,7 @@ public void shouldThrowStreamsExceptionOnUndefinedKeySerdeForProcessorContext()
.thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG"));

final Throwable exception = assertThrows(StreamsException.class,
() -> StoreSerdeInitializer.prepareStoreSerde((ProcessorContext) context, "myStore", "topic",
() -> StoreSerdeInitializer.prepareStoreSerde(context, "myStore", "topic",
new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde));

assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for store myStore"));
Expand All @@ -92,7 +90,7 @@ public void shouldThrowStreamsExceptionOnUndefinedValueSerdeForProcessorContext(
.thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG"));

final Throwable exception = assertThrows(StreamsException.class,
() -> StoreSerdeInitializer.prepareStoreSerde((ProcessorContext) context, "myStore", "topic",
() -> StoreSerdeInitializer.prepareStoreSerde(context, "myStore", "topic",
new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde));

assertThat(exception.getMessage(), equalTo("Failed to initialize value serdes for store myStore"));
Expand All @@ -107,7 +105,7 @@ public void shouldThrowStreamsExceptionOnUndefinedKeySerdeForStateStoreContext()
.thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG"));

final Throwable exception = assertThrows(StreamsException.class,
() -> StoreSerdeInitializer.prepareStoreSerde((StateStoreContext) context, "myStore", "topic",
() -> StoreSerdeInitializer.prepareStoreSerde(context, "myStore", "topic",
new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde));

assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for store myStore"));
Expand All @@ -122,7 +120,7 @@ public void shouldThrowStreamsExceptionOnUndefinedValueSerdeForStateStoreContext
.thenThrow(new ConfigException("Please set StreamsConfig#DEFAULT_VALUE_SERDE_CLASS_CONFIG"));

final Throwable exception = assertThrows(StreamsException.class,
() -> StoreSerdeInitializer.prepareStoreSerde((StateStoreContext) context, "myStore", "topic",
() -> StoreSerdeInitializer.prepareStoreSerde(context, "myStore", "topic",
new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde));

assertThat(exception.getMessage(), equalTo("Failed to initialize value serdes for store myStore"));
Expand All @@ -136,7 +134,7 @@ public void shouldThrowStreamsExceptionWithExplicitErrorMessageForProcessorConte
utilsMock.when(() -> WrappingNullableUtils.prepareKeySerde(any(), any())).thenThrow(new StreamsException(""));

final Throwable exception = assertThrows(StreamsException.class,
() -> StoreSerdeInitializer.prepareStoreSerde((ProcessorContext) context, "myStore", "topic",
() -> StoreSerdeInitializer.prepareStoreSerde(context, "myStore", "topic",
new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde));

assertThat(exception.getMessage(), equalTo("Failed to initialize key serdes for store myStore"));
Expand All @@ -149,7 +147,7 @@ public void shouldThrowStreamsExceptionWithExplicitErrorMessageForStateStoreCont
utilsMock.when(() -> WrappingNullableUtils.prepareValueSerde(any(), any())).thenThrow(new StreamsException(""));

final Throwable exception = assertThrows(StreamsException.class,
() -> StoreSerdeInitializer.prepareStoreSerde((StateStoreContext) context, "myStore", "topic",
() -> StoreSerdeInitializer.prepareStoreSerde(context, "myStore", "topic",
new Serdes.StringSerde(), new Serdes.StringSerde(), WrappingNullableUtils::prepareValueSerde));

assertThat(exception.getMessage(), equalTo("Failed to initialize value serdes for store myStore"));
Expand Down
Loading