KAFKA-7663: Reprocessing on user added global stores restore (#15414)

When custom processors are added via StreamBuilder#addGlobalStore they will now reprocess all records through the custom transformer instead of loading directly.

We do this so that users that transform the records will not get improperly formatted records down stream.

Reviewers: Matthias J. Sax <matthias@confluent.io>
This commit is contained in:
Walker Carlson 2024-03-28 10:30:18 -05:00 committed by GitHub
parent 355873aa54
commit 8b274d8c1b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
18 changed files with 569 additions and 93 deletions

View File

@ -534,9 +534,7 @@ public class StreamsBuilder {
* of the input topic.
* <p>
* The provided {@link org.apache.kafka.streams.processor.ProcessorSupplier} will be used to create an {@link ProcessorNode} that will receive all
* records forwarded from the {@link SourceNode}. NOTE: you should not use the {@code Processor} to insert transformed records into
* the global state store. This store uses the source topic as changelog and during restore will insert records directly
* from the source.
* records forwarded from the {@link SourceNode}.
* This {@link ProcessorNode} should be used to keep the {@link StateStore} up-to-date.
* The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used.
* <p>
@ -567,7 +565,8 @@ public class StreamsBuilder {
new StoreBuilderWrapper(storeBuilder),
topic,
new ConsumedInternal<>(consumed),
() -> ProcessorAdapter.adapt(stateUpdateSupplier.get())
() -> ProcessorAdapter.adapt(stateUpdateSupplier.get()),
true
);
return this;
}
@ -585,9 +584,6 @@ public class StreamsBuilder {
* The supplier should always generate a new instance. Creating a single {@link Processor} object
* and returning the same object reference in {@link ProcessorSupplier#get()} is a
* violation of the supplier pattern and leads to runtime exceptions.
* NOTE: you should not use the {@link Processor} to insert transformed records into
* the global state store. This store uses the source topic as changelog and during restore will insert records directly
* from the source.
* This {@link Processor} should be used to keep the {@link StateStore} up-to-date.
* The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used.
* <p>
@ -611,7 +607,8 @@ public class StreamsBuilder {
new StoreBuilderWrapper(storeBuilder),
topic,
new ConsumedInternal<>(consumed),
stateUpdateSupplier
stateUpdateSupplier,
true
);
return this;
}

View File

@ -865,7 +865,8 @@ public class Topology {
valueDeserializer,
topic,
processorName,
() -> ProcessorAdapter.adapt(stateUpdateSupplier.get())
() -> ProcessorAdapter.adapt(stateUpdateSupplier.get()),
true
);
return this;
}
@ -917,7 +918,8 @@ public class Topology {
valueDeserializer,
topic,
processorName,
() -> ProcessorAdapter.adapt(stateUpdateSupplier.get())
() -> ProcessorAdapter.adapt(stateUpdateSupplier.get()),
true
);
return this;
}
@ -960,7 +962,8 @@ public class Topology {
valueDeserializer,
topic,
processorName,
stateUpdateSupplier
stateUpdateSupplier,
true
);
return this;
}
@ -1005,7 +1008,8 @@ public class Topology {
valueDeserializer,
topic,
processorName,
stateUpdateSupplier
stateUpdateSupplier,
true
);
return this;
}

View File

@ -41,6 +41,7 @@ import org.apache.kafka.streams.kstream.internals.graph.TableSuppressNode;
import org.apache.kafka.streams.kstream.internals.graph.TableSourceNode;
import org.apache.kafka.streams.kstream.internals.graph.VersionedSemanticsGraphNode;
import org.apache.kafka.streams.kstream.internals.graph.WindowedStreamProcessorNode;
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
import org.apache.kafka.streams.processor.internals.StoreFactory;
import org.apache.kafka.streams.state.KeyValueStore;
@ -221,7 +222,8 @@ public class InternalStreamsBuilder implements InternalNameProvider {
public synchronized <KIn, VIn> void addGlobalStore(final StoreFactory storeFactory,
final String topic,
final ConsumedInternal<KIn, VIn> consumed,
final org.apache.kafka.streams.processor.api.ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier) {
final ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier,
final boolean reprocessOnRestore) {
// explicitly disable logging for global stores
storeFactory.withLoggingDisabled();
@ -235,7 +237,8 @@ public class InternalStreamsBuilder implements InternalNameProvider {
topic,
consumed,
processorName,
stateUpdateSupplier
stateUpdateSupplier,
reprocessOnRestore
);
addGraphNode(root, globalStoreNode);

View File

@ -29,6 +29,7 @@ public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreN
private final ConsumedInternal<KIn, VIn> consumed;
private final String processorName;
private final ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier;
private final boolean reprocessOnRestore;
public GlobalStoreNode(final StoreFactory storeBuilder,
@ -36,7 +37,8 @@ public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreN
final String topic,
final ConsumedInternal<KIn, VIn> consumed,
final String processorName,
final ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier) {
final ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier,
final boolean reprocessOnRestore) {
super(storeBuilder);
this.sourceName = sourceName;
@ -44,6 +46,7 @@ public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreN
this.consumed = consumed;
this.processorName = processorName;
this.stateUpdateSupplier = stateUpdateSupplier;
this.reprocessOnRestore = reprocessOnRestore;
}
@Override
@ -56,7 +59,8 @@ public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreN
consumed.valueDeserializer(),
topic,
processorName,
stateUpdateSupplier);
stateUpdateSupplier,
reprocessOnRestore);
}
@ -66,6 +70,7 @@ public class GlobalStoreNode<KIn, VIn, S extends StateStore> extends StateStoreN
"sourceName='" + sourceName + '\'' +
", topic='" + topic + '\'' +
", processorName='" + processorName + '\'' +
", reprocessOnRestore='" + reprocessOnRestore + '\'' +
"} ";
}
}

View File

@ -105,7 +105,8 @@ public class TableSourceNode<K, V> extends SourceGraphNode<K, V> {
consumedInternal().valueDeserializer(),
topicName,
processorParameters.processorName(),
(ProcessorSupplier<K, V, Void, Void>) processorParameters.processorSupplier()
(ProcessorSupplier<K, V, Void, Void>) processorParameters.processorSupplier(),
false
);
} else {
topologyBuilder.addSource(consumedInternal().offsetResetPolicy(),

View File

@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.FixedOrderMap;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.CommitCallback;
@ -35,6 +36,8 @@ import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateRestoreListener;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.processor.api.Processor;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.processor.internals.Task.TaskType;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.streams.state.internals.RecordConverter;
@ -53,8 +56,10 @@ import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import static org.apache.kafka.streams.processor.internals.RecordDeserializer.handleDeserializationFailure;
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.converterForStore;
import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
/**
* This class is responsible for the initialization, restoration, closing, flushing etc
@ -77,8 +82,8 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
private final Set<String> globalStoreNames = new HashSet<>();
private final Set<String> globalNonPersistentStoresTopics = new HashSet<>();
private final FixedOrderMap<String, Optional<StateStore>> globalStores = new FixedOrderMap<>();
private InternalProcessorContext globalProcessorContext;
private DeserializationExceptionHandler deserializationExceptionHandler;
public GlobalStateManagerImpl(final LogContext logContext,
final Time time,
@ -116,6 +121,7 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
config.getLong(StreamsConfig.POLL_MS_CONFIG) + requestTimeoutMs
);
taskTimeoutMs = config.getLong(StreamsConfig.TASK_TIMEOUT_MS_CONFIG);
deserializationExceptionHandler = config.defaultDeserializationExceptionHandler();
}
@Override
@ -203,6 +209,15 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
);
try {
final Optional<InternalTopologyBuilder.ReprocessFactory<?, ?, ?, ?>> reprocessFactory = topology
.storeNameToReprocessOnRestore().getOrDefault(store.name(), Optional.empty());
if (reprocessFactory.isPresent()) {
reprocessState(
topicPartitions,
highWatermarks,
reprocessFactory.get(),
store.name());
} else {
restoreState(
stateRestoreCallback,
topicPartitions,
@ -210,6 +225,7 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
store.name(),
converterForStore(store)
);
}
} finally {
globalConsumer.unsubscribe();
}
@ -237,6 +253,97 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
return topicPartitions;
}
//Visible for testing
public void setDeserializationExceptionHandler(final DeserializationExceptionHandler deserializationExceptionHandler) {
this.deserializationExceptionHandler = deserializationExceptionHandler;
}
@SuppressWarnings("unchecked")
private void reprocessState(final List<TopicPartition> topicPartitions,
final Map<TopicPartition, Long> highWatermarks,
final InternalTopologyBuilder.ReprocessFactory reprocessFactory,
final String storeName) {
final Processor source = reprocessFactory.processorSupplier().get();
source.init(globalProcessorContext);
for (final TopicPartition topicPartition : topicPartitions) {
long currentDeadline = NO_DEADLINE;
globalConsumer.assign(Collections.singletonList(topicPartition));
long offset;
final Long checkpoint = checkpointFileCache.get(topicPartition);
if (checkpoint != null) {
globalConsumer.seek(topicPartition, checkpoint);
offset = checkpoint;
} else {
globalConsumer.seekToBeginning(Collections.singletonList(topicPartition));
offset = getGlobalConsumerOffset(topicPartition);
}
final Long highWatermark = highWatermarks.get(topicPartition);
stateRestoreListener.onRestoreStart(topicPartition, storeName, offset, highWatermark);
long restoreCount = 0L;
while (offset < highWatermark) {
// we add `request.timeout.ms` to `poll.ms` because `poll.ms` might be too short
// to give a fetch request a fair chance to actually complete and we don't want to
// start `task.timeout.ms` too early
//
// TODO with https://issues.apache.org/jira/browse/KAFKA-10315 we can just call
// `poll(pollMS)` without adding the request timeout and do a more precise
// timeout handling
final ConsumerRecords<byte[], byte[]> records = globalConsumer.poll(pollMsPlusRequestTimeout);
if (records.isEmpty()) {
currentDeadline = maybeUpdateDeadlineOrThrow(currentDeadline);
} else {
currentDeadline = NO_DEADLINE;
}
for (final ConsumerRecord<byte[], byte[]> record : records.records(topicPartition)) {
final ProcessorRecordContext recordContext =
new ProcessorRecordContext(
record.timestamp(),
record.offset(),
record.partition(),
record.topic(),
record.headers());
globalProcessorContext.setRecordContext(recordContext);
try {
if (record.key() != null) {
source.process(new Record<>(
reprocessFactory.keyDeserializer().deserialize(record.topic(), record.key()),
reprocessFactory.valueDeserializer().deserialize(record.topic(), record.value()),
record.timestamp(),
record.headers()));
restoreCount++;
}
} catch (final Exception deserializationException) {
handleDeserializationFailure(
deserializationExceptionHandler,
globalProcessorContext,
deserializationException,
record,
log,
droppedRecordsSensor(
Thread.currentThread().getName(),
globalProcessorContext.taskId().toString(),
globalProcessorContext.metrics()
)
);
}
}
offset = getGlobalConsumerOffset(topicPartition);
stateRestoreListener.onBatchRestored(topicPartition, storeName, offset, restoreCount);
}
stateRestoreListener.onRestoreEnd(topicPartition, storeName, restoreCount);
checkpointFileCache.put(topicPartition, offset);
}
}
private void restoreState(final StateRestoreCallback stateRestoreCallback,
final List<TopicPartition> topicPartitions,
final Map<TopicPartition, Long> highWatermarks,

View File

@ -123,6 +123,9 @@ public class InternalTopologyBuilder {
// map from changelog topic name to its corresponding state store.
private final Map<String, String> changelogTopicToStore = new HashMap<>();
// map of store name to restore behavior
private final Map<String, Optional<ReprocessFactory<?, ?, ?, ?>>> storeNameToReprocessOnRestore = new HashMap<>();
// all global topics
private final Set<String> globalTopics = new HashSet<>();
@ -154,6 +157,32 @@ public class InternalTopologyBuilder {
private boolean hasPersistentStores = false;
public static class ReprocessFactory<KIn, VIn, KOut, VOut> {
private final ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier;
private final Deserializer<KIn> keyDeserializer;
private final Deserializer<VIn> valueDeserializer;
private ReprocessFactory(final ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier,
final Deserializer<KIn> key,
final Deserializer<VIn> value) {
this.processorSupplier = processorSupplier;
this.keyDeserializer = key;
this.valueDeserializer = value;
}
public ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier() {
return processorSupplier;
}
public Deserializer<KIn> keyDeserializer() {
return keyDeserializer;
}
public Deserializer<VIn> valueDeserializer() {
return valueDeserializer;
}
}
private static abstract class NodeFactory<KIn, VIn, KOut, VOut> {
final String name;
final String[] predecessors;
@ -584,7 +613,8 @@ public class InternalTopologyBuilder {
final Deserializer<VIn> valueDeserializer,
final String topic,
final String processorName,
final ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier) {
final ProcessorSupplier<KIn, VIn, Void, Void> stateUpdateSupplier,
final boolean reprocessOnRestore) {
Objects.requireNonNull(storeFactory, "store builder must not be null");
ApiUtils.checkSupplier(stateUpdateSupplier);
validateGlobalStoreArguments(sourceName,
@ -613,6 +643,10 @@ public class InternalTopologyBuilder {
keyDeserializer,
valueDeserializer)
);
storeNameToReprocessOnRestore.put(storeFactory.name(),
reprocessOnRestore ?
Optional.of(new ReprocessFactory<>(stateUpdateSupplier, keyDeserializer, valueDeserializer))
: Optional.empty());
nodeToSourceTopics.put(sourceName, Arrays.asList(topics));
nodeGrouper.add(sourceName);
nodeFactory.addStateStore(storeFactory.name());
@ -996,7 +1030,8 @@ public class InternalTopologyBuilder {
new ArrayList<>(stateStoreMap.values()),
new ArrayList<>(globalStateStores.values()),
storeToChangelogTopic,
repartitionTopics);
repartitionTopics,
storeNameToReprocessOnRestore);
}
private void buildSinkNode(final Map<String, ProcessorNode<?, ?, ?, ?>> processorMap,

View File

@ -26,6 +26,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
public class ProcessorTopology {
@ -42,6 +43,7 @@ public class ProcessorTopology {
// the following contains entries for the entire topology, eg stores that do not belong to this ProcessorTopology
private final List<StateStore> globalStateStores;
private final Map<String, String> storeToChangelogTopic;
private final Map<String, Optional<InternalTopologyBuilder.ReprocessFactory<?, ?, ?, ?>>> storeNameToReprocessOnRestore;
public ProcessorTopology(final List<ProcessorNode<?, ?, ?, ?>> processorNodes,
final Map<String, SourceNode<?, ?>> sourceNodesByTopic,
@ -49,7 +51,8 @@ public class ProcessorTopology {
final List<StateStore> stateStores,
final List<StateStore> globalStateStores,
final Map<String, String> storeToChangelogTopic,
final Set<String> repartitionTopics) {
final Set<String> repartitionTopics,
final Map<String, Optional<InternalTopologyBuilder.ReprocessFactory<?, ?, ?, ?>>> storeNameToReprocessOnRestore) {
this.processorNodes = Collections.unmodifiableList(processorNodes);
this.sourceNodesByTopic = new HashMap<>(sourceNodesByTopic);
this.sinksByTopic = Collections.unmodifiableMap(sinksByTopic);
@ -57,6 +60,7 @@ public class ProcessorTopology {
this.globalStateStores = Collections.unmodifiableList(globalStateStores);
this.storeToChangelogTopic = Collections.unmodifiableMap(storeToChangelogTopic);
this.repartitionTopics = Collections.unmodifiableSet(repartitionTopics);
this.storeNameToReprocessOnRestore = storeNameToReprocessOnRestore;
this.terminalNodes = new HashSet<>();
for (final ProcessorNode<?, ?, ?, ?> node : processorNodes) {
@ -103,6 +107,10 @@ public class ProcessorTopology {
return stateStores;
}
public Map<String, Optional<InternalTopologyBuilder.ReprocessFactory<?, ?, ?, ?>>> storeNameToReprocessOnRestore() {
return storeNameToReprocessOnRestore;
}
public List<StateStore> globalStateStores() {
return Collections.unmodifiableList(globalStateStores);
}

View File

@ -29,7 +29,7 @@ import java.util.Optional;
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG;
class RecordDeserializer {
public class RecordDeserializer {
private final Logger log;
private final SourceNode<?, ?> sourceNode;
private final Sensor droppedRecordsSensor;
@ -68,6 +68,17 @@ class RecordDeserializer {
Optional.empty()
);
} catch (final Exception deserializationException) {
handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor);
return null; // 'handleDeserializationFailure' would either throw or swallow -- if we swallow we need to skip the record by returning 'null'
}
}
public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler,
final ProcessorContext<?, ?> processorContext,
final Exception deserializationException,
final ConsumerRecord<byte[], byte[]> rawRecord,
final Logger log,
final Sensor droppedRecordsSensor) {
final DeserializationExceptionHandler.DeserializationHandlerResponse response;
try {
response = deserializationExceptionHandler.handle(
@ -81,7 +92,6 @@ class RecordDeserializer {
deserializationException);
throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException);
}
if (response == DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL) {
throw new StreamsException("Deserialization exception handler is set to fail upon" +
" a deserialization error. If you would rather have the streaming pipeline" +
@ -97,11 +107,10 @@ class RecordDeserializer {
deserializationException
);
droppedRecordsSensor.record();
return null;
}
}
}
SourceNode<?, ?> sourceNode() {
return sourceNode;
}

View File

@ -76,6 +76,8 @@ public class InMemoryKeyValueStore implements KeyValueStore<Bytes, byte[]> {
false
);
// register the store
open = true;
context.register(
root,
(RecordBatchingStateRestoreCallback) records -> {

View File

@ -0,0 +1,201 @@
/*
* 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.integration;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.LongSerializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StoreQueryParameters;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.processor.api.ContextualProcessor;
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsString;
@Timeout(600)
@Tag("integration")
public class GlobalStateReprocessTest {
private static final int NUM_BROKERS = 1;
private static final Properties BROKER_CONFIG;
static {
BROKER_CONFIG = new Properties();
BROKER_CONFIG.put("transaction.state.log.replication.factor", (short) 1);
BROKER_CONFIG.put("transaction.state.log.min.isr", 1);
}
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG);
@BeforeAll
public static void startCluster() throws IOException {
CLUSTER.start();
}
@AfterAll
public static void closeCluster() {
CLUSTER.stop();
}
private final MockTime mockTime = CLUSTER.time;
private final String globalStore = "globalStore";
private StreamsBuilder builder;
private Properties streamsConfiguration;
private KafkaStreams kafkaStreams;
private String globalStoreTopic;
@BeforeEach
public void before(final TestInfo testInfo) throws Exception {
builder = new StreamsBuilder();
createTopics();
streamsConfiguration = new Properties();
final String safeTestName = safeUniqueTestName(testInfo);
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
final KeyValueStoreBuilder<String, Long> storeBuilder = new KeyValueStoreBuilder<>(
Stores.persistentKeyValueStore(globalStore),
Serdes.String(),
Serdes.Long(),
mockTime);
final ProcessorSupplier<String, Long, Void, Void> processorSupplier;
processorSupplier = () -> new ContextualProcessor<String, Long, Void, Void>() {
@Override
public void process(final Record<String, Long> record) {
final KeyValueStore<String, Long> stateStore =
context().getStateStore(storeBuilder.name());
stateStore.put(
record.key() + "- this is the right value.",
record.value()
);
}
};
builder.addGlobalStore(
storeBuilder,
globalStoreTopic,
Consumed.with(Serdes.String(), Serdes.Long()),
processorSupplier
);
}
@AfterEach
public void after() throws Exception {
if (kafkaStreams != null) {
kafkaStreams.close();
}
IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
}
@Test
public void shouldReprocessWithUserProvidedStore() throws Exception {
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
populateTopics(globalStoreTopic);
kafkaStreams.start();
TestUtils.waitForCondition(
() -> !storeContents(kafkaStreams).isEmpty(),
30000,
"Has not processed record within 30 seconds");
assertThat(storeContents(kafkaStreams).get(0), containsString("- this is the right value."));
kafkaStreams.close();
kafkaStreams.cleanUp();
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
kafkaStreams.start();
TestUtils.waitForCondition(
() -> !storeContents(kafkaStreams).isEmpty(),
30000,
"Has not processed record within 30 seconds");
assertThat(storeContents(kafkaStreams).get(0), containsString("- this is the right value."));
}
private void createTopics() throws Exception {
globalStoreTopic = "global-store-topic";
CLUSTER.createTopic(globalStoreTopic);
}
private void populateTopics(final String topicName) throws Exception {
IntegrationTestUtils.produceKeyValuesSynchronously(
topicName,
Collections.singletonList(new KeyValue<>("A", 1L)),
TestUtils.producerConfig(
CLUSTER.bootstrapServers(),
StringSerializer.class,
LongSerializer.class,
new Properties()),
mockTime);
}
private List<String> storeContents(final KafkaStreams streams) {
final ArrayList<String> keySet = new ArrayList<>();
final ReadOnlyKeyValueStore<String, Long> keyValueStore =
streams.store(StoreQueryParameters.fromNameAndType(globalStore, QueryableStoreTypes.keyValueStore()));
final KeyValueIterator<String, Long> range = keyValueStore.reverseAll();
while (range.hasNext()) {
keySet.add(range.next().key);
}
range.close();
return keySet;
}
}

View File

@ -30,13 +30,14 @@ import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.processor.api.ContextualProcessor;
import org.apache.kafka.streams.processor.api.Processor;
import org.apache.kafka.streams.processor.api.ProcessorContext;
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.test.MockApiProcessorSupplier;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
@ -126,11 +127,24 @@ public class GlobalThreadShutDownOrderTest {
Serdes.Long(),
mockTime);
final ProcessorSupplier<String, Long, Void, Void> processorSupplier;
processorSupplier = () -> new ContextualProcessor<String, Long, Void, Void>() {
@Override
public void process(final Record<String, Long> record) {
final KeyValueStore<String, Long> stateStore =
context().getStateStore(storeBuilder.name());
stateStore.put(
record.key(),
record.value()
);
}
};
builder.addGlobalStore(
storeBuilder,
globalStoreTopic,
Consumed.with(Serdes.String(), Serdes.Long()),
new MockApiProcessorSupplier<>()
processorSupplier
);
builder

View File

@ -23,15 +23,20 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.api.Processor;
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
import org.apache.kafka.streams.state.TimestampedBytesStore;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.streams.state.internals.WrappedStateStore;
@ -55,6 +60,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@ -75,6 +81,9 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class GlobalStateManagerImplTest {
@ -86,30 +95,37 @@ public class GlobalStateManagerImplTest {
private final String storeName2 = "t2-store";
private final String storeName3 = "t3-store";
private final String storeName4 = "t4-store";
private final String storeName5 = "t5-store";
private final TopicPartition t1 = new TopicPartition("t1", 1);
private final TopicPartition t2 = new TopicPartition("t2", 1);
private final TopicPartition t3 = new TopicPartition("t3", 1);
private final TopicPartition t4 = new TopicPartition("t4", 1);
private final TopicPartition t5 = new TopicPartition("t5", 1);
private GlobalStateManagerImpl stateManager;
private StateDirectory stateDirectory;
private StreamsConfig streamsConfig;
private NoOpReadOnlyStore<Object, Object> store1, store2, store3, store4;
private NoOpReadOnlyStore<Object, Object> store1, store2, store3, store4, store5;
private MockConsumer<byte[], byte[]> consumer;
private File checkpointFile;
private ProcessorTopology topology;
private InternalMockProcessorContext processorContext;
private Optional<InternalTopologyBuilder.ReprocessFactory<?, ?, ?, ?>> optionalMockReprocessFactory;
private DeserializationExceptionHandler deserializationExceptionHandler;
static ProcessorTopology withGlobalStores(final List<StateStore> stateStores,
final Map<String, String> storeToChangelogTopic) {
final Map<String, String> storeToChangelogTopic,
final Map<String, Optional<InternalTopologyBuilder.ReprocessFactory<?, ?, ?, ?>>> reprocessFactoryMap) {
return new ProcessorTopology(Collections.emptyList(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyList(),
stateStores,
storeToChangelogTopic,
Collections.emptySet());
Collections.emptySet(),
reprocessFactoryMap);
}
@SuppressWarnings("unchecked")
@Before
public void before() {
final Map<String, String> storeToTopic = new HashMap<>();
@ -118,14 +134,25 @@ public class GlobalStateManagerImplTest {
storeToTopic.put(storeName2, t2.topic());
storeToTopic.put(storeName3, t3.topic());
storeToTopic.put(storeName4, t4.topic());
storeToTopic.put(storeName5, t5.topic());
store1 = new NoOpReadOnlyStore<>(storeName1, true);
store2 = new ConverterStore<>(storeName2, true);
store3 = new NoOpReadOnlyStore<>(storeName3);
store4 = new NoOpReadOnlyStore<>(storeName4);
store5 = new NoOpReadOnlyStore<>(storeName5);
topology = withGlobalStores(asList(store1, store2, store3, store4), storeToTopic);
optionalMockReprocessFactory = mock(Optional.class);
when(optionalMockReprocessFactory.isPresent()).thenReturn(false);
topology = withGlobalStores(asList(store1, store2, store3, store4, store5), storeToTopic,
mkMap(
mkEntry(storeName1, Optional.empty()),
mkEntry(storeName2, Optional.empty()),
mkEntry(storeName3, Optional.empty()),
mkEntry(storeName4, Optional.empty()),
mkEntry(storeName5, optionalMockReprocessFactory)
)
);
streamsConfig = new StreamsConfig(new Properties() {
{
put(StreamsConfig.APPLICATION_ID_CONFIG, "appId");
@ -223,7 +250,7 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldReturnInitializedStoreNames() {
final Set<String> storeNames = stateManager.initialize();
assertEquals(Utils.mkSet(storeName1, storeName2, storeName3, storeName4), storeNames);
assertEquals(Utils.mkSet(storeName1, storeName2, storeName3, storeName4, storeName5), storeNames);
}
@Test
@ -691,7 +718,7 @@ public class GlobalStateManagerImplTest {
return numberOfCalls.incrementAndGet();
}
};
initializeConsumer(0, 0, t1, t2, t3, t4);
initializeConsumer(0, 0, t1, t2, t3, t4, t5);
streamsConfig = new StreamsConfig(mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"),
@ -857,7 +884,7 @@ public class GlobalStateManagerImplTest {
return numberOfCalls.incrementAndGet();
}
};
initializeConsumer(0, 0, t1, t2, t3, t4);
initializeConsumer(0, 0, t1, t2, t3, t4, t5);
streamsConfig = new StreamsConfig(mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"),
@ -1018,7 +1045,7 @@ public class GlobalStateManagerImplTest {
throw new TimeoutException("KABOOM!");
}
};
initializeConsumer(0, 0, t1, t2, t3, t4);
initializeConsumer(0, 0, t1, t2, t3, t4, t5);
streamsConfig = new StreamsConfig(mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"),
@ -1092,6 +1119,44 @@ public class GlobalStateManagerImplTest {
assertThat(time.milliseconds() - startTime, equalTo(331_100L));
}
@SuppressWarnings("unchecked")
private void setUpReprocessing() {
final InternalTopologyBuilder.ReprocessFactory reprocessFactory = mock(InternalTopologyBuilder.ReprocessFactory.class);
final ProcessorSupplier processorSupplier = mock(ProcessorSupplier.class);
final Processor processor = mock(Processor.class);
final Deserializer deserializer = mock(Deserializer.class);
when(optionalMockReprocessFactory.isPresent()).thenReturn(true);
when(optionalMockReprocessFactory.get()).thenReturn(reprocessFactory);
when(reprocessFactory.processorSupplier()).thenReturn(processorSupplier);
when(processorSupplier.get()).thenReturn(processor);
when(reprocessFactory.keyDeserializer()).thenReturn(deserializer);
when(reprocessFactory.valueDeserializer()).thenReturn(deserializer);
when(deserializer.deserialize(any(), any())).thenThrow(new StreamsException("fail"));
}
@Test
public void shouldFailOnDeserializationErrorsWhenReprocessing() {
setUpReprocessing();
initializeConsumer(2, 0, t5);
stateManager.initialize();
assertThrows(StreamsException.class, () -> stateManager.registerStore(store5, stateRestoreCallback, null));
}
@Test
public void shouldSkipOnDeserializationErrorsWhenReprocessing() {
stateManager.setDeserializationExceptionHandler(new LogAndContinueExceptionHandler());
setUpReprocessing();
initializeConsumer(2, 0, t5);
stateManager.initialize();
stateManager.registerStore(store5, stateRestoreCallback, null);
assertEquals(0, stateRestoreCallback.restored.size());
}
private void writeCorruptCheckpoint() throws IOException {
final File checkpointFile = new File(stateManager.baseDir(), StateManagerUtil.CHECKPOINT_FILE_NAME);
try (final OutputStream stream = Files.newOutputStream(checkpointFile.toPath())) {

View File

@ -113,7 +113,9 @@ public class GlobalStreamThreadTest {
null,
GLOBAL_STORE_TOPIC_NAME,
"processorName",
processorSupplier);
processorSupplier,
false
);
baseDirectoryName = TestUtils.tempDirectory().getAbsolutePath();
final HashMap<String, Object> properties = new HashMap<>();

View File

@ -221,7 +221,9 @@ public class InternalTopologyBuilderTest {
null,
"globalTopic",
"global-processor",
() -> processor)
() -> processor,
false
)
);
assertThat(exception.getMessage(), containsString("#get() must return a new object each time it is called."));
}
@ -329,7 +331,8 @@ public class InternalTopologyBuilderTest {
null,
"globalTopic",
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
);
builder.initializeSubscription();
@ -353,7 +356,8 @@ public class InternalTopologyBuilderTest {
null,
"globalTopic",
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
);
builder.initializeSubscription();
@ -465,7 +469,8 @@ public class InternalTopologyBuilderTest {
null,
"global-topic",
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
);
final TopologyException exception = assertThrows(
@ -496,7 +501,8 @@ public class InternalTopologyBuilderTest {
null,
"global-topic",
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
)
);
@ -521,7 +527,8 @@ public class InternalTopologyBuilderTest {
null,
"global-topic",
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
);
final TopologyException exception = assertThrows(
@ -534,7 +541,8 @@ public class InternalTopologyBuilderTest {
null,
"global-topic",
"global-processor-2",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
)
);
@ -729,7 +737,8 @@ public class InternalTopologyBuilderTest {
null,
"globalTopic",
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
);
newNodeGroups = builder.nodeGroups();
assertNotEquals(oldNodeGroups, newNodeGroups);
@ -1153,7 +1162,8 @@ public class InternalTopologyBuilderTest {
null,
"anyTopicName",
sameNameForSourceAndProcessor,
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
));
}
@ -1297,7 +1307,8 @@ public class InternalTopologyBuilderTest {
null,
globalTopic,
"global-processor",
new MockApiProcessorSupplier<>()
new MockApiProcessorSupplier<>(),
false
);
builder.initializeSubscription();

View File

@ -35,7 +35,8 @@ public final class ProcessorTopologyFactories {
stateStoresByName,
Collections.emptyList(),
storeToChangelogTopic,
Collections.emptySet());
Collections.emptySet(),
Collections.emptyMap());
}
static ProcessorTopology withLocalStores(final List<StateStore> stateStores,
@ -46,7 +47,9 @@ public final class ProcessorTopologyFactories {
stateStores,
Collections.emptyList(),
storeToChangelogTopic,
Collections.emptySet());
Collections.emptySet(),
Collections.emptyMap());
}
}

View File

@ -168,7 +168,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,
@ -333,7 +334,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,
@ -400,7 +402,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,
@ -467,7 +470,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,
@ -546,7 +550,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,
@ -599,7 +604,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,
@ -649,7 +655,8 @@ public class RecordCollectorTest {
emptyList(),
emptyList(),
emptyMap(),
emptySet()
emptySet(),
emptyMap()
);
collector = new RecordCollectorImpl(
logContext,

View File

@ -212,7 +212,8 @@ public class StreamTaskTest {
emptyList(),
emptyList(),
emptyMap(),
repartitionTopics);
repartitionTopics,
emptyMap());
}
private static ProcessorTopology withSources(final List<ProcessorNode<?, ?, ?, ?>> processorNodes,
@ -223,7 +224,8 @@ public class StreamTaskTest {
emptyList(),
emptyList(),
emptyMap(),
Collections.emptySet());
Collections.emptySet(),
emptyMap());
}
private static StreamsConfig createConfig() {