KAFKA-9113: Clean up task management and state management (#7997)

This PR is collaborated by Guozhang Wang and John Roesler. It is a significant tech debt cleanup on task management and state management, and is broken down by several sub-tasks listed below:

Extract embedded clients (producer and consumer) into RecordCollector from StreamTask.
guozhangwang#2
guozhangwang#5

Consolidate the standby updating and active restoring logic into ChangelogReader and extract out of StreamThread.
guozhangwang#3
guozhangwang#4

Introduce Task state life cycle (created, restoring, running, suspended, closing), and refactor the task operations based on the current state.
guozhangwang#6
guozhangwang#7

Consolidate AssignedTasks into TaskManager and simplify the logic of changelog management and task management (since they are already moved in step 2) and 3)).
guozhangwang#8
guozhangwang#9

Also simplified the StreamThread logic a bit as the embedded clients / changelog restoration logic has been moved into step 1) and 2).
guozhangwang#10

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Boyang Chen <boyang@confluent.io>
This commit is contained in:
Guozhang Wang 2020-02-04 21:06:39 -08:00 committed by GitHub
parent a16dfe6739
commit 4090f9a2b0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
87 changed files with 7102 additions and 9859 deletions

View File

@ -160,6 +160,8 @@
<suppress checks="ParameterNumber"
files="StreamTask.java"/>
<suppress checks="ParameterNumber"
files="StreamThread.java"/>
<suppress checks="ParameterNumber"
files="RocksDBWindowStoreSupplier.java"/>

View File

@ -1043,7 +1043,7 @@ public abstract class AbstractCoordinator implements Closeable {
log.error("Received fatal exception: group.instance.id gets fenced");
future.raise(error);
} else if (error == Errors.UNKNOWN_MEMBER_ID) {
log.info("Attempt to heartbeat failed for since member id {} is not valid.", generation.memberId);
log.info("Attempt to heartbeat failed since member id {} is not valid.", generation.memberId);
resetGenerationOnResponseError(ApiKeys.HEARTBEAT, error);
future.raise(error);
} else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {

View File

@ -19,16 +19,16 @@ package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.clients.GroupRebalanceConfig;
import org.apache.kafka.clients.consumer.CommitFailedException;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.clients.consumer.RetriableCommitFailedException;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
@ -366,14 +366,14 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
revokedPartitions.removeAll(assignedPartitions);
log.info("Updating assignment with\n" +
"now assigned partitions: {}\n" +
"compare with previously owned partitions: {}\n" +
"newly added partitions: {}\n" +
"revoked partitions: {}\n",
Utils.join(assignedPartitions, ", "),
Utils.join(ownedPartitions, ", "),
Utils.join(addedPartitions, ", "),
Utils.join(revokedPartitions, ", ")
"\tAssigned partitions: {}\n" +
"\tCurrent owned partitions: {}\n" +
"\tAdded partitions (assigned - owned): {}\n" +
"\tRevoked partitions (owned - assigned): {}\n",
assignedPartitions,
ownedPartitions,
addedPartitions,
revokedPartitions
);
if (!revokedPartitions.isEmpty()) {
@ -709,10 +709,10 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
if (subscriptions.hasAutoAssignedPartitions() && !droppedPartitions.isEmpty()) {
final Exception e;
if (generation() != Generation.NO_GENERATION) {
e = invokePartitionsRevoked(droppedPartitions);
} else {
if (generation() == Generation.NO_GENERATION || rebalanceInProgress()) {
e = invokePartitionsLost(droppedPartitions);
} else {
e = invokePartitionsRevoked(droppedPartitions);
}
subscriptions.assignFromSubscribed(Collections.emptySet());

View File

@ -39,6 +39,7 @@ public interface Callback {
* RecordBatchTooLargeException
* RecordTooLargeException
* UnknownServerException
* UnknownProducerIdException
*
* Retriable exceptions (transient, may be covered by increasing #.retries):
*

View File

@ -1143,7 +1143,9 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
* block forever.</strong>
* <p>
*
* @throws InterruptException If the thread is interrupted while blocked
* @throws InterruptException If the thread is interrupted while blocked.
* @throws KafkaException If a unexpected error occurs while trying to close the client, this error should be treated
* as fatal and indicate the client is no longer functionable.
*/
@Override
public void close() {
@ -1163,7 +1165,9 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
*
* @param timeout The maximum time to wait for producer to complete any pending requests. The value should be
* non-negative. Specifying a timeout of zero means do not wait for pending send requests to complete.
* @throws InterruptException If the thread is interrupted while blocked
* @throws InterruptException If the thread is interrupted while blocked.
* @throws KafkaException If a unexpected error occurs while trying to close the client, this error should be treated
* as fatal and indicate the client is no longer functionable.
* @throws IllegalArgumentException If the <code>timeout</code> is negative.
*
*/

View File

@ -67,7 +67,7 @@ public class MockProducer<K, V> implements Producer<K, V> {
private boolean transactionCommitted;
private boolean transactionAborted;
private boolean producerFenced;
private boolean producerFencedOnClose;
private boolean producerFencedOnCommitTxn;
private boolean sentOffsets;
private long commitCount = 0L;
private Map<MetricName, Metric> mockMetrics;
@ -182,6 +182,10 @@ public class MockProducer<K, V> implements Producer<K, V> {
@Override
public void commitTransaction() throws ProducerFencedException {
if (producerFencedOnCommitTxn) {
throw new ProducerFencedException("Producer is fenced");
}
verifyProducerState();
verifyTransactionsInitialized();
verifyNoTransactionInFlight();
@ -325,9 +329,6 @@ public class MockProducer<K, V> implements Producer<K, V> {
@Override
public void close(Duration timeout) {
if (producerFencedOnClose) {
throw new ProducerFencedException("MockProducer is fenced.");
}
this.closed = true;
}
@ -341,10 +342,10 @@ public class MockProducer<K, V> implements Producer<K, V> {
this.producerFenced = true;
}
public void fenceProducerOnClose() {
public void fenceProducerOnCommitTxn() {
verifyProducerState();
verifyTransactionsInitialized();
this.producerFencedOnClose = true;
this.producerFencedOnCommitTxn = true;
}
public boolean transactionInitialized() {

View File

@ -52,17 +52,15 @@ import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.StateRestoreListener;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.ThreadMetadata;
import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier;
import org.apache.kafka.streams.processor.internals.GlobalStreamThread;
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
import org.apache.kafka.streams.processor.internals.ProcessorTopology;
import org.apache.kafka.streams.processor.internals.StandbyTask;
import org.apache.kafka.streams.processor.internals.StateDirectory;
import org.apache.kafka.streams.processor.internals.StreamTask;
import org.apache.kafka.streams.processor.internals.StreamThread;
import org.apache.kafka.streams.processor.internals.StreamsMetadataState;
import org.apache.kafka.streams.processor.internals.Task;
import org.apache.kafka.streams.processor.internals.ThreadStateTransitionValidator;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.state.HostInfo;
@ -1210,34 +1208,16 @@ public class KafkaStreams implements AutoCloseable {
* @return map of store names to another map of partition to {@link LagInfo}s
*/
public Map<String, Map<Integer, LagInfo>> allLocalStorePartitionLags() {
final long latestSentinel = -2L;
final Map<String, Map<Integer, LagInfo>> localStorePartitionLags = new TreeMap<>();
final Collection<TopicPartition> allPartitions = new LinkedList<>();
final Map<TopicPartition, Long> allChangelogPositions = new HashMap<>();
// Obtain the current positions, of all the active-restoring and standby tasks
for (final StreamThread streamThread : threads) {
for (final StandbyTask standbyTask : streamThread.allStandbyTasks()) {
allPartitions.addAll(standbyTask.changelogPartitions());
for (final Task task : streamThread.allTasks().values()) {
allPartitions.addAll(task.changelogPartitions());
// Note that not all changelog partitions, will have positions; since some may not have started
allChangelogPositions.putAll(standbyTask.changelogPositions());
}
final Set<TaskId> restoringTaskIds = streamThread.restoringTaskIds();
for (final StreamTask activeTask : streamThread.allStreamsTasks()) {
final Collection<TopicPartition> taskChangelogPartitions = activeTask.changelogPartitions();
allPartitions.addAll(taskChangelogPartitions);
final boolean isRestoring = restoringTaskIds.contains(activeTask.id());
final Map<TopicPartition, Long> restoredOffsets = activeTask.restoredOffsets();
for (final TopicPartition topicPartition : taskChangelogPartitions) {
if (isRestoring && restoredOffsets.containsKey(topicPartition)) {
allChangelogPositions.put(topicPartition, restoredOffsets.get(topicPartition));
} else {
allChangelogPositions.put(topicPartition, latestSentinel);
}
}
allChangelogPositions.putAll(task.changelogOffsets());
}
}
@ -1251,6 +1231,7 @@ public class KafkaStreams implements AutoCloseable {
} catch (final RuntimeException | InterruptedException | ExecutionException e) {
throw new StreamsException("Unable to obtain end offsets from kafka", e);
}
log.debug("Current end offsets :{}", allEndOffsets);
for (final Map.Entry<TopicPartition, ListOffsetsResultInfo> entry : allEndOffsets.entrySet()) {
// Avoiding an extra admin API lookup by computing lags for not-yet-started restorations
@ -1260,7 +1241,7 @@ public class KafkaStreams implements AutoCloseable {
final long earliestOffset = 0L;
final long changelogPosition = allChangelogPositions.getOrDefault(entry.getKey(), earliestOffset);
final long latestOffset = entry.getValue().offset();
final LagInfo lagInfo = new LagInfo(changelogPosition == latestSentinel ? latestOffset : changelogPosition, latestOffset);
final LagInfo lagInfo = new LagInfo(changelogPosition == Task.LATEST_OFFSET ? latestOffset : changelogPosition, latestOffset);
final String storeName = streamsMetadataState.getStoreForChangelogTopic(entry.getKey().topic());
localStorePartitionLags.computeIfAbsent(storeName, ignored -> new TreeMap<>())
.put(entry.getKey().partition(), lagInfo);

View File

@ -796,6 +796,8 @@ public class StreamsConfig extends AbstractConfig {
public static class InternalConfig {
public static final String TASK_MANAGER_FOR_PARTITION_ASSIGNOR = "__task.manager.instance__";
public static final String STREAMS_METADATA_STATE_FOR_PARTITION_ASSIGNOR = "__streams.metadata.state.instance__";
public static final String STREAMS_ADMIN_CLIENT = "__streams.admin.client.instance__";
public static final String ASSIGNMENT_ERROR_CODE = "__assignment.error.code__";
}

View File

@ -18,53 +18,57 @@ package org.apache.kafka.streams.errors;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.streams.processor.internals.Task;
import org.apache.kafka.streams.processor.TaskId;
/**
* Indicates that a task got migrated to another thread.
* Thus, the task raising this exception can be cleaned up and closed as "zombie".
* Indicates that one or more tasks got migrated to another thread.
*
* 1) if the task field is specified, then that single task should be cleaned up and closed as "zombie" while the
* thread can continue as normal;
* 2) if no tasks are specified (i.e. taskId == null), it means that the hosted thread has been fenced and all
* tasks are migrated, in which case the thread should rejoin the group
*/
public class TaskMigratedException extends StreamsException {
private final static long serialVersionUID = 1L;
private final Task task;
private final TaskId taskId;
// this is for unit test only
public TaskMigratedException() {
super("A task has been migrated unexpectedly", null);
this.task = null;
}
public TaskMigratedException(final Task task,
public TaskMigratedException(final TaskId taskId,
final TopicPartition topicPartition,
final long endOffset,
final long pos) {
super(String.format("Log end offset of %s should not change while restoring: old end offset %d, current offset %d",
topicPartition,
endOffset,
pos),
null);
this.task = task;
this(taskId, String.format("Log end offset of %s should not change while restoring: old end offset %d, current offset %d",
topicPartition,
endOffset,
pos), null);
}
public TaskMigratedException(final Task task) {
super(String.format("Task %s is unexpectedly closed during processing", task.id()), null);
this.task = task;
public TaskMigratedException(final TaskId taskId) {
this(taskId, String.format("Task %s is unexpectedly closed during processing", taskId), null);
}
public TaskMigratedException(final Task task,
public TaskMigratedException(final TaskId taskId,
final Throwable throwable) {
super(String.format("Client request for task %s has been fenced due to a rebalance", task.id()), throwable);
this.task = task;
this(taskId, String.format("Client request for task %s has been fenced due to a rebalance", taskId), throwable);
}
public Task migratedTask() {
return task;
public TaskMigratedException(final TaskId taskId,
final String message,
final Throwable throwable) {
super(message, throwable);
this.taskId = taskId;
}
public TaskMigratedException(final String message, final Throwable throwable) {
this(null, message + " It means all tasks belonging to this thread have been migrated", throwable);
}
public TaskId migratedTaskId() {
return taskId;
}
public TaskMigratedException() {
this(null, "A task has been migrated unexpectedly", null);
}
}

View File

@ -98,7 +98,7 @@ public abstract class AbstractProcessorContext implements InternalProcessorConte
throw new IllegalStateException("Can only create state stores during initialization.");
}
Objects.requireNonNull(store, "store must not be null");
stateManager.register(store, stateRestoreCallback);
stateManager.registerStore(store, stateRestoreCallback);
}
/**

View File

@ -16,82 +16,33 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.LockException;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.TaskId;
import org.slf4j.Logger;
import java.io.IOException;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import static org.apache.kafka.streams.processor.internals.Task.State.CREATED;
public abstract class AbstractTask implements Task {
private Task.State state = CREATED;
final TaskId id;
final String applicationId;
final ProcessorTopology topology;
final ProcessorStateManager stateMgr;
final Set<TopicPartition> partitions;
final Consumer<byte[], byte[]> consumer;
final String logPrefix;
final boolean eosEnabled;
final Logger log;
final LogContext logContext;
final StateDirectory stateDirectory;
protected final TaskId id;
protected final ProcessorTopology topology;
protected final StateDirectory stateDirectory;
protected final Set<TopicPartition> partitions;
protected final ProcessorStateManager stateMgr;
boolean taskInitialized;
boolean taskClosed;
boolean commitNeeded;
InternalProcessorContext processorContext;
/**
* @throws ProcessorStateException if the state manager cannot be created
*/
AbstractTask(final TaskId id,
final Set<TopicPartition> partitions,
final ProcessorTopology topology,
final Consumer<byte[], byte[]> consumer,
final ChangelogReader changelogReader,
final boolean isStandby,
final StateDirectory stateDirectory,
final StreamsConfig config) {
final ProcessorStateManager stateMgr,
final Set<TopicPartition> partitions) {
this.id = id;
this.applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
this.partitions = new HashSet<>(partitions);
this.stateMgr = stateMgr;
this.topology = topology;
this.consumer = consumer;
this.eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
this.partitions = partitions;
this.stateDirectory = stateDirectory;
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
this.logPrefix = threadIdPrefix + String.format("%s [%s] ", isStandby ? "standby-task" : "task", id);
this.logContext = new LogContext(logPrefix);
this.log = logContext.logger(getClass());
// create the processor state manager
try {
stateMgr = new ProcessorStateManager(
id,
partitions,
isStandby,
stateDirectory,
topology.storeToChangelogTopic(),
changelogReader,
eosEnabled,
logContext);
} catch (final IOException e) {
throw new ProcessorStateException(String.format("%sError while creating the state manager", logPrefix), e);
}
}
@Override
@ -100,158 +51,32 @@ public abstract class AbstractTask implements Task {
}
@Override
public String applicationId() {
return applicationId;
}
@Override
public Set<TopicPartition> partitions() {
public Set<TopicPartition> inputPartitions() {
return partitions;
}
@Override
public ProcessorTopology topology() {
return topology;
}
@Override
public ProcessorContext context() {
return processorContext;
}
@Override
public StateStore getStore(final String name) {
return stateMgr.getStore(name);
}
/**
* Produces a string representation containing useful information about a Task.
* This is useful in debugging scenarios.
*
* @return A string representation of the StreamTask instance.
*/
@Override
public String toString() {
return toString("");
}
public boolean isEosEnabled() {
return eosEnabled;
}
/**
* Produces a string representation containing useful information about a Task starting with the given indent.
* This is useful in debugging scenarios.
*
* @return A string representation of the Task instance.
*/
public String toString(final String indent) {
final StringBuilder sb = new StringBuilder();
sb.append(indent);
sb.append("TaskId: ");
sb.append(id);
sb.append("\n");
// print topology
if (topology != null) {
sb.append(indent).append(topology.toString(indent + "\t"));
}
// print assigned partitions
if (partitions != null && !partitions.isEmpty()) {
sb.append(indent).append("Partitions [");
for (final TopicPartition topicPartition : partitions) {
sb.append(topicPartition.toString()).append(", ");
}
sb.setLength(sb.length() - 2);
sb.append("]\n");
}
return sb.toString();
}
/**
* Flush all state stores owned by this task
*/
void flushState() {
try {
stateMgr.flush();
} catch (final ProcessorStateException e) {
if (e.getCause() instanceof RecoverableClientException) {
throw new TaskMigratedException(this, e);
}
}
}
/**
* Package-private for testing only
*
* @throws StreamsException If the store's change log does not contain the partition
*/
void registerStateStores() {
if (topology.stateStores().isEmpty()) {
return;
}
try {
if (!stateDirectory.lock(id)) {
throw new LockException(String.format("%sFailed to lock the state directory for task %s", logPrefix, id));
}
} catch (final IOException e) {
throw new StreamsException(
String.format("%sFatal error while trying to lock the state directory for task %s", logPrefix, id),
e
);
}
log.trace("Initializing state stores");
for (final StateStore store : topology.stateStores()) {
log.debug("Initializing store {}", store.name());
processorContext.uninitialize();
store.init(processorContext, store);
}
}
void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions) {
stateMgr.reinitializeStateStoresForPartitions(partitions, processorContext);
}
/**
* @throws ProcessorStateException if there is an error while closing the state manager
*/
void closeStateManager(final boolean clean) throws ProcessorStateException {
ProcessorStateException exception = null;
log.trace("Closing state manager");
try {
stateMgr.close(clean);
} catch (final ProcessorStateException e) {
exception = e;
} finally {
try {
stateDirectory.unlock(id);
} catch (final IOException e) {
if (exception == null) {
exception = new ProcessorStateException(String.format("%sFailed to release state dir lock", logPrefix), e);
}
}
}
if (exception != null) {
throw exception;
}
}
public boolean isClosed() {
return taskClosed;
return state() == State.CLOSED;
}
public boolean commitNeeded() {
return commitNeeded;
@Override
public final Task.State state() {
return state;
}
public boolean hasStateStores() {
return !topology.stateStores().isEmpty();
}
final void transitionTo(final Task.State newState) {
final State oldState = state();
public Collection<TopicPartition> changelogPartitions() {
return stateMgr.changelogPartitions();
if (oldState.isValidTransition(newState)) {
state = newState;
} else {
throw new IllegalStateException("Invalid transition from " + oldState + " to " + newState);
}
}
}

View File

@ -1,90 +0,0 @@
/*
* 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.processor.internals;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.processor.TaskId;
class AssignedStandbyTasks extends AssignedTasks<StandbyTask> {
AssignedStandbyTasks(final LogContext logContext) {
super(logContext, "standby task");
}
@Override
public void shutdown(final boolean clean) {
final String shutdownType = clean ? "Clean" : "Unclean";
log.debug("{} shutdown of all standby tasks" + "\n" +
"non-initialized standby tasks to close: {}" + "\n" +
"running standby tasks to close: {}",
shutdownType, created.keySet(), running.keySet());
super.shutdown(clean);
}
@Override
int commit() {
final int committed = super.commit();
// TODO: this contortion would not be necessary if we got rid of the two-step
// task.commitNeeded and task.commit and instead just had task.commitIfNeeded. Currently
// we only call commit if commitNeeded is true, which means that we need a way to indicate
// that we are eligible for updating the offset limit outside of commit.
running.forEach((id, task) -> task.allowUpdateOfOffsetLimit());
return committed;
}
/**
* Closes standby tasks that were reassigned elsewhere after a rebalance.
*
* @param revokedTasks the tasks which are no longer owned
* @return the changelogs of all standby tasks that were reassigned
*/
List<TopicPartition> closeRevokedStandbyTasks(final Map<TaskId, Set<TopicPartition>> revokedTasks) {
log.debug("Closing revoked standby tasks {}", revokedTasks);
final List<TopicPartition> revokedChangelogs = new ArrayList<>();
for (final Map.Entry<TaskId, Set<TopicPartition>> entry : revokedTasks.entrySet()) {
final TaskId taskId = entry.getKey();
final StandbyTask task;
if (running.containsKey(taskId)) {
task = running.get(taskId);
} else if (created.containsKey(taskId)) {
task = created.get(taskId);
} else {
log.error("Could not find the standby task {} while closing it", taskId);
continue;
}
try {
task.close(true, false);
} catch (final RuntimeException e) {
log.error("Closing the standby task {} failed due to the following error:", task.id(), e);
} finally {
removeTaskFromAllStateMaps(task, Collections.emptyMap());
revokedChangelogs.addAll(task.changelogPartitions());
}
}
return revokedChangelogs;
}
}

View File

@ -1,557 +0,0 @@
/*
* 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.processor.internals;
import java.util.ArrayList;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.TaskId;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
class AssignedStreamsTasks extends AssignedTasks<StreamTask> implements RestoringTasks {
private final Map<TaskId, StreamTask> suspended = new ConcurrentHashMap<>();
private final Map<TaskId, StreamTask> restoring = new ConcurrentHashMap<>();
private final Set<TopicPartition> restoredPartitions = new HashSet<>();
private final Map<TopicPartition, StreamTask> restoringByPartition = new HashMap<>();
private final Set<TaskId> prevActiveTasks = new HashSet<>();
AssignedStreamsTasks(final LogContext logContext) {
super(logContext, "stream task");
}
@Override
public StreamTask restoringTaskFor(final TopicPartition partition) {
return restoringByPartition.get(partition);
}
@Override
List<StreamTask> allTasks() {
final List<StreamTask> tasks = super.allTasks();
tasks.addAll(restoring.values());
tasks.addAll(suspended.values());
return tasks;
}
@Override
Set<TaskId> allAssignedTaskIds() {
final Set<TaskId> taskIds = super.allAssignedTaskIds();
taskIds.addAll(restoring.keySet());
taskIds.addAll(suspended.keySet());
return taskIds;
}
@Override
boolean allTasksRunning() {
// If we have some tasks that are suspended but others are running, count this as all tasks are running
// since they will be closed soon anyway (eg if partitions are revoked at beginning of cooperative rebalance)
return super.allTasksRunning() && restoring.isEmpty() && (suspended.isEmpty() || !running.isEmpty());
}
@Override
void closeTask(final StreamTask task, final boolean clean) {
if (suspended.containsKey(task.id())) {
task.closeSuspended(clean, null);
} else {
task.close(clean, false);
}
}
boolean hasRestoringTasks() {
return !restoring.isEmpty();
}
void clearRestoringPartitions() {
if (!restoring.isEmpty()) {
log.error("Tried to clear restoring partitions but was still restoring the stream tasks {}", restoring);
throw new IllegalStateException("Should not clear restoring partitions while set of restoring tasks is non-empty");
}
restoredPartitions.clear();
restoringByPartition.clear();
}
Set<TaskId> suspendedTaskIds() {
return suspended.keySet();
}
Set<TaskId> previousRunningTaskIds() {
return prevActiveTasks;
}
RuntimeException suspendOrCloseTasks(final Set<TaskId> revokedTasks,
final List<TopicPartition> revokedTaskChangelogs) {
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
final Set<TaskId> revokedRunningTasks = new HashSet<>();
final Set<TaskId> revokedNonRunningTasks = new HashSet<>();
final Set<TaskId> revokedRestoringTasks = new HashSet<>();
// This set is used only for eager rebalancing, so we can just clear it and add any/all tasks that were running
prevActiveTasks.clear();
prevActiveTasks.addAll(runningTaskIds());
for (final TaskId task : revokedTasks) {
if (running.containsKey(task)) {
revokedRunningTasks.add(task);
} else if (created.containsKey(task)) {
revokedNonRunningTasks.add(task);
} else if (restoring.containsKey(task)) {
revokedRestoringTasks.add(task);
} else if (!suspended.containsKey(task)) {
log.warn("Stream task {} was revoked but cannot be found in the assignment, may have been closed due to error", task);
}
}
firstException.compareAndSet(null, suspendRunningTasks(revokedRunningTasks, revokedTaskChangelogs));
firstException.compareAndSet(null, closeNonRunningTasks(revokedNonRunningTasks, revokedTaskChangelogs));
firstException.compareAndSet(null, closeRestoringTasks(revokedRestoringTasks, revokedTaskChangelogs));
return firstException.get();
}
private RuntimeException suspendRunningTasks(final Set<TaskId> runningTasksToSuspend,
final List<TopicPartition> taskChangelogs) {
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
log.debug("Suspending the running stream tasks {}", running.keySet());
for (final TaskId id : runningTasksToSuspend) {
final StreamTask task = running.get(id);
try {
task.suspend();
suspended.put(id, task);
} catch (final TaskMigratedException closeAsZombieAndSwallow) {
// swallow and move on since we are rebalancing
log.info("Failed to suspend stream task {} since it got migrated to another thread already. " +
"Closing it as zombie and moving on.", id);
tryCloseZombieTask(task);
prevActiveTasks.remove(id);
} catch (final RuntimeException e) {
log.error("Suspending stream task {} failed due to the following error:", id, e);
firstException.compareAndSet(null, e);
try {
prevActiveTasks.remove(id);
task.close(false, false);
} catch (final RuntimeException f) {
log.error(
"After suspending failed, closing the same stream task {} failed again due to the following error:",
id, f);
}
} finally {
removeTaskFromAllStateMaps(task, suspended);
taskChangelogs.addAll(task.changelogPartitions());
}
}
log.trace("Successfully suspended the running stream task {}", suspended.keySet());
return firstException.get();
}
private RuntimeException closeNonRunningTasks(final Set<TaskId> nonRunningTasksToClose,
final List<TopicPartition> closedTaskChangelogs) {
log.debug("Closing the created but not initialized stream tasks {}", nonRunningTasksToClose);
final AtomicReference<RuntimeException> firstException = new AtomicReference<>();
for (final TaskId id : nonRunningTasksToClose) {
final StreamTask task = created.get(id);
firstException.compareAndSet(null, closeNonRunning(false, task, closedTaskChangelogs));
}
return firstException.get();
}
RuntimeException closeRestoringTasks(final Set<TaskId> restoringTasksToClose,
final List<TopicPartition> closedTaskChangelogs) {
log.debug("Closing restoring stream tasks {}", restoringTasksToClose);
final AtomicReference<RuntimeException> firstException = new AtomicReference<>();
for (final TaskId id : restoringTasksToClose) {
final StreamTask task = restoring.get(id);
firstException.compareAndSet(null, closeRestoring(false, task, closedTaskChangelogs));
}
return firstException.get();
}
private RuntimeException closeRunning(final boolean isZombie,
final StreamTask task) {
removeTaskFromAllStateMaps(task, Collections.emptyMap());
try {
final boolean clean = !isZombie;
task.close(clean, isZombie);
} catch (final RuntimeException e) {
log.error("Failed to close the stream task {}", task.id(), e);
return e;
}
return null;
}
private RuntimeException closeNonRunning(final boolean isZombie,
final StreamTask task,
final List<TopicPartition> closedTaskChangelogs) {
removeTaskFromAllStateMaps(task, Collections.emptyMap());
closedTaskChangelogs.addAll(task.changelogPartitions());
try {
task.close(false, isZombie);
} catch (final RuntimeException e) {
log.error("Failed to close the stream task {}", task.id(), e);
return e;
}
return null;
}
// Since a restoring task has not had its topology initialized yet, we need only close the state manager
private RuntimeException closeRestoring(final boolean isZombie,
final StreamTask task,
final List<TopicPartition> closedTaskChangelogs) {
removeTaskFromAllStateMaps(task, Collections.emptyMap());
closedTaskChangelogs.addAll(task.changelogPartitions());
try {
final boolean clean = !isZombie;
task.closeStateManager(clean);
} catch (final RuntimeException e) {
log.error("Failed to close the restoring stream task {} due to the following error:", task.id(), e);
return e;
}
return null;
}
private RuntimeException closeSuspended(final boolean isZombie,
final StreamTask task) {
removeTaskFromAllStateMaps(task, Collections.emptyMap());
try {
final boolean clean = !isZombie;
task.closeSuspended(clean, null);
} catch (final RuntimeException e) {
log.error("Failed to close the suspended stream task {} due to the following error:", task.id(), e);
return e;
}
return null;
}
RuntimeException closeNotAssignedSuspendedTasks(final Set<TaskId> revokedTasks) {
log.debug("Closing the revoked active stream tasks {}", revokedTasks);
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
for (final TaskId revokedTask : revokedTasks) {
final StreamTask suspendedTask = suspended.get(revokedTask);
if (suspendedTask != null) {
firstException.compareAndSet(null, closeSuspended(false, suspendedTask));
} else {
log.debug("Revoked stream task {} could not be found in suspended, may have already been closed", revokedTask);
}
}
return firstException.get();
}
RuntimeException closeAllTasksAsZombies() {
log.debug("Closing all active tasks as zombies, current state of active tasks: {}", toString());
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
final List<TopicPartition> changelogs = new ArrayList<>(); // not used, as we clear/unsubscribe all changelogs
for (final TaskId id : allAssignedTaskIds()) {
if (running.containsKey(id)) {
log.debug("Closing the zombie running stream task {}.", id);
firstException.compareAndSet(null, closeRunning(true, running.get(id)));
} else if (created.containsKey(id)) {
log.debug("Closing the zombie created stream task {}.", id);
firstException.compareAndSet(null, closeNonRunning(true, created.get(id), changelogs));
} else if (restoring.containsKey(id)) {
log.debug("Closing the zombie restoring stream task {}.", id);
firstException.compareAndSet(null, closeRestoring(true, restoring.get(id), changelogs));
} else if (suspended.containsKey(id)) {
log.debug("Closing the zombie suspended stream task {}.", id);
firstException.compareAndSet(null, closeSuspended(true, suspended.get(id)));
}
}
clear();
return firstException.get();
}
/**
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
boolean maybeResumeSuspendedTask(final TaskId taskId,
final Set<TopicPartition> partitions) {
if (suspended.containsKey(taskId)) {
final StreamTask task = suspended.get(taskId);
log.trace("Found suspended stream task {}", taskId);
removeTaskFromAllStateMaps(task, Collections.emptyMap());
if (task.partitions().equals(partitions)) {
task.resume();
try {
transitionToRunning(task);
} catch (final TaskMigratedException e) {
// we need to catch migration exception internally since this function
// is triggered in the rebalance callback
log.info("Failed to resume stream task {} since it got migrated to another thread already. " +
"Will trigger a new rebalance and close all tasks as zombies together.", task.id());
throw e;
}
log.trace("Resuming the suspended stream task {}", task.id());
return true;
} else {
log.warn("Couldn't resume stream task {} assigned partitions {}, task partitions {}", taskId, partitions, task.partitions());
task.closeSuspended(true, null);
}
}
return false;
}
void updateRestored(final Collection<TopicPartition> restored) {
if (restored.isEmpty()) {
return;
}
log.trace("Stream task changelog partitions that have completed restoring so far: {}", restored);
restoredPartitions.addAll(restored);
for (final Iterator<Map.Entry<TaskId, StreamTask>> it = restoring.entrySet().iterator(); it.hasNext(); ) {
final Map.Entry<TaskId, StreamTask> entry = it.next();
final StreamTask task = entry.getValue();
if (restoredPartitions.containsAll(task.changelogPartitions())) {
transitionToRunning(task);
it.remove();
// Note that because we add back all restored partitions at the top of this loop, clearing them from
// restoredPartitions here doesn't really matter. We do it anyway as it is the correct thing to do,
// and may matter with future changes.
removeFromRestoredPartitions(task);
removeFromRestoringByPartition(task);
log.debug("Stream task {} completed restoration as all its changelog partitions {} have been applied to restore state",
task.id(),
task.changelogPartitions());
} else {
if (log.isTraceEnabled()) {
final HashSet<TopicPartition> outstandingPartitions = new HashSet<>(task.changelogPartitions());
outstandingPartitions.removeAll(restoredPartitions);
log.trace("Stream task {} cannot resume processing yet since some of its changelog partitions have not completed restoring: {}",
task.id(),
outstandingPartitions);
}
}
}
if (allTasksRunning()) {
restoredPartitions.clear();
if (!restoringByPartition.isEmpty()) {
log.error("Finished restoring all tasks but found leftover partitions in restoringByPartition: {}",
restoringByPartition);
throw new IllegalStateException("Restoration is complete but not all partitions were cleared.");
}
}
}
@Override
void removeTaskFromAllStateMaps(final StreamTask task, final Map<TaskId, StreamTask> currentStateMap) {
super.removeTaskFromAllStateMaps(task, currentStateMap);
final TaskId id = task.id();
final Set<TopicPartition> taskPartitions = new HashSet<>(task.partitions());
taskPartitions.addAll(task.changelogPartitions());
if (currentStateMap != restoring) {
restoring.remove(id);
restoringByPartition.keySet().removeAll(taskPartitions);
restoredPartitions.removeAll(taskPartitions);
}
if (currentStateMap != suspended) {
suspended.remove(id);
}
}
void addTaskToRestoring(final StreamTask task) {
restoring.put(task.id(), task);
for (final TopicPartition topicPartition : task.partitions()) {
restoringByPartition.put(topicPartition, task);
}
for (final TopicPartition topicPartition : task.changelogPartitions()) {
restoringByPartition.put(topicPartition, task);
}
}
private void removeFromRestoringByPartition(final StreamTask task) {
restoringByPartition.keySet().removeAll(task.partitions());
restoringByPartition.keySet().removeAll(task.changelogPartitions());
}
private void removeFromRestoredPartitions(final StreamTask task) {
restoredPartitions.removeAll(task.partitions());
restoredPartitions.removeAll(task.changelogPartitions());
}
/**
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
int maybeCommitPerUserRequested() {
int committed = 0;
RuntimeException firstException = null;
for (final StreamTask task : running.values()) {
try {
if (task.commitRequested() && task.commitNeeded()) {
task.commit();
committed++;
log.debug("Committed stream task {} per user request in", task.id());
}
} catch (final TaskMigratedException e) {
log.info("Failed to commit stream task {} since it got migrated to another thread already. " +
"Will trigger a new rebalance and close all tasks as zombies together.", task.id());
throw e;
} catch (final RuntimeException t) {
log.error("Failed to commit stream task {} due to the following error:", task.id(), t);
if (firstException == null) {
firstException = t;
}
}
}
if (firstException != null) {
throw firstException;
}
return committed;
}
/**
* Returns a map of offsets up to which the records can be deleted; this function should only be called
* after the commit call to make sure all consumed offsets are actually committed as well
*/
Map<TopicPartition, Long> recordsToDelete() {
final Map<TopicPartition, Long> recordsToDelete = new HashMap<>();
for (final StreamTask task : running.values()) {
recordsToDelete.putAll(task.purgableOffsets());
}
return recordsToDelete;
}
/**
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
int process(final long now) {
int processed = 0;
for (final StreamTask task : running.values()) {
try {
if (task.isProcessable(now) && task.process()) {
processed++;
}
} catch (final TaskMigratedException e) {
log.info("Failed to process stream task {} since it got migrated to another thread already. " +
"Will trigger a new rebalance and close all tasks as zombies together.", task.id());
throw e;
} catch (final RuntimeException e) {
log.error("Failed to process stream task {} due to the following error:", task.id(), e);
throw e;
}
}
return processed;
}
/**
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
int punctuate() {
int punctuated = 0;
for (final StreamTask task : running.values()) {
try {
if (task.maybePunctuateStreamTime()) {
punctuated++;
}
if (task.maybePunctuateSystemTime()) {
punctuated++;
}
} catch (final TaskMigratedException e) {
log.info("Failed to punctuate stream task {} since it got migrated to another thread already. " +
"Will trigger a new rebalance and close all tasks as zombies together.", task.id());
throw e;
} catch (final KafkaException e) {
log.error("Failed to punctuate stream task {} due to the following error:", task.id(), e);
throw e;
}
}
return punctuated;
}
void clear() {
super.clear();
restoring.clear();
restoringByPartition.clear();
restoredPartitions.clear();
suspended.clear();
prevActiveTasks.clear();
}
@Override
public void shutdown(final boolean clean) {
final String shutdownType = clean ? "Clean" : "Unclean";
log.debug("{} shutdown of all active tasks" + "\n" +
"non-initialized stream tasks to close: {}" + "\n" +
"restoring tasks to close: {}" + "\n" +
"running stream tasks to close: {}" + "\n" +
"suspended stream tasks to close: {}",
shutdownType, created.keySet(), restoring.keySet(), running.keySet(), suspended.keySet());
super.shutdown(clean);
}
public String toString(final String indent) {
final StringBuilder builder = new StringBuilder();
builder.append(super.toString(indent));
describeTasks(builder, restoring.values(), indent, "Restoring:");
describePartitions(builder, restoringByPartition.keySet(), indent, "Restoring Partitions:");
describePartitions(builder, restoredPartitions, indent, "Restored Partitions:");
describeTasks(builder, suspended.values(), indent, "Suspended:");
return builder.toString();
}
// the following are for testing only
Collection<StreamTask> restoringTasks() {
return Collections.unmodifiableCollection(restoring.values());
}
Set<TaskId> restoringTaskIds() {
return new HashSet<>(restoring.keySet());
}
}

View File

@ -1,297 +0,0 @@
/*
* 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.processor.internals;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.errors.LockException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.TaskId;
import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
abstract class AssignedTasks<T extends Task> {
final Logger log;
final String taskTypeName;
final Map<TaskId, T> created = new ConcurrentHashMap<>();
// IQ may access this map.
final Map<TaskId, T> running = new ConcurrentHashMap<>();
final Map<TopicPartition, T> runningByPartition = new HashMap<>();
AssignedTasks(final LogContext logContext,
final String taskTypeName) {
this.taskTypeName = taskTypeName;
this.log = logContext.logger(getClass());
}
void addNewTask(final T task) {
created.put(task.id(), task);
}
/**
* @throws IllegalStateException If store gets registered after initialized is already finished
* @throws StreamsException if the store's changelog does not contain the partition
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
void initializeNewTasks() {
if (!created.isEmpty()) {
log.debug("Initializing {}s {}", taskTypeName, created.keySet());
}
for (final Iterator<Map.Entry<TaskId, T>> it = created.entrySet().iterator(); it.hasNext(); ) {
final Map.Entry<TaskId, T> entry = it.next();
try {
final T task = entry.getValue();
task.initializeMetadata();
// don't remove from created until the task has been successfully initialized
removeTaskFromAllStateMaps(task, created);
if (!task.initializeStateStores()) {
log.debug("Transitioning {} {} to restoring", taskTypeName, entry.getKey());
((AssignedStreamsTasks) this).addTaskToRestoring((StreamTask) task);
} else {
transitionToRunning(task);
}
it.remove();
} catch (final LockException e) {
// If this is a permanent error, then we could spam the log since this is in the run loop. But, other related
// messages show up anyway. So keeping in debug for sake of faster discoverability of problem
log.debug("Could not create {} {} due to {}; will retry", taskTypeName, entry.getKey(), e.toString());
}
}
}
boolean allTasksRunning() {
return created.isEmpty();
}
Collection<T> running() {
return running.values();
}
void tryCloseZombieTask(final T task) {
try {
task.close(false, true);
} catch (final RuntimeException e) {
log.warn("Failed to close zombie {} {} due to {}; ignore and proceed.", taskTypeName, task.id(), e.toString());
}
}
boolean hasRunningTasks() {
return !running.isEmpty();
}
/**
* @throws TaskMigratedException if the task producer got fenced (EOS only)
*/
void transitionToRunning(final T task) {
log.debug("Transitioning {} {} to running", taskTypeName, task.id());
running.put(task.id(), task);
task.initializeTopology();
for (final TopicPartition topicPartition : task.partitions()) {
runningByPartition.put(topicPartition, task);
}
for (final TopicPartition topicPartition : task.changelogPartitions()) {
runningByPartition.put(topicPartition, task);
}
}
/**
* Removes the passed in task (and its corresponding partitions) from all state maps and sets,
* except for the one it currently resides in.
*
* @param task the task to be removed
* @param currentStateMap the current state map, which the task should not be removed from
*/
void removeTaskFromAllStateMaps(final T task, final Map<TaskId, T> currentStateMap) {
final TaskId id = task.id();
final Set<TopicPartition> taskPartitions = new HashSet<>(task.partitions());
taskPartitions.addAll(task.changelogPartitions());
if (currentStateMap != running) {
running.remove(id);
runningByPartition.keySet().removeAll(taskPartitions);
}
if (currentStateMap != created) {
created.remove(id);
}
}
T runningTaskFor(final TopicPartition partition) {
return runningByPartition.get(partition);
}
Set<TaskId> runningTaskIds() {
return running.keySet();
}
Map<TaskId, T> runningTaskMap() {
return Collections.unmodifiableMap(running);
}
@Override
public String toString() {
return toString("");
}
public String toString(final String indent) {
final StringBuilder builder = new StringBuilder();
describeTasks(builder, running.values(), indent, "Running:");
describePartitions(builder, runningByPartition.keySet(), indent, "Running Partitions:");
describeTasks(builder, created.values(), indent, "New:");
return builder.toString();
}
void describeTasks(final StringBuilder builder,
final Collection<T> tasks,
final String indent,
final String name) {
builder.append(indent).append(name);
for (final T t : tasks) {
builder.append(indent).append(t.toString(indent + "\t\t"));
}
builder.append("\n");
}
void describePartitions(final StringBuilder builder,
final Collection<TopicPartition> partitions,
final String indent,
final String name) {
builder.append(indent).append(name);
for (final TopicPartition tp : partitions) {
builder.append(indent).append(tp.toString());
}
builder.append("\n");
}
List<T> allTasks() {
final List<T> tasks = new ArrayList<>();
tasks.addAll(running.values());
tasks.addAll(created.values());
return tasks;
}
Set<TaskId> allAssignedTaskIds() {
final Set<TaskId> taskIds = new HashSet<>();
taskIds.addAll(running.keySet());
taskIds.addAll(created.keySet());
return taskIds;
}
void clear() {
runningByPartition.clear();
running.clear();
created.clear();
}
/**
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
*/
int commit() {
int committed = 0;
RuntimeException firstException = null;
for (final T task : running.values()) {
try {
if (task.commitNeeded()) {
task.commit();
committed++;
}
} catch (final TaskMigratedException e) {
log.info("Failed to commit {} {} since it got migrated to another thread already. " +
"Will trigger a new rebalance and close all tasks as zombies together.", taskTypeName, task.id());
throw e;
} catch (final RuntimeException t) {
log.error("Failed to commit {} {} due to the following error:",
taskTypeName,
task.id(),
t);
if (firstException == null) {
firstException = t;
}
}
}
if (firstException != null) {
throw firstException;
}
return committed;
}
void shutdown(final boolean clean) {
final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null);
for (final T task: allTasks()) {
try {
closeTask(task, clean);
} catch (final TaskMigratedException e) {
log.info("Failed to close {} {} since it got migrated to another thread already. " +
"Closing it as zombie and move on.", taskTypeName, task.id());
tryCloseZombieTask(task);
} catch (final RuntimeException t) {
log.error("Failed while closing {} {} due to the following error:",
task.getClass().getSimpleName(),
task.id(),
t);
if (clean) {
closeUnclean(task);
}
firstException.compareAndSet(null, t);
}
}
clear();
final RuntimeException fatalException = firstException.get();
if (fatalException != null) {
throw fatalException;
}
}
void closeTask(final T task, final boolean clean) {
task.close(clean, false);
}
private void closeUnclean(final T task) {
log.info("Try to close {} {} unclean.", task.getClass().getSimpleName(), task.id());
try {
task.close(false, false);
} catch (final RuntimeException fatalException) {
log.error("Failed while closing {} {} due to the following error:",
task.getClass().getSimpleName(),
task.id(),
fatalException);
}
}
}

View File

@ -16,40 +16,45 @@
*/
package org.apache.kafka.streams.processor.internals;
import java.util.List;
import org.apache.kafka.common.TopicPartition;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
/**
* Performs bulk read operations from a set of partitions. Used to
* restore {@link org.apache.kafka.streams.processor.StateStore}s from their
* changelogs
* See {@link StoreChangelogReader}.
*/
public interface ChangelogReader {
public interface ChangelogReader extends ChangelogRegister {
/**
* Register a state store and it's partition for later restoration.
* @param restorer the state restorer to register
* Restore all registered state stores by reading from their changelogs
*/
void register(final StateRestorer restorer);
void restore();
/**
* Restore all registered state stores by reading from their changelogs.
* @return all topic partitions that have been restored
* Update offset limit of a given changelog partition
*/
Collection<TopicPartition> restore(final RestoringTasks active);
void updateLimitOffsets();
/**
* @return the restored offsets for all persistent stores.
* Transit to restore active changelogs mode
*/
Map<TopicPartition, Long> restoredOffsets();
void transitToRestoreActive();
/**
* Transit to update standby changelogs mode
*/
void transitToUpdateStandby();
/**
* @return the changelog partitions that have been completed restoring
*/
Set<TopicPartition> completedChangelogs();
/**
* Removes the passed in partitions from the set of changelogs
* @param revokedPartitions the set of partitions to remove
*/
void remove(List<TopicPartition> revokedPartitions);
void remove(Collection<TopicPartition> revokedPartitions);
/**
* Clear all partitions
@ -57,7 +62,7 @@ public interface ChangelogReader {
void clear();
/**
* @return whether the changelog reader has just been cleared or is uninitialized.
* @return whether the changelog reader has just been cleared or is uninitialized
*/
boolean isEmpty();
}

View File

@ -18,10 +18,15 @@ package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.common.TopicPartition;
import java.util.Map;
// Interface to indicate that an object has associated partition offsets that can be checkpointed
interface Checkpointable {
void checkpoint(final Map<TopicPartition, Long> offsets);
Map<TopicPartition, Long> checkpointed();
/**
* See {@link StoreChangelogReader}.
*/
interface ChangelogRegister {
/**
* Register a state store for restoration.
*
* @param partition the state store's changelog partition for restoring
* @param stateManager the state manager used for restoring (one per task)
*/
void register(final TopicPartition partition, final ProcessorStateManager stateManager);
}

View File

@ -41,7 +41,6 @@ import java.io.File;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -59,7 +58,6 @@ import static org.apache.kafka.streams.processor.internals.StateManagerUtil.conv
*/
public class GlobalStateManagerImpl implements GlobalStateManager {
private final Logger log;
private final boolean eosEnabled;
private final ProcessorTopology topology;
private final Consumer<byte[], byte[]> globalConsumer;
private final File baseDir;
@ -81,7 +79,6 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
final StateDirectory stateDirectory,
final StateRestoreListener stateRestoreListener,
final StreamsConfig config) {
eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
baseDir = stateDirectory.globalStateDir();
checkpointFile = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME));
checkpointFileCache = new HashMap<>();
@ -158,26 +155,6 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
return Collections.unmodifiableSet(globalStoreNames);
}
@Override
public void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext) {
StateManagerUtil.reinitializeStateStoresForPartitions(
log,
eosEnabled,
baseDir,
globalStores,
topology.storeToChangelogTopic(),
partitions,
processorContext,
checkpointFile,
checkpointFileCache
);
globalConsumer.assign(partitions);
globalConsumer.seekToBeginning(partitions);
}
@Override
public StateStore getGlobalStore(final String name) {
return globalStores.getOrDefault(name, Optional.empty()).orElse(null);
}
@ -191,9 +168,8 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
return baseDir;
}
public void register(final StateStore store,
final StateRestoreCallback stateRestoreCallback) {
@Override
public void registerStore(final StateStore store, final StateRestoreCallback stateRestoreCallback) {
if (globalStores.containsKey(store.name())) {
throw new IllegalArgumentException(String.format("Global Store %s has already been registered", store.name()));
}
@ -331,9 +307,9 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
log.warn("Restoring GlobalStore {} failed due to: {}. Deleting global store to recreate from scratch.",
storeName,
recoverableException.toString());
reinitializeStateStoresForPartitions(recoverableException.partitions(), globalProcessorContext);
stateRestoreListener.onRestoreStart(topicPartition, storeName, offset, highWatermark);
// TODO K9113: we remove the re-init logic and push it to be handled by the thread directly
restoreCount = 0L;
}
}
@ -365,7 +341,7 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
@Override
public void close(final boolean clean) throws IOException {
public void close() throws IOException {
try {
if (globalStores.isEmpty()) {
return;
@ -418,10 +394,9 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
}
}
@Override
public Map<TopicPartition, Long> checkpointed() {
public Map<TopicPartition, Long> changelogOffsets() {
return Collections.unmodifiableMap(checkpointFileCache);
}
}

View File

@ -81,7 +81,7 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
}
initTopology();
processorContext.initialize();
return stateMgr.checkpointed();
return stateMgr.changelogOffsets();
}
@SuppressWarnings("unchecked")
@ -114,7 +114,7 @@ public class GlobalStateUpdateTask implements GlobalStateMaintainer {
}
public void close() throws IOException {
stateMgr.close(true);
stateMgr.close();
}
private void initTopology() {

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.streams.StreamsConfig;
@ -1225,14 +1226,15 @@ public class InternalTopologyBuilder {
}
synchronized Pattern sourceTopicPattern() {
log.debug("Found pattern subscribed source topics, falling back to pattern subscription for the main consumer.");
if (topicPattern == null) {
final List<String> allSourceTopics = maybeDecorateInternalSourceTopics(sourceTopicNames);
Collections.sort(allSourceTopics);
topicPattern = buildPattern(allSourceTopics, nodeToSourcePatterns.values());
}
log.debug("Found pattern subscribed source topics, falling back to pattern " +
"subscription for the main consumer: {}", topicPattern);
return topicPattern;
}
@ -1861,16 +1863,40 @@ public class InternalTopologyBuilder {
return sb.toString();
}
Set<String> subscriptionUpdates() {
private Set<String> subscriptionUpdates() {
return Collections.unmodifiableSet(subscriptionUpdates);
}
boolean hasSubscriptionUpdates() {
private boolean hasSubscriptionUpdates() {
return !subscriptionUpdates.isEmpty();
}
synchronized void updateSubscribedTopics(final Set<String> topics,
final String logPrefix) {
synchronized void addSubscribedTopicsFromAssignment(final List<TopicPartition> partitions, final String logPrefix) {
if (sourceTopicPattern() != null) {
final Set<String> assignedTopics = new HashSet<>();
for (final TopicPartition topicPartition : partitions) {
assignedTopics.add(topicPartition.topic());
}
final Collection<String> existingTopics = subscriptionUpdates();
if (!existingTopics.containsAll(assignedTopics)) {
assignedTopics.addAll(existingTopics);
updateSubscribedTopics(assignedTopics, logPrefix);
}
}
}
synchronized void addSubscribedTopicsFromMetadata(final Set<String> topics, final String logPrefix) {
if (sourceTopicPattern() != null) {
final Collection<String> existingTopics = subscriptionUpdates();
if (!existingTopics.equals(topics)) {
topics.addAll(existingTopics);
updateSubscribedTopics(topics, logPrefix);
}
}
}
private void updateSubscribedTopics(final Set<String> topics, final String logPrefix) {
log.debug("{}found {} topics possibly matching subscription", logPrefix, topics.size());
subscriptionUpdates.clear();
subscriptionUpdates.addAll(topics);
@ -1880,7 +1906,6 @@ public class InternalTopologyBuilder {
}
// following functions are for test only
public synchronized Set<String> sourceTopicNames() {
return sourceTopicNames;
}

View File

@ -207,11 +207,11 @@ public class PartitionGroup {
public void close() {
clear();
partitionQueues.clear();
streamTime = RecordQueue.UNKNOWN;
}
public void clear() {
nonEmptyQueuesByTime.clear();
streamTime = RecordQueue.UNKNOWN;
for (final RecordQueue queue : partitionQueues.values()) {
queue.clear();
}

View File

@ -21,6 +21,6 @@ import org.apache.kafka.streams.processor.Punctuator;
public interface ProcessorNodePunctuator {
void punctuate(ProcessorNode node, long streamTime, PunctuationType type, Punctuator punctuator);
void punctuate(ProcessorNode node, long timestamp, PunctuationType type, Punctuator punctuator);
}

View File

@ -16,13 +16,14 @@
*/
package org.apache.kafka.streams.processor.internals;
import java.util.Collections;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.FixedOrderMap;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.internals.Task.TaskType;
import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.TaskId;
@ -32,262 +33,349 @@ import org.slf4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableList;
import static java.lang.String.format;
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.StateRestoreCallbackAdapter.adapt;
/**
* ProcessorStateManager is the source of truth for the current offset for each state store,
* which is either the read offset during restoring, or the written offset during normal processing.
*
* The offset is initialized as null when the state store is registered, and then it can be updated by
* loading checkpoint file, restore state stores, or passing from the record collector's written offsets.
*
* When checkpointing, if the offset is not null it would be written to the file.
*
* The manager is also responsible for restoring state stores via their registered restore callback,
* which is used for both updating standby tasks as well as restoring active tasks.
*/
public class ProcessorStateManager implements StateManager {
public static class StateStoreMetadata {
private final StateStore stateStore;
// corresponding changelog partition of the store, this and the following two fields
// will only be not-null if the state store is logged (i.e. changelog partition and restorer provided)
private final TopicPartition changelogPartition;
// could be used for both active restoration and standby
private final StateRestoreCallback restoreCallback;
// record converters used for restoration and standby
private final RecordConverter recordConverter;
// indicating the current snapshot of the store as the offset of last changelog record that has been
// applied to the store used for both restoration (active and standby tasks restored offset) and
// normal processing that update stores (written offset); could be null (when initialized)
//
// the offset is updated in three ways:
// 1. when loading from the checkpoint file, when the corresponding task has acquired the state
// directory lock and have registered all the state store; it is only one-time
// 2. when updating with restore records (by both restoring active and standby),
// update to the last restore record's offset
// 3. when checkpointing with the given written offsets from record collector,
// update blindly with the given offset
private Long offset;
private StateStoreMetadata(final StateStore stateStore) {
this.stateStore = stateStore;
this.restoreCallback = null;
this.recordConverter = null;
this.changelogPartition = null;
this.offset = null;
}
private StateStoreMetadata(final StateStore stateStore,
final TopicPartition changelogPartition,
final StateRestoreCallback restoreCallback,
final RecordConverter recordConverter) {
if (restoreCallback == null) {
throw new IllegalStateException("Log enabled store should always provide a restore callback upon registration");
}
this.stateStore = stateStore;
this.changelogPartition = changelogPartition;
this.restoreCallback = restoreCallback;
this.recordConverter = recordConverter;
this.offset = null;
}
private void setOffset(final Long offset) {
this.offset = offset;
}
// the offset is exposed to the changelog reader to determine if restoration is completed
Long offset() {
return this.offset;
}
TopicPartition changelogPartition() {
return this.changelogPartition;
}
StateStore store() {
return this.stateStore;
}
@Override
public String toString() {
return "StateStoreMetadata (" + stateStore.name() + " : " + changelogPartition + " @ " + offset;
}
}
private static final String STATE_CHANGELOG_TOPIC_SUFFIX = "-changelog";
private final Logger log;
private final TaskId taskId;
private final String logPrefix;
private final boolean isStandby;
private final ChangelogReader changelogReader;
private final Map<TopicPartition, Long> offsetLimits;
private final Map<TopicPartition, Long> standbyRestoredOffsets;
private final Map<String, StateRestoreCallback> restoreCallbacks; // used for standby tasks, keyed by state topic name
private final Map<String, RecordConverter> recordConverters; // used for standby tasks, keyed by state topic name
private final TaskType taskType;
private final ChangelogRegister changelogReader;
private final Map<String, String> storeToChangelogTopic;
private final Collection<TopicPartition> sourcePartitions;
// must be maintained in topological order
private final FixedOrderMap<String, Optional<StateStore>> registeredStores = new FixedOrderMap<>();
private final FixedOrderMap<String, Optional<StateStore>> globalStores = new FixedOrderMap<>();
private final FixedOrderMap<String, StateStoreMetadata> stores = new FixedOrderMap<>();
private final FixedOrderMap<String, StateStore> globalStores = new FixedOrderMap<>();
private final List<TopicPartition> changelogPartitions = new ArrayList<>();
// TODO: this map does not work with customized grouper where multiple partitions
// of the same topic can be assigned to the same task.
private final Map<String, TopicPartition> partitionForTopic;
private final boolean eosEnabled;
private final File baseDir;
private OffsetCheckpoint checkpointFile;
private final Map<TopicPartition, Long> checkpointFileCache = new HashMap<>();
private final Map<TopicPartition, Long> initialLoadedCheckpoints;
/**
* @throws ProcessorStateException if the task directory does not exist and could not be created
* @throws IOException if any severe error happens while creating or locking the state directory
*/
public ProcessorStateManager(final TaskId taskId,
final Collection<TopicPartition> sources,
final boolean isStandby,
final StateDirectory stateDirectory,
final Map<String, String> storeToChangelogTopic,
final ChangelogReader changelogReader,
final boolean eosEnabled,
final LogContext logContext) throws IOException {
this.eosEnabled = eosEnabled;
log = logContext.logger(ProcessorStateManager.class);
this.taskId = taskId;
this.changelogReader = changelogReader;
logPrefix = String.format("task [%s] ", taskId);
partitionForTopic = new HashMap<>();
for (final TopicPartition source : sources) {
partitionForTopic.put(source.topic(), source);
}
offsetLimits = new HashMap<>();
standbyRestoredOffsets = new ConcurrentHashMap<>();
this.isStandby = isStandby;
restoreCallbacks = isStandby ? new ConcurrentHashMap<>() : null;
recordConverters = isStandby ? new HashMap<>() : null;
this.storeToChangelogTopic = new HashMap<>(storeToChangelogTopic);
baseDir = stateDirectory.directoryForTask(taskId);
checkpointFile = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME));
initialLoadedCheckpoints = checkpointFile.read();
log.trace("Checkpointable offsets read from checkpoint: {}", initialLoadedCheckpoints);
if (eosEnabled) {
// with EOS enabled, there should never be a checkpoint file _during_ processing.
// delete the checkpoint file after loading its stored offsets.
checkpointFile.delete();
checkpointFile = null;
}
log.debug("Created state store manager for task {}", taskId);
}
private final OffsetCheckpoint checkpointFile;
public static String storeChangelogTopic(final String applicationId,
final String storeName) {
return applicationId + "-" + storeName + STATE_CHANGELOG_TOPIC_SUFFIX;
}
/**
* @throws ProcessorStateException if the task directory does not exist and could not be created
*/
public ProcessorStateManager(final TaskId taskId,
final Collection<TopicPartition> sources,
final TaskType taskType,
final StateDirectory stateDirectory,
final Map<String, String> storeToChangelogTopic,
final ChangelogRegister changelogReader,
final LogContext logContext) throws ProcessorStateException {
this.logPrefix = format("task [%s] ", taskId);
this.log = logContext.logger(ProcessorStateManager.class);
this.taskId = taskId;
this.taskType = taskType;
this.sourcePartitions = sources;
this.changelogReader = changelogReader;
this.storeToChangelogTopic = storeToChangelogTopic;
this.baseDir = stateDirectory.directoryForTask(taskId);
this.checkpointFile = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME));
log.debug("Created state store manager for task {}", taskId);
}
void registerGlobalStateStores(final List<StateStore> stateStores) {
log.debug("Register global stores {}", stateStores);
for (final StateStore stateStore : stateStores) {
globalStores.put(stateStore.name(), stateStore);
}
}
@Override
public StateStore getGlobalStore(final String name) {
return globalStores.get(name);
}
// package-private for test only
void initializeStoreOffsetsFromCheckpoint() {
try {
final Map<TopicPartition, Long> loadedCheckpoints = checkpointFile.read();
log.trace("Loaded offsets from the checkpoint file: {}", loadedCheckpoints);
for (final StateStoreMetadata store : stores.values()) {
if (store.changelogPartition == null) {
log.info("State store {} is not logged and hence would not be restored", store.stateStore.name());
} else {
if (loadedCheckpoints.containsKey(store.changelogPartition)) {
store.setOffset(loadedCheckpoints.remove(store.changelogPartition));
log.debug("State store {} initialized from checkpoint with offset {} at changelog {}",
store.stateStore.name(), store.offset, store.changelogPartition);
} else {
// TODO K9113: for EOS when there's no checkpointed offset, we should treat it as TaskCorrupted
log.info("State store {} did not find checkpoint offset, hence would " +
"default to the starting offset at changelog {}",
store.stateStore.name(), store.changelogPartition);
}
}
}
if (!loadedCheckpoints.isEmpty()) {
log.warn("Some loaded checkpoint offsets cannot find their corresponding state stores: {}", loadedCheckpoints);
}
checkpointFile.delete();
} catch (final IOException | RuntimeException e) {
// both IOException or runtime exception like number parsing can throw
throw new ProcessorStateException(format("%sError loading and deleting checkpoint file when creating the state manager",
logPrefix), e);
}
}
@Override
public File baseDir() {
return baseDir;
}
@Override
public void register(final StateStore store,
final StateRestoreCallback stateRestoreCallback) {
public void registerStore(final StateStore store, final StateRestoreCallback stateRestoreCallback) {
final String storeName = store.name();
log.debug("Registering state store {} to its state manager", storeName);
if (CHECKPOINT_FILE_NAME.equals(storeName)) {
throw new IllegalArgumentException(String.format("%sIllegal store name: %s", logPrefix, storeName));
throw new IllegalArgumentException(format("%sIllegal store name: %s, which collides with the pre-defined " +
"checkpoint file name", logPrefix, storeName));
}
if (registeredStores.containsKey(storeName) && registeredStores.get(storeName).isPresent()) {
throw new IllegalArgumentException(String.format("%sStore %s has already been registered.", logPrefix, storeName));
if (stores.containsKey(storeName)) {
throw new IllegalArgumentException(format("%sStore %s has already been registered.", logPrefix, storeName));
}
// check that the underlying change log topic exist or not
final String topic = storeToChangelogTopic.get(storeName);
// if the store name does not exist in the changelog map, it means the underlying store
// is not log enabled (including global stores), and hence it does not need to be restored
if (topic != null) {
final TopicPartition storePartition = new TopicPartition(topic, getPartition(topic));
// NOTE we assume the partition of the topic can always be inferred from the task id;
// if user ever use a default partition grouper (deprecated in KIP-528) this would break and
// it is not a regression (it would always break anyways)
final TopicPartition storePartition = new TopicPartition(topic, taskId.partition);
final StateStoreMetadata storeMetadata = new StateStoreMetadata(
store,
storePartition,
stateRestoreCallback,
converterForStore(store));
stores.put(storeName, storeMetadata);
final RecordConverter recordConverter = converterForStore(store);
if (isStandby) {
log.trace("Preparing standby replica of persistent state store {} with changelog topic {}", storeName, topic);
restoreCallbacks.put(topic, stateRestoreCallback);
recordConverters.put(topic, recordConverter);
} else {
final Long restoreCheckpoint = store.persistent() ? initialLoadedCheckpoints.get(storePartition) : null;
if (restoreCheckpoint != null) {
checkpointFileCache.put(storePartition, restoreCheckpoint);
}
log.trace("Restoring state store {} from changelog topic {} at checkpoint {}", storeName, topic, restoreCheckpoint);
final StateRestorer restorer = new StateRestorer(
storePartition,
new CompositeRestoreListener(stateRestoreCallback),
restoreCheckpoint,
offsetLimit(storePartition),
store.persistent(),
storeName,
recordConverter
);
changelogReader.register(restorer);
}
changelogPartitions.add(storePartition);
changelogReader.register(storePartition, this);
} else {
stores.put(storeName, new StateStoreMetadata(store));
}
registeredStores.put(storeName, Optional.of(store));
}
@Override
public void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext) {
StateManagerUtil.reinitializeStateStoresForPartitions(log,
eosEnabled,
baseDir,
registeredStores,
storeToChangelogTopic,
partitions,
processorContext,
checkpointFile,
checkpointFileCache
);
}
void clearCheckpoints() throws IOException {
if (checkpointFile != null) {
checkpointFile.delete();
checkpointFile = null;
checkpointFileCache.clear();
}
}
@Override
public Map<TopicPartition, Long> checkpointed() {
updateCheckpointFileCache(emptyMap());
final Map<TopicPartition, Long> partitionsAndOffsets = new HashMap<>();
for (final Map.Entry<String, StateRestoreCallback> entry : restoreCallbacks.entrySet()) {
final String topicName = entry.getKey();
final int partition = getPartition(topicName);
final TopicPartition storePartition = new TopicPartition(topicName, partition);
partitionsAndOffsets.put(storePartition, checkpointFileCache.getOrDefault(storePartition, -1L));
}
return partitionsAndOffsets;
}
void updateStandbyStates(final TopicPartition storePartition,
final List<ConsumerRecord<byte[], byte[]>> restoreRecords,
final long lastOffset) {
// restore states from changelog records
final RecordBatchingStateRestoreCallback restoreCallback = adapt(restoreCallbacks.get(storePartition.topic()));
if (!restoreRecords.isEmpty()) {
final RecordConverter converter = recordConverters.get(storePartition.topic());
final List<ConsumerRecord<byte[], byte[]>> convertedRecords = new ArrayList<>(restoreRecords.size());
for (final ConsumerRecord<byte[], byte[]> record : restoreRecords) {
convertedRecords.add(converter.convert(record));
}
try {
restoreCallback.restoreBatch(convertedRecords);
} catch (final RuntimeException e) {
throw new ProcessorStateException(String.format("%sException caught while trying to restore state from %s", logPrefix, storePartition), e);
}
}
// record the restored offset for its change log partition
standbyRestoredOffsets.put(storePartition, lastOffset + 1);
}
void putOffsetLimits(final Map<TopicPartition, Long> offsets) {
log.trace("Updating store offset limit with {}", offsets);
offsetLimits.putAll(offsets);
}
private long offsetLimit(final TopicPartition partition) {
final Long limit = offsetLimits.get(partition);
return limit != null ? limit : Long.MAX_VALUE;
}
ChangelogReader changelogReader() {
return changelogReader;
log.debug("Registered state store {} to its state manager", storeName);
}
@Override
public StateStore getStore(final String name) {
return registeredStores.getOrDefault(name, Optional.empty()).orElse(null);
if (stores.containsKey(name)) {
return stores.get(name).stateStore;
} else {
return null;
}
}
Collection<TopicPartition> changelogPartitions() {
return changelogOffsets().keySet();
}
@Override
public Map<TopicPartition, Long> changelogOffsets() {
// return the current offsets for those logged stores
final Map<TopicPartition, Long> changelogOffsets = new HashMap<>();
for (final StateStoreMetadata storeMetadata : stores.values()) {
if (storeMetadata.changelogPartition != null) {
// for changelog whose offset is unknown, use 0L indicating earliest offset
// otherwise return the current offset + 1 as the next offset to fetch
changelogOffsets.put(
storeMetadata.changelogPartition,
storeMetadata.offset == null ? 0L : storeMetadata.offset + 1L);
}
}
return changelogOffsets;
}
TaskId taskId() {
return taskId;
}
// used by the changelog reader only
boolean changelogAsSource(final TopicPartition partition) {
return sourcePartitions.contains(partition);
}
// used by the changelog reader only
TaskType taskType() {
return taskType;
}
// used by the changelog reader only
StateStoreMetadata storeMetadata(final TopicPartition partition) {
for (final StateStoreMetadata storeMetadata : stores.values()) {
if (partition.equals(storeMetadata.changelogPartition)) {
return storeMetadata;
}
}
return null;
}
// used by the changelog reader only
void restore(final StateStoreMetadata storeMetadata, final List<ConsumerRecord<byte[], byte[]>> restoreRecords) {
if (!stores.values().contains(storeMetadata)) {
throw new IllegalStateException("Restoring " + storeMetadata + " which is not registered in this state manager, " +
"this should not happen.");
}
if (!restoreRecords.isEmpty()) {
// restore states from changelog records and update the snapshot offset as the batch end record's offset
final Long batchEndOffset = restoreRecords.get(restoreRecords.size() - 1).offset();
final RecordBatchingStateRestoreCallback restoreCallback = adapt(storeMetadata.restoreCallback);
final List<ConsumerRecord<byte[], byte[]>> convertedRecords = restoreRecords.stream()
.map(storeMetadata.recordConverter::convert)
.collect(Collectors.toList());
try {
restoreCallback.restoreBatch(convertedRecords);
} catch (final RuntimeException e) {
throw new ProcessorStateException(
format("%sException caught while trying to restore state from %s", logPrefix, storeMetadata.changelogPartition),
e
);
}
storeMetadata.setOffset(batchEndOffset);
}
}
/**
* @throws TaskMigratedException recoverable error sending changelog records that would cause the task to be removed
* @throws StreamsException fatal error when flushing the state store, for example sending changelog records failed
* or flushing state store get IO errors; such error should cause the thread to die
*/
@Override
public void flush() {
ProcessorStateException firstException = null;
RuntimeException firstException = null;
// attempting to flush the stores
if (!registeredStores.isEmpty()) {
log.debug("Flushing all stores registered in the state manager");
for (final Map.Entry<String, Optional<StateStore>> entry : registeredStores.entrySet()) {
if (entry.getValue().isPresent()) {
final StateStore store = entry.getValue().get();
log.trace("Flushing store {}", store.name());
try {
store.flush();
} catch (final RuntimeException e) {
if (firstException == null) {
firstException = new ProcessorStateException(String.format("%sFailed to flush state store %s", logPrefix, store.name()), e);
}
log.error("Failed to flush state store {}: ", store.name(), e);
if (!stores.isEmpty()) {
log.debug("Flushing all stores registered in the state manager: {}", stores);
for (final Map.Entry<String, StateStoreMetadata> entry : stores.entrySet()) {
final StateStore store = entry.getValue().stateStore;
log.trace("Flushing store {}", store.name());
try {
store.flush();
} catch (final RuntimeException exception) {
if (firstException == null) {
// do NOT wrap the error if it is actually caused by Streams itself
if (exception instanceof StreamsException)
firstException = exception;
else
firstException = new ProcessorStateException(
format("%sFailed to flush state store %s", logPrefix, store.name()), exception);
}
} else {
throw new IllegalStateException("Expected " + entry.getKey() + " to have been initialized");
log.error("Failed to flush state store {}: ", store.name(), exception);
}
}
}
@ -304,162 +392,79 @@ public class ProcessorStateManager implements StateManager {
* @throws ProcessorStateException if any error happens when closing the state stores
*/
@Override
public void close(final boolean clean) throws ProcessorStateException {
ProcessorStateException firstException = null;
public void close() throws ProcessorStateException {
RuntimeException firstException = null;
// attempting to close the stores, just in case they
// are not closed by a ProcessorNode yet
if (!registeredStores.isEmpty()) {
log.debug("Closing its state manager and all the registered state stores");
for (final Map.Entry<String, Optional<StateStore>> entry : registeredStores.entrySet()) {
if (entry.getValue().isPresent()) {
final StateStore store = entry.getValue().get();
log.debug("Closing storage engine {}", store.name());
try {
store.close();
registeredStores.put(store.name(), Optional.empty());
} catch (final RuntimeException e) {
if (firstException == null) {
firstException = new ProcessorStateException(String.format("%sFailed to close state store %s", logPrefix, store.name()), e);
}
log.error("Failed to close state store {}: ", store.name(), e);
if (!stores.isEmpty()) {
log.debug("Closing its state manager and all the registered state stores: {}", stores);
for (final Map.Entry<String, StateStoreMetadata> entry : stores.entrySet()) {
final StateStore store = entry.getValue().stateStore;
log.trace("Closing store {}", store.name());
try {
store.close();
} catch (final RuntimeException exception) {
if (firstException == null) {
// do NOT wrap the error if it is actually caused by Streams itself
if (exception instanceof StreamsException)
firstException = exception;
else
firstException = new ProcessorStateException(
format("%sFailed to close state store %s", logPrefix, store.name()), exception);
}
} else {
log.info("Skipping to close non-initialized store {}", entry.getKey());
log.error("Failed to close state store {}: ", store.name(), exception);
}
}
}
if (!clean && eosEnabled) {
// delete the checkpoint file if this is an unclean close
try {
clearCheckpoints();
} catch (final IOException e) {
throw new ProcessorStateException(String.format("%sError while deleting the checkpoint file", logPrefix), e);
}
}
if (firstException != null) {
throw firstException;
}
}
@Override
public void checkpoint(final Map<TopicPartition, Long> checkpointableOffsetsFromProcessing) {
ensureStoresRegistered();
public void checkpoint(final Map<TopicPartition, Long> writtenOffsets) {
// first update each state store's current offset, then checkpoint
// those stores that are only logged and persistent to the checkpoint file
for (final Map.Entry<TopicPartition, Long> entry : writtenOffsets.entrySet()) {
final StateStoreMetadata store = findStore(entry.getKey());
// write the checkpoint file before closing
if (checkpointFile == null) {
checkpointFile = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME));
if (store != null) {
store.setOffset(entry.getValue());
log.debug("State store {} updated to written offset {} at changelog {}",
store.stateStore.name(), store.offset, store.changelogPartition);
}
}
updateCheckpointFileCache(checkpointableOffsetsFromProcessing);
final Map<TopicPartition, Long> checkpointingOffsets = new HashMap<>();
for (final StateStoreMetadata storeMetadata : stores.values()) {
// store is logged, persistent, and has a valid current offset
if (storeMetadata.changelogPartition != null &&
storeMetadata.stateStore.persistent() &&
storeMetadata.offset != null) {
checkpointingOffsets.put(storeMetadata.changelogPartition, storeMetadata.offset);
}
}
log.debug("Writing checkpoint: {}", checkpointFileCache);
log.debug("Writing checkpoint: {}", checkpointingOffsets);
try {
checkpointFile.write(checkpointFileCache);
checkpointFile.write(checkpointingOffsets);
} catch (final IOException e) {
log.warn("Failed to write offset checkpoint file to [{}]", checkpointFile, e);
}
}
private void updateCheckpointFileCache(final Map<TopicPartition, Long> checkpointableOffsetsFromProcessing) {
final Set<TopicPartition> validCheckpointableTopics = validCheckpointableTopics();
final Map<TopicPartition, Long> restoredOffsets = validCheckpointableOffsets(
changelogReader.restoredOffsets(),
validCheckpointableTopics
);
log.trace("Checkpointable offsets updated with restored offsets: {}", checkpointFileCache);
for (final TopicPartition topicPartition : validCheckpointableTopics) {
if (checkpointableOffsetsFromProcessing.containsKey(topicPartition)) {
// if we have just recently processed some offsets,
// store the last offset + 1 (the log position after restoration)
checkpointFileCache.put(topicPartition, checkpointableOffsetsFromProcessing.get(topicPartition) + 1);
} else if (standbyRestoredOffsets.containsKey(topicPartition)) {
// or if we restored some offset as a standby task, use it
checkpointFileCache.put(topicPartition, standbyRestoredOffsets.get(topicPartition));
} else if (restoredOffsets.containsKey(topicPartition)) {
// or if we restored some offset as an active task, use it
checkpointFileCache.put(topicPartition, restoredOffsets.get(topicPartition));
} else if (checkpointFileCache.containsKey(topicPartition)) {
// or if we have a prior value we've cached (and written to the checkpoint file), then keep it
} else {
// As a last resort, fall back to the offset we loaded from the checkpoint file at startup, but
// only if the offset is actually valid for our current state stores.
final Long loadedOffset =
validCheckpointableOffsets(initialLoadedCheckpoints, validCheckpointableTopics).get(topicPartition);
if (loadedOffset != null) {
checkpointFileCache.put(topicPartition, loadedOffset);
}
}
}
}
private StateStoreMetadata findStore(final TopicPartition changelogPartition) {
final List<StateStoreMetadata> found = stores.values().stream()
.filter(metadata -> changelogPartition.equals(metadata.changelogPartition))
.collect(Collectors.toList());
private int getPartition(final String topic) {
final TopicPartition partition = partitionForTopic.get(topic);
return partition == null ? taskId.partition : partition.partition();
}
void registerGlobalStateStores(final List<StateStore> stateStores) {
log.debug("Register global stores {}", stateStores);
for (final StateStore stateStore : stateStores) {
globalStores.put(stateStore.name(), Optional.of(stateStore));
}
}
@Override
public StateStore getGlobalStore(final String name) {
return globalStores.getOrDefault(name, Optional.empty()).orElse(null);
}
Collection<TopicPartition> changelogPartitions() {
return unmodifiableList(changelogPartitions);
}
void ensureStoresRegistered() {
for (final Map.Entry<String, Optional<StateStore>> entry : registeredStores.entrySet()) {
if (!entry.getValue().isPresent()) {
throw new IllegalStateException(
"store [" + entry.getKey() + "] has not been correctly registered. This is a bug in Kafka Streams."
);
}
}
}
private Set<TopicPartition> validCheckpointableTopics() {
// it's only valid to record checkpoints for registered stores that are both persistent and change-logged
final Set<TopicPartition> result = new HashSet<>(storeToChangelogTopic.size());
for (final Map.Entry<String, String> storeToChangelog : storeToChangelogTopic.entrySet()) {
final String storeName = storeToChangelog.getKey();
if (registeredStores.containsKey(storeName)
&& registeredStores.get(storeName).isPresent()
&& registeredStores.get(storeName).get().persistent()) {
final String changelogTopic = storeToChangelog.getValue();
result.add(new TopicPartition(changelogTopic, getPartition(changelogTopic)));
}
}
return result;
}
private static Map<TopicPartition, Long> validCheckpointableOffsets(
final Map<TopicPartition, Long> checkpointableOffsets,
final Set<TopicPartition> validCheckpointableTopics) {
final Map<TopicPartition, Long> result = new HashMap<>(checkpointableOffsets.size());
for (final Map.Entry<TopicPartition, Long> topicToCheckpointableOffset : checkpointableOffsets.entrySet()) {
final TopicPartition topic = topicToCheckpointableOffset.getKey();
if (validCheckpointableTopics.contains(topic)) {
final Long checkpointableOffset = topicToCheckpointableOffset.getValue();
result.put(topic, checkpointableOffset);
}
if (found.size() > 1) {
throw new IllegalStateException("Multiple state stores are found for changelog partition " + changelogPartition +
", this should never happen: " + found);
}
return result;
}
Map<TopicPartition, Long> standbyRestoredOffsets() {
return Collections.unmodifiableMap(standbyRestoredOffsets);
return found.isEmpty() ? null : found.get(0);
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
@ -44,11 +45,7 @@ public interface RecordCollector extends AutoCloseable {
final Serializer<V> valueSerializer,
final StreamPartitioner<? super K, ? super V> partitioner);
/**
* Initialize the collector with a producer.
* @param producer the producer that should be used by this collector
*/
void init(final Producer<byte[], byte[]> producer);
void commit(final Map<TopicPartition, OffsetAndMetadata> offsets);
/**
* Flush the internal {@link Producer}.
@ -70,6 +67,8 @@ public interface RecordCollector extends AutoCloseable {
/**
* A supplier of a {@link RecordCollectorImpl} instance.
*/
// TODO: after we have done KAFKA-9088 we should just add this function
// to InternalProcessorContext interface
interface Supplier {
/**
* Get the record collector.

View File

@ -16,10 +16,11 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.consumer.CommitFailedException;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
@ -27,6 +28,7 @@ import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.SecurityDisabledException;
@ -38,10 +40,15 @@ import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.ProductionExceptionHandler;
import org.apache.kafka.streams.errors.ProductionExceptionHandler.ProductionExceptionHandlerResponse;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
import org.slf4j.Logger;
import java.util.Collections;
@ -50,65 +57,132 @@ import java.util.List;
import java.util.Map;
public class RecordCollectorImpl implements RecordCollector {
private final Logger log;
private final String logPrefix;
private final Sensor droppedRecordsSensor;
private Producer<byte[], byte[]> producer;
private final Map<TopicPartition, Long> offsets;
private final ProductionExceptionHandler productionExceptionHandler;
private final static String LOG_MESSAGE = "Error sending record to topic {} due to {}; " +
"No more records will be sent and no more offsets will be recorded for this task. " +
"Enable TRACE logging to view failed record key and value.";
private final static String EXCEPTION_MESSAGE = "%sAbort sending since %s with a previous record (timestamp %d) to topic %s due to %s";
private final static String PARAMETER_HINT = "\nYou can increase the producer configs `delivery.timeout.ms` and/or " +
"`retries` to avoid this error. Note that `retries` is set to infinite by default.";
private final static String TIMEOUT_HINT_TEMPLATE = "%nTimeout exception caught when sending record to topic %s. " +
"This might happen if the producer cannot send data to the Kafka cluster and thus, " +
"its internal buffer fills up. " +
"This can also happen if the broker is slow to respond, if the network connection to " +
"the broker was interrupted, or if similar circumstances arise. " +
"You can increase producer parameter `max.block.ms` to increase this timeout.";
private final static String SEND_EXCEPTION_MESSAGE = "Error encountered sending record to topic %s for task %s due to:%n%s";
private final Logger log;
private final TaskId taskId;
private final boolean eosEnabled;
private final String applicationId;
private final Sensor droppedRecordsSensor;
private final Map<TopicPartition, Long> offsets;
private final Consumer<byte[], byte[]> consumer;
private final ProductionExceptionHandler productionExceptionHandler;
// used when eosEnabled is true only
private boolean transactionInFlight = false;
private Producer<byte[], byte[]> producer;
private volatile KafkaException sendException;
public RecordCollectorImpl(final String streamTaskId,
/**
* @throws StreamsException fatal error that should cause the thread to die (from producer.initTxn)
*/
public RecordCollectorImpl(final TaskId taskId,
final StreamsConfig config,
final LogContext logContext,
final ProductionExceptionHandler productionExceptionHandler,
final Sensor droppedRecordsSensor) {
final StreamsMetricsImpl streamsMetrics,
final Consumer<byte[], byte[]> consumer,
final StreamThread.ProducerSupplier producerSupplier) {
this.taskId = taskId;
this.consumer = consumer;
this.offsets = new HashMap<>();
this.logPrefix = String.format("task [%s] ", streamTaskId);
this.log = logContext.logger(getClass());
this.productionExceptionHandler = productionExceptionHandler;
this.droppedRecordsSensor = droppedRecordsSensor;
this.applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
this.productionExceptionHandler = config.defaultProductionExceptionHandler();
this.eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
final String threadId = Thread.currentThread().getName();
this.droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, taskId.toString(), streamsMetrics);
producer = producerSupplier.get(taskId);
maybeInitTxns();
}
@Override
public void init(final Producer<byte[], byte[]> producer) {
this.producer = producer;
private void maybeInitTxns() {
if (eosEnabled) {
// initialize transactions if eos is turned on, which will block if the previous transaction has not
// completed yet; do not start the first transaction until the topology has been initialized later
try {
producer.initTransactions();
} catch (final TimeoutException exception) {
final String errorMessage = "Timeout exception caught when initializing transactions for task " + taskId + ". " +
"\nThe broker is either slow or in bad state (like not having enough replicas) in responding to the request, " +
"or the connection to broker was interrupted sending the request or receiving the response. " +
"\n Consider overwriting `max.block.ms` to a larger value to avoid timeout errors";
// TODO K9113: we do NOT try to handle timeout exception here but throw it as a fatal error
throw new StreamsException(errorMessage, exception);
} catch (final KafkaException exception) {
throw new StreamsException("Error encountered while initializing transactions for task " + taskId, exception);
}
}
}
@Override
public <K, V> void send(final String topic,
final K key,
final V value,
final Headers headers,
final Long timestamp,
final Serializer<K> keySerializer,
final Serializer<V> valueSerializer,
final StreamPartitioner<? super K, ? super V> partitioner) {
Integer partition = null;
private void maybeBeginTxn() {
if (eosEnabled && !transactionInFlight) {
try {
producer.beginTransaction();
} catch (final ProducerFencedException error) {
throw new TaskMigratedException(taskId, "Producer get fenced trying to begin a new transaction", error);
} catch (final KafkaException error) {
throw new StreamsException("Producer encounter unexpected error trying to begin a new transaction", error);
}
transactionInFlight = true;
}
}
if (partitioner != null) {
final List<PartitionInfo> partitions = producer.partitionsFor(topic);
if (partitions.size() > 0) {
partition = partitioner.partition(topic, key, value, partitions.size());
} else {
throw new StreamsException("Could not get partition information for topic '" + topic + "'." +
" This can happen if the topic does not exist.");
private void maybeAbortTxn() {
if (eosEnabled && transactionInFlight) {
try {
producer.abortTransaction();
} catch (final ProducerFencedException ignore) {
/* TODO
* this should actually never happen atm as we guard the call to #abortTransaction
* -> the reason for the guard is a "bug" in the Producer -- it throws IllegalStateException
* instead of ProducerFencedException atm. We can remove the isZombie flag after KAFKA-5604 got
* fixed and fall-back to this catch-and-swallow code
*/
// can be ignored: transaction got already aborted by brokers/transactional-coordinator if this happens
} catch (final KafkaException error) {
throw new StreamsException("Producer encounter unexpected error trying to abort the transaction", error);
}
transactionInFlight = false;
}
}
public void commit(final Map<TopicPartition, OffsetAndMetadata> offsets) {
if (eosEnabled) {
maybeBeginTxn();
try {
producer.sendOffsetsToTransaction(offsets, applicationId);
producer.commitTransaction();
transactionInFlight = false;
} catch (final ProducerFencedException error) {
throw new TaskMigratedException(taskId, "Producer get fenced trying to commit a transaction", error);
} catch (final TimeoutException error) {
// TODO K9113: currently handle timeout exception as a fatal error, should discuss whether we want to handle it
throw new StreamsException("Timed out while committing transaction via producer for task " + taskId, error);
} catch (final KafkaException error) {
throw new StreamsException("Error encountered sending offsets and committing transaction " +
"via producer for task " + taskId, error);
}
} else {
try {
consumer.commitSync(offsets);
} catch (final CommitFailedException error) {
throw new TaskMigratedException(taskId, "Consumer committing offsets failed, " +
"indicating the corresponding thread is no longer part of the group.", error);
} catch (final TimeoutException error) {
// TODO K9113: currently handle timeout exception as a fatal error
throw new StreamsException("Timed out while committing offsets via consumer for task " + taskId, error);
} catch (final KafkaException error) {
throw new StreamsException("Error encountered committing offsets via consumer for task " + taskId, error);
}
}
send(topic, key, value, headers, partition, timestamp, keySerializer, valueSerializer);
}
private boolean productionExceptionIsFatal(final Exception exception) {
@ -125,41 +199,65 @@ public class RecordCollectorImpl implements RecordCollector {
return securityException || communicationException;
}
private <K, V> void recordSendError(
final K key,
final V value,
final Long timestamp,
final String topic,
final Exception exception
) {
String errorLogMessage = LOG_MESSAGE;
String errorMessage = EXCEPTION_MESSAGE;
private void recordSendError(final String topic, final Exception exception, final ProducerRecord<byte[], byte[]> serializedRecord) {
String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, exception.toString());
if (exception instanceof TimeoutException) {
final String topicTimeoutHint = String.format(TIMEOUT_HINT_TEMPLATE, topic);
errorLogMessage += topicTimeoutHint;
errorMessage += topicTimeoutHint;
} else if (exception instanceof RetriableException) {
// There is no documented API for detecting retriable errors, so we rely on `RetriableException`
// even though it's an implementation detail (i.e. we do the best we can given what's available)
errorLogMessage += PARAMETER_HINT;
errorMessage += PARAMETER_HINT;
if (productionExceptionIsFatal(exception)) {
errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error.";
sendException = new StreamsException(errorMessage, exception);
} else if (exception instanceof ProducerFencedException || exception instanceof OutOfOrderSequenceException) {
errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced, " +
"indicating the task may be migrated out.";
sendException = new TaskMigratedException(taskId, errorMessage, exception);
} else {
if (exception instanceof RetriableException) {
errorMessage += "\nThe broker is either slow or in bad state (like not having enough replicas) in responding the request, " +
"or the connection to broker was interrupted sending the request or receiving the response. " +
"\nConsider overwriting `max.block.ms` and /or " +
"`delivery.timeout.ms` to a larger value to wait longer for such scenarios and avoid timeout errors";
}
if (productionExceptionHandler.handle(serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) {
errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent.";
sendException = new StreamsException(errorMessage, exception);
} else {
errorMessage += "\nException handler choose to CONTINUE processing in spite of this error but written offsets would not be recorded.";
droppedRecordsSensor.record();
}
}
log.error(errorLogMessage, topic, exception.getMessage(), exception);
// KAFKA-7510 put message key and value in TRACE level log so we don't leak data by default
log.trace("Failed message: key {} value {} timestamp {}", key, value, timestamp);
log.error(errorMessage);
}
sendException = new StreamsException(
String.format(
errorMessage,
logPrefix,
"an error caught",
timestamp,
topic,
exception.toString()
),
exception);
/**
* @throws StreamsException fatal error that should cause the thread to die
* @throws TaskMigratedException recoverable error that would cause the task to be removed
*/
@Override
public <K, V> void send(final String topic,
final K key,
final V value,
final Headers headers,
final Long timestamp,
final Serializer<K> keySerializer,
final Serializer<V> valueSerializer,
final StreamPartitioner<? super K, ? super V> partitioner) {
final Integer partition;
// TODO K9113: we need to decide how to handle exceptions from partitionsFor
if (partitioner != null) {
final List<PartitionInfo> partitions = producer.partitionsFor(topic);
if (partitions.size() > 0) {
partition = partitioner.partition(topic, key, value, partitions.size());
} else {
throw new StreamsException("Could not get partition information for topic '" + topic + "' for task " + taskId +
". This can happen if the topic does not exist.");
}
} else {
partition = null;
}
send(topic, key, value, headers, partition, timestamp, keySerializer, valueSerializer);
}
@Override
@ -172,85 +270,44 @@ public class RecordCollectorImpl implements RecordCollector {
final Serializer<K> keySerializer,
final Serializer<V> valueSerializer) {
checkForException();
final byte[] keyBytes = keySerializer.serialize(topic, headers, key);
final byte[] valBytes = valueSerializer.serialize(topic, headers, value);
final ProducerRecord<byte[], byte[]> serializedRecord = new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes, headers);
maybeBeginTxn();
try {
producer.send(serializedRecord, new Callback() {
@Override
public void onCompletion(final RecordMetadata metadata,
final Exception exception) {
if (exception == null) {
if (sendException != null) {
return;
}
final TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition());
offsets.put(tp, metadata.offset());
} else {
if (sendException == null) {
if (exception instanceof ProducerFencedException || exception instanceof UnknownProducerIdException) {
log.warn(LOG_MESSAGE, topic, exception.getMessage(), exception);
final byte[] keyBytes = keySerializer.serialize(topic, headers, key);
final byte[] valBytes = valueSerializer.serialize(topic, headers, value);
// KAFKA-7510 put message key and value in TRACE level log so we don't leak data by default
log.trace("Failed message: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition);
final ProducerRecord<byte[], byte[]> serializedRecord = new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes, headers);
sendException = new RecoverableClientException(
String.format(
EXCEPTION_MESSAGE,
logPrefix,
"producer got fenced",
timestamp,
topic,
exception.toString()
),
exception
);
} else {
if (productionExceptionIsFatal(exception)) {
recordSendError(key, value, timestamp, topic, exception);
} else if (productionExceptionHandler.handle(serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) {
recordSendError(key, value, timestamp, topic, exception);
} else {
log.warn(
"Error sending records topic=[{}] and partition=[{}]; " +
"The exception handler chose to CONTINUE processing in spite of this error. " +
"Enable TRACE logging to view failed messages key and value.",
topic, partition, exception
);
producer.send(serializedRecord, (metadata, exception) -> {
// if there's already an exception record, skip logging offsets or new exceptions
if (sendException != null) {
return;
}
// KAFKA-7510 put message key and value in TRACE level log so we don't leak data by default
log.trace("Failed message: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition);
if (exception == null) {
final TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition());
offsets.put(tp, metadata.offset());
} else {
recordSendError(topic, exception, serializedRecord);
droppedRecordsSensor.record();
}
}
}
}
// KAFKA-7510 only put message key and value in TRACE level log so we don't leak data by default
log.trace("Failed record: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition);
}
});
} catch (final RuntimeException uncaughtException) {
if (isRecoverable(uncaughtException)) {
// producer.send() call may throw a KafkaException which wraps a FencedException,
// in this case we should throw its wrapped inner cause so that it can be captured and re-wrapped as TaskMigrationException
throw new RecoverableClientException("Caught a recoverable exception", uncaughtException);
throw new TaskMigratedException(taskId, "Producer cannot send records anymore since it got fenced", uncaughtException.getCause());
} else {
throw new StreamsException(
String.format(
EXCEPTION_MESSAGE,
logPrefix,
"an error caught",
timestamp,
topic,
uncaughtException.toString()
),
uncaughtException);
final String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, uncaughtException.toString());
throw new StreamsException(errorMessage, uncaughtException);
}
}
}
public static boolean isRecoverable(final RuntimeException uncaughtException) {
private static boolean isRecoverable(final RuntimeException uncaughtException) {
return uncaughtException instanceof KafkaException && (
uncaughtException.getCause() instanceof ProducerFencedException ||
uncaughtException.getCause() instanceof UnknownProducerIdException);
@ -262,34 +319,42 @@ public class RecordCollectorImpl implements RecordCollector {
}
}
/**
* @throws StreamsException fatal error that should cause the thread to die
* @throws TaskMigratedException recoverable error that would cause the task to be removed
*/
@Override
public void flush() {
log.debug("Flushing producer");
try {
producer.flush();
} catch (final ProducerFencedException | UnknownProducerIdException e) {
throw new RecoverableClientException("Caught a recoverable exception while flushing", e);
}
log.debug("Flushing record collector");
producer.flush();
checkForException();
}
/**
* @throws StreamsException fatal error that should cause the thread to die
* @throws TaskMigratedException recoverable error that would cause the task to be removed
*/
@Override
public void close() {
log.debug("Closing producer");
if (producer != null) {
log.debug("Closing record collector");
maybeAbortTxn();
if (eosEnabled) {
try {
producer.close();
} catch (final ProducerFencedException | UnknownProducerIdException e) {
throw new RecoverableClientException("Caught a recoverable exception while closing", e);
} catch (final KafkaException e) {
throw new StreamsException("Caught a recoverable exception while closing", e);
}
producer = null;
}
checkForException();
}
@Override
public Map<TopicPartition, Long> offsets() {
return Collections.unmodifiableMap(offsets);
return Collections.unmodifiableMap(new HashMap<>(offsets));
}
// for testing only

View File

@ -28,10 +28,10 @@ import org.slf4j.Logger;
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG;
class RecordDeserializer {
private final SourceNode sourceNode;
private final DeserializationExceptionHandler deserializationExceptionHandler;
private final Logger log;
private final SourceNode sourceNode;
private final Sensor droppedRecordsSensor;
private final DeserializationExceptionHandler deserializationExceptionHandler;
RecordDeserializer(final SourceNode sourceNode,
final DeserializationExceptionHandler deserializationExceptionHandler,

View File

@ -157,12 +157,11 @@ public class RecordQueue {
}
/**
* Clear the fifo queue of its elements, also clear the time tracker's kept stamped elements
* Clear the fifo queue of its elements
*/
public void clear() {
fifoQueue.clear();
headRecord = null;
partitionTime = RecordQueue.UNKNOWN;
}
private void updateHead() {

View File

@ -1,33 +0,0 @@
/*
* 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.processor.internals;
import org.apache.kafka.common.KafkaException;
/**
* Denotes an exception that is recoverable by re-creating the client (ie, the client is no longer in a valid state),
* as opposed to retriable (the failure was transient, so the same client can be used again later),
* or fatal (the request was actually invalid, so retrying or recovering would not help)
*
* This class also serves the dual purpose of capturing the stack trace as early as possible,
* at the site of the Producer call, since the exeptions that cause this don't record stack traces.
*/
public class RecoverableClientException extends KafkaException {
public RecoverableClientException(final String message, final Throwable cause) {
super(message, cause);
}
}

View File

@ -16,232 +16,223 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsMetrics;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics;
import org.slf4j.Logger;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* A StandbyTask
*/
public class StandbyTask extends AbstractTask {
private boolean updateOffsetLimits;
public class StandbyTask extends AbstractTask implements Task {
private final Logger log;
private final String logPrefix;
private final Sensor closeTaskSensor;
private final Map<TopicPartition, Long> offsetLimits = new HashMap<>();
private final InternalProcessorContext processorContext;
/**
* Create {@link StandbyTask} with its assigned partitions
*
* @param id the ID of this task
* @param partitions the collection of assigned {@link TopicPartition}
* @param partitions input topic partitions, used for thread metadata only
* @param topology the instance of {@link ProcessorTopology}
* @param consumer the instance of {@link Consumer}
* @param config the {@link StreamsConfig} specified by the user
* @param metrics the {@link StreamsMetrics} created by the thread
* @param stateMgr the {@link ProcessorStateManager} for this task
* @param stateDirectory the {@link StateDirectory} created by the thread
*/
StandbyTask(final TaskId id,
final Set<TopicPartition> partitions,
final ProcessorTopology topology,
final Consumer<byte[], byte[]> consumer,
final ChangelogReader changelogReader,
final StreamsConfig config,
final StreamsMetricsImpl metrics,
final ProcessorStateManager stateMgr,
final StateDirectory stateDirectory) {
super(id, partitions, topology, consumer, changelogReader, true, stateDirectory, config);
super(id, topology, stateDirectory, stateMgr, partitions);
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
logPrefix = threadIdPrefix + String.format("%s [%s] ", "standby-task", id);
final LogContext logContext = new LogContext(logPrefix);
log = logContext.logger(getClass());
closeTaskSensor = ThreadMetrics.closeTaskSensor(Thread.currentThread().getName(), metrics);
processorContext = new StandbyContextImpl(id, config, stateMgr, metrics);
final Set<String> changelogTopicNames = new HashSet<>(topology.storeToChangelogTopic().values());
partitions.stream()
.filter(tp -> changelogTopicNames.contains(tp.topic()))
.forEach(tp -> offsetLimits.put(tp, 0L));
updateOffsetLimits = true;
closeTaskSensor = ThreadMetrics.closeTaskSensor(Thread.currentThread().getName(), metrics);
}
@Override
public void initializeMetadata() {}
@Override
public boolean initializeStateStores() {
log.trace("Initializing state stores");
registerStateStores();
processorContext.initialize();
taskInitialized = true;
return true;
public boolean isActive() {
return false;
}
@Override
public void initializeTopology() {}
/**
* <pre>
* - update offset limits
* </pre>
* @throws StreamsException fatal error, should close the thread
*/
@Override
public void initializeIfNeeded() {
if (state() == State.CREATED) {
StateManagerUtil.registerStateStores(log, logPrefix, topology, stateMgr, stateDirectory, processorContext);
// no topology needs initialized, we can transit to RUNNING
// right after registered the stores
transitionTo(State.RESTORING);
transitionTo(State.RUNNING);
processorContext.initialize();
log.info("Initialized");
}
}
@Override
public void completeRestoration() {
throw new IllegalStateException("Standby task " + id + " should never be completing restoration");
}
@Override
public void suspend() {
log.trace("No-op suspend with state {}", state());
}
@Override
public void resume() {
log.debug("Resuming");
allowUpdateOfOffsetLimit();
log.trace("No-op resume with state {}", state());
}
/**
* <pre>
* - flush store
* - checkpoint store
* - update offset limits
* </pre>
* 1. flush store
* 2. write checkpoint file
*
* @throws TaskMigratedException all the task has been migrated
* @throws StreamsException fatal error, should close the thread
*/
@Override
public void commit() {
log.trace("Committing");
flushAndCheckpointState();
allowUpdateOfOffsetLimit();
commitNeeded = false;
switch (state()) {
case RUNNING:
stateMgr.flush();
// since there's no written offsets we can checkpoint with empty map,
// and the state current offset would be used to checkpoint
stateMgr.checkpoint(Collections.emptyMap());
log.info("Committed");
break;
case CLOSING:
// do nothing and also not throw
log.trace("Skip committing since task is closing");
break;
default:
throw new IllegalStateException("Illegal state " + state() + " while committing standby task " + id);
}
}
private void flushAndCheckpointState() {
// this could theoretically throw a ProcessorStateException caused by a ProducerFencedException,
// but in practice this shouldn't happen for standby tasks, since they don't produce to changelog topics
// or downstream topics.
stateMgr.flush();
stateMgr.checkpoint(Collections.emptyMap());
@Override
public void closeClean() {
close(true);
log.info("Closed clean");
}
@Override
public void closeDirty() {
close(false);
log.info("Closed dirty");
}
/**
* <pre>
* - {@link #commit()}
* - close state
* <pre>
* @param isZombie ignored by {@code StandbyTask} as it can never be a zombie
* 1. commit if we are running and clean close;
* 2. close the state manager.
*
* @throws TaskMigratedException all the task has been migrated
* @throws StreamsException fatal error, should close the thread
*/
private void close(final boolean clean) {
if (state() == State.CREATED) {
// the task is created and not initialized, do nothing
transitionTo(State.CLOSING);
} else {
if (state() == State.RUNNING) {
if (clean)
commit();
transitionTo(State.CLOSING);
}
if (state() == State.CLOSING) {
StateManagerUtil.closeStateManager(log, logPrefix, clean, stateMgr, stateDirectory);
// TODO: if EOS is enabled, we should wipe out the state stores like we did for StreamTask too
} else {
throw new IllegalStateException("Illegal state " + state() + " while closing standby task " + id);
}
}
closeTaskSensor.record();
transitionTo(State.CLOSED);
}
@Override
public void addRecords(final TopicPartition partition, final Iterable<ConsumerRecord<byte[], byte[]>> records) {
throw new IllegalStateException("Attempted to add records to task " + id() + " for invalid input partition " + partition);
}
/**
* Produces a string representation containing useful information about a Task.
* This is useful in debugging scenarios.
*
* @return A string representation of the StreamTask instance.
*/
@Override
public void close(final boolean clean,
final boolean isZombie) {
closeTaskSensor.record();
if (!taskInitialized) {
return;
}
log.debug("Closing");
try {
if (clean) {
commit();
}
} finally {
closeStateManager(true);
}
taskClosed = true;
public String toString() {
return toString("");
}
/**
* Updates a state store using records from one change log partition
* Produces a string representation containing useful information about a Task starting with the given indent.
* This is useful in debugging scenarios.
*
* @return a list of records not consumed
* @return A string representation of the Task instance.
*/
public List<ConsumerRecord<byte[], byte[]>> update(final TopicPartition partition,
final List<ConsumerRecord<byte[], byte[]>> records) {
if (records.isEmpty()) {
return Collections.emptyList();
public String toString(final String indent) {
final StringBuilder sb = new StringBuilder();
sb.append(indent);
sb.append("TaskId: ");
sb.append(id);
sb.append("\n");
// print topology
if (topology != null) {
sb.append(indent).append(topology.toString(indent + "\t"));
}
log.trace("Updating standby replicas of its state store for partition [{}]", partition);
long limit = offsetLimits.getOrDefault(partition, Long.MAX_VALUE);
long lastOffset = -1L;
final List<ConsumerRecord<byte[], byte[]>> restoreRecords = new ArrayList<>(records.size());
final List<ConsumerRecord<byte[], byte[]>> remainingRecords = new ArrayList<>();
for (final ConsumerRecord<byte[], byte[]> record : records) {
// Check if we're unable to process records due to an offset limit (e.g. when our
// partition is both a source and a changelog). If we're limited then try to refresh
// the offset limit if possible.
if (record.offset() >= limit && updateOffsetLimits) {
limit = updateOffsetLimits(partition);
}
if (record.offset() < limit) {
restoreRecords.add(record);
lastOffset = record.offset();
} else {
remainingRecords.add(record);
}
}
if (!restoreRecords.isEmpty()) {
stateMgr.updateStandbyStates(partition, restoreRecords, lastOffset);
commitNeeded = true;
}
return remainingRecords;
return sb.toString();
}
Map<TopicPartition, Long> checkpointedOffsets() {
return Collections.unmodifiableMap(stateMgr.checkpointed());
public boolean commitNeeded() {
return false;
}
public Map<TopicPartition, Long> changelogPositions() {
// this maintains the most upto date value of the latest offset for a record consumed off
// the changelog topic, that is also within the offsetLimit tracked.
return stateMgr.standbyRestoredOffsets();
public Collection<TopicPartition> changelogPartitions() {
return stateMgr.changelogPartitions();
}
private long updateOffsetLimits(final TopicPartition partition) {
if (!offsetLimits.containsKey(partition)) {
throw new IllegalArgumentException("Topic is not both a source and a changelog: " + partition);
}
final Map<TopicPartition, Long> newLimits = committedOffsetForPartitions(offsetLimits.keySet());
for (final Map.Entry<TopicPartition, Long> newlimit : newLimits.entrySet()) {
final Long previousLimit = offsetLimits.get(newlimit.getKey());
if (previousLimit != null && previousLimit > newlimit.getValue()) {
throw new IllegalStateException("Offset limit should monotonically increase, but was reduced. " +
"New limit: " + newlimit.getValue() + ". Previous limit: " + previousLimit);
}
}
offsetLimits.putAll(newLimits);
updateOffsetLimits = false;
return offsetLimits.get(partition);
public Map<TopicPartition, Long> changelogOffsets() {
return Collections.unmodifiableMap(stateMgr.changelogOffsets());
}
private Map<TopicPartition, Long> committedOffsetForPartitions(final Set<TopicPartition> partitions) {
try {
// those do not have a committed offset would default to 0
return consumer.committed(partitions).entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset()));
} catch (final AuthorizationException e) {
throw new ProcessorStateException(String.format("task [%s] AuthorizationException when initializing offsets for %s", id, partitions), e);
} catch (final WakeupException e) {
throw e;
} catch (final KafkaException e) {
throw new ProcessorStateException(String.format("task [%s] Failed to initialize offsets for %s", id, partitions), e);
}
}
void allowUpdateOfOffsetLimit() {
updateOffsetLimits = true;
}
}
}

View File

@ -23,9 +23,10 @@ import org.apache.kafka.streams.processor.StateStore;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
interface StateManager extends Checkpointable {
interface StateManager {
File baseDir();
/**
@ -33,16 +34,18 @@ interface StateManager extends Checkpointable {
* (e.g., when it conflicts with the names of internal topics, like the checkpoint file name)
* @throws StreamsException if the store's change log does not contain the partition
*/
void register(final StateStore store, final StateRestoreCallback stateRestoreCallback);
void registerStore(final StateStore store, final StateRestoreCallback stateRestoreCallback);
StateStore getStore(final String name);
void flush();
void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext);
void checkpoint(final Map<TopicPartition, Long> offsets);
void close(final boolean clean) throws IOException;
Map<TopicPartition, Long> changelogOffsets();
void close() throws IOException;
// TODO: we can remove this when consolidating global state manager into processor state manager
StateStore getGlobalStore(final String name);
StateStore getStore(final String name);
}

View File

@ -16,23 +16,16 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.FixedOrderMap;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.errors.LockException;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.state.internals.RecordConverter;
import org.slf4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import static org.apache.kafka.streams.state.internals.RecordConverters.identity;
import static org.apache.kafka.streams.state.internals.RecordConverters.rawValueToTimestampedValue;
@ -47,82 +40,85 @@ final class StateManagerUtil {
return isTimestamped(store) ? rawValueToTimestampedValue() : identity();
}
public static void reinitializeStateStoresForPartitions(final Logger log,
final boolean eosEnabled,
final File baseDir,
final FixedOrderMap<String, Optional<StateStore>> stateStores,
final Map<String, String> storeToChangelogTopic,
final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext,
final OffsetCheckpoint checkpointFile,
final Map<TopicPartition, Long> checkpointFileCache) {
final Map<String, String> changelogTopicToStore = inverseOneToOneMap(storeToChangelogTopic);
final Set<String> storesToBeReinitialized = new HashSet<>();
for (final TopicPartition topicPartition : partitions) {
checkpointFileCache.remove(topicPartition);
storesToBeReinitialized.add(changelogTopicToStore.get(topicPartition.topic()));
/**
* @throws StreamsException If the store's change log does not contain the partition
*/
static void registerStateStores(final Logger log,
final String logPrefix,
final ProcessorTopology topology,
final ProcessorStateManager stateMgr,
final StateDirectory stateDirectory,
final InternalProcessorContext processorContext) {
if (topology.stateStores().isEmpty()) {
return;
}
if (!eosEnabled) {
try {
checkpointFile.write(checkpointFileCache);
} catch (final IOException fatalException) {
log.error("Failed to write offset checkpoint file to {} while re-initializing {}: {}",
checkpointFile,
stateStores,
fatalException);
throw new StreamsException("Failed to reinitialize global store.", fatalException);
final TaskId id = stateMgr.taskId();
try {
if (!stateDirectory.lock(id)) {
throw new LockException(String.format("%sFailed to lock the state directory for task %s", logPrefix, id));
}
} catch (final IOException e) {
throw new StreamsException(
String.format("%sFatal error while trying to lock the state directory for task %s", logPrefix, id),
e
);
}
log.debug("Acquired state directory lock");
for (final String storeName : storesToBeReinitialized) {
if (!stateStores.containsKey(storeName)) {
// the store has never been registered; carry on...
continue;
}
final StateStore stateStore = stateStores
.get(storeName)
.orElseThrow(
() -> new IllegalStateException(
"Re-initializing store that has not been initialized. This is a bug in Kafka Streams."
)
);
try {
stateStore.close();
} catch (final RuntimeException ignoreAndSwallow) { /* ignore */ }
// We should only load checkpoint AFTER the corresponding state directory lock has been acquired and
// the state stores have been registered; we should not try to load at the state manager construction time.
// See https://issues.apache.org/jira/browse/KAFKA-8574
for (final StateStore store : topology.stateStores()) {
processorContext.uninitialize();
stateStores.put(storeName, Optional.empty());
store.init(processorContext, store);
log.trace("Registered state store {}", store.name());
}
stateMgr.initializeStoreOffsetsFromCheckpoint();
log.debug("Initialized state stores");
}
// TODO remove this eventually
// -> (only after we are sure, we don't need it for backward compatibility reasons anymore; maybe 2.0 release?)
// this is an ugly "hack" that is required because RocksDBStore does not follow the pattern to put the
// store directory as <taskDir>/<storeName> but nests it with an intermediate <taskDir>/rocksdb/<storeName>
try {
Utils.delete(new File(baseDir + File.separator + "rocksdb" + File.separator + storeName));
} catch (final IOException fatalException) {
log.error("Failed to reinitialize store {}.", storeName, fatalException);
throw new StreamsException(String.format("Failed to reinitialize store %s.", storeName), fatalException);
}
try {
Utils.delete(new File(baseDir + File.separator + storeName));
} catch (final IOException fatalException) {
log.error("Failed to reinitialize store {}.", storeName, fatalException);
throw new StreamsException(String.format("Failed to reinitialize store %s.", storeName), fatalException);
}
stateStore.init(processorContext, stateStore);
static void wipeStateStores(final Logger log, final ProcessorStateManager stateMgr) {
// we can just delete the whole dir of the task, including the state store images and the checkpoint files
try {
Utils.delete(stateMgr.baseDir());
} catch (final IOException fatalException) {
// since it is only called under dirty close, we always swallow the exception
log.warn("Failed to wiping state stores for task {}", stateMgr.taskId());
}
}
private static Map<String, String> inverseOneToOneMap(final Map<String, String> origin) {
final Map<String, String> reversedMap = new HashMap<>();
for (final Map.Entry<String, String> entry : origin.entrySet()) {
reversedMap.put(entry.getValue(), entry.getKey());
}
return reversedMap;
}
/**
* @throws ProcessorStateException if there is an error while closing the state manager
*/
static void closeStateManager(final Logger log,
final String logPrefix,
final boolean closeClean,
final ProcessorStateManager stateMgr,
final StateDirectory stateDirectory) {
ProcessorStateException exception = null;
log.trace("Closing state manager");
final TaskId id = stateMgr.taskId();
try {
stateMgr.close();
} catch (final ProcessorStateException e) {
exception = e;
} finally {
try {
stateDirectory.unlock(id);
} catch (final IOException e) {
if (exception == null) {
exception = new ProcessorStateException(String.format("%sFailed to release state dir lock", logPrefix), e);
}
}
}
if (exception != null) {
if (closeClean)
throw exception;
else
log.warn("Closing standby task " + id + " uncleanly throws an exception " + exception);
}
}
}

View File

@ -1,139 +0,0 @@
/*
* 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.processor.internals;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.streams.processor.StateRestoreListener;
import org.apache.kafka.streams.state.internals.RecordConverter;
import java.util.ArrayList;
import java.util.Collection;
public class StateRestorer {
static final int NO_CHECKPOINT = -1;
private final long offsetLimit;
private final boolean persistent;
private final String storeName;
private final TopicPartition partition;
private final CompositeRestoreListener compositeRestoreListener;
private final RecordConverter recordConverter;
private long checkpointOffset;
private long restoredOffset;
private long startingOffset;
private long endingOffset;
StateRestorer(final TopicPartition partition,
final CompositeRestoreListener compositeRestoreListener,
final Long checkpoint,
final long offsetLimit,
final boolean persistent,
final String storeName,
final RecordConverter recordConverter) {
this.partition = partition;
this.compositeRestoreListener = compositeRestoreListener;
this.checkpointOffset = checkpoint == null ? NO_CHECKPOINT : checkpoint;
this.offsetLimit = offsetLimit;
this.persistent = persistent;
this.storeName = storeName;
this.recordConverter = recordConverter;
}
public TopicPartition partition() {
return partition;
}
public String storeName() {
return storeName;
}
long checkpoint() {
return checkpointOffset;
}
void setCheckpointOffset(final long checkpointOffset) {
this.checkpointOffset = checkpointOffset;
}
void restoreStarted() {
compositeRestoreListener.onRestoreStart(partition, storeName, startingOffset, endingOffset);
}
void restoreDone() {
compositeRestoreListener.onRestoreEnd(partition, storeName, restoredNumRecords());
}
void restoreBatchCompleted(final long currentRestoredOffset, final int numRestored) {
compositeRestoreListener.onBatchRestored(partition, storeName, currentRestoredOffset, numRestored);
}
void restore(final Collection<ConsumerRecord<byte[], byte[]>> records) {
final Collection<ConsumerRecord<byte[], byte[]>> convertedRecords = new ArrayList<>(records.size());
for (final ConsumerRecord<byte[], byte[]> record : records) {
convertedRecords.add(recordConverter.convert(record));
}
compositeRestoreListener.restoreBatch(convertedRecords);
}
boolean isPersistent() {
return persistent;
}
void setUserRestoreListener(final StateRestoreListener userRestoreListener) {
this.compositeRestoreListener.setUserRestoreListener(userRestoreListener);
}
void setRestoredOffset(final long restoredOffset) {
this.restoredOffset = Math.min(offsetLimit, restoredOffset);
}
void setStartingOffset(final long startingOffset) {
this.startingOffset = Math.min(offsetLimit, startingOffset);
}
void setEndingOffset(final long endingOffset) {
this.endingOffset = Math.min(offsetLimit, endingOffset);
}
long startingOffset() {
return startingOffset;
}
boolean hasCompleted(final long recordOffset, final long endOffset) {
return endOffset == 0 || recordOffset >= readTo(endOffset);
}
Long restoredOffset() {
return restoredOffset;
}
long restoredNumRecords() {
return restoredOffset - startingOffset;
}
long offsetLimit() {
return offsetLimit;
}
private Long readTo(final long endOffset) {
return endOffset < offsetLimit ? endOffset : offsetLimit;
}
}

View File

@ -20,7 +20,6 @@ import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.InvalidOffsetException;
import org.apache.kafka.clients.producer.Producer;
@ -53,20 +52,18 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import static java.util.Collections.singleton;
public class StreamThread extends Thread {
private final Admin adminClient;
/**
* Stream thread states are the possible states that a stream thread can be in.
* A thread must only be in one state at a time
@ -223,7 +220,7 @@ public class StreamThread extends Thread {
state = newState;
if (newState == State.RUNNING) {
updateThreadMetadata(taskManager.activeTasks(), taskManager.standbyTasks());
updateThreadMetadata(taskManager.activeTaskMap(), taskManager.standbyTaskMap());
} else {
updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap());
}
@ -250,6 +247,11 @@ public class StreamThread extends Thread {
this.rebalanceException = rebalanceException;
}
// public for testing purposes
public interface ProducerSupplier {
Producer<byte[], byte[]> get(final TaskId id);
}
static abstract class AbstractTaskCreator<T extends Task> {
final String applicationId;
final InternalTopologyBuilder builder;
@ -304,14 +306,15 @@ public class StreamThread extends Thread {
abstract T createTask(final Consumer<byte[], byte[]> consumer, final TaskId id, final Set<TopicPartition> partitions);
public void close() {}
void close() {}
}
static class TaskCreator extends AbstractTaskCreator<StreamTask> {
private final ThreadCache cache;
private final KafkaClientSupplier clientSupplier;
private final String threadId;
private final ThreadCache cache;
private final Producer<byte[], byte[]> threadProducer;
private final KafkaClientSupplier clientSupplier;
private final ProducerSupplier producerSupplier;
private final Sensor createTaskSensor;
TaskCreator(final InternalTopologyBuilder builder,
@ -322,7 +325,6 @@ public class StreamThread extends Thread {
final ThreadCache cache,
final Time time,
final KafkaClientSupplier clientSupplier,
final Producer<byte[], byte[]> threadProducer,
final String threadId,
final Logger log) {
super(
@ -333,11 +335,22 @@ public class StreamThread extends Thread {
storeChangelogReader,
time,
log);
final boolean eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
if (!eosEnabled) {
final Map<String, Object> producerConfigs = config.getProducerConfigs(getThreadProducerClientId(threadId));
log.info("Creating thread producer client");
threadProducer = clientSupplier.getProducer(producerConfigs);
} else {
threadProducer = null;
}
this.cache = cache;
this.clientSupplier = clientSupplier;
this.threadProducer = threadProducer;
this.threadId = threadId;
createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics);
this.clientSupplier = clientSupplier;
this.producerSupplier = new TaskProducerSupplier();
this.createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics);
}
@Override
@ -346,33 +359,60 @@ public class StreamThread extends Thread {
final Set<TopicPartition> partitions) {
createTaskSensor.record();
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final String logPrefix = threadIdPrefix + String.format("%s [%s] ", "task", taskId);
final LogContext logContext = new LogContext(logPrefix);
final ProcessorTopology topology = builder.build(taskId.topicGroupId);
final ProcessorStateManager stateManager = new ProcessorStateManager(
taskId,
partitions,
Task.TaskType.ACTIVE,
stateDirectory,
topology.storeToChangelogTopic(),
storeChangelogReader,
logContext);
final RecordCollector recordCollector = new RecordCollectorImpl(
taskId,
config,
logContext,
streamsMetrics,
consumer,
producerSupplier);
return new StreamTask(
taskId,
partitions,
builder.build(taskId.topicGroupId),
topology,
consumer,
storeChangelogReader,
config,
streamsMetrics,
stateDirectory,
cache,
time,
() -> createProducer(taskId));
stateManager,
recordCollector);
}
private Producer<byte[], byte[]> createProducer(final TaskId id) {
// eos
if (threadProducer == null) {
final Map<String, Object> producerConfigs = config.getProducerConfigs(getTaskProducerClientId(threadId, id));
log.info("Creating producer client for task {}", id);
producerConfigs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, applicationId + "-" + id);
return clientSupplier.getProducer(producerConfigs);
private class TaskProducerSupplier implements ProducerSupplier {
@Override
public Producer<byte[], byte[]> get(final TaskId id) {
if (threadProducer == null) {
// create one producer per task for EOS
// TODO: after KIP-447 this would be removed
final Map<String, Object> producerConfigs = config.getProducerConfigs(getTaskProducerClientId(threadId, id));
producerConfigs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, applicationId + "-" + id);
log.info("Creating producer client for task {}", id);
return clientSupplier.getProducer(producerConfigs);
} else {
return threadProducer;
}
}
return threadProducer;
}
@Override
public void close() {
if (threadProducer != null) {
try {
@ -412,17 +452,29 @@ public class StreamThread extends Thread {
final Set<TopicPartition> partitions) {
createTaskSensor.record();
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final String logPrefix = threadIdPrefix + String.format("%s [%s] ", "standby-task", taskId);
final LogContext logContext = new LogContext(logPrefix);
final ProcessorTopology topology = builder.build(taskId.topicGroupId);
if (!topology.stateStores().isEmpty() && !topology.storeToChangelogTopic().isEmpty()) {
final ProcessorStateManager stateManager = new ProcessorStateManager(
taskId,
partitions,
Task.TaskType.STANDBY,
stateDirectory,
topology.storeToChangelogTopic(),
storeChangelogReader,
logContext);
return new StandbyTask(
taskId,
partitions,
topology,
consumer,
storeChangelogReader,
config,
streamsMetrics,
stateManager,
stateDirectory);
} else {
log.trace(
@ -457,17 +509,17 @@ public class StreamThread extends Thread {
private long lastCommitMs;
private int numIterations;
private Throwable rebalanceException = null;
private boolean processStandbyRecords = false;
private volatile State state = State.CREATED;
private volatile ThreadMetadata threadMetadata;
private StreamThread.StateListener stateListener;
private Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> standbyRecords;
private final ChangelogReader changelogReader;
// package-private for testing
final ConsumerRebalanceListener rebalanceListener;
final Producer<byte[], byte[]> producer;
final Consumer<byte[], byte[]> restoreConsumer;
final Consumer<byte[], byte[]> consumer;
final Producer<byte[], byte[]> producer;
final InternalTopologyBuilder builder;
public static StreamThread create(final InternalTopologyBuilder builder,
@ -492,20 +544,12 @@ public class StreamThread extends Thread {
log.info("Creating restore consumer client");
final Map<String, Object> restoreConsumerConfigs = config.getRestoreConsumerConfigs(getRestoreConsumerClientId(threadId));
final Consumer<byte[], byte[]> restoreConsumer = clientSupplier.getRestoreConsumer(restoreConsumerConfigs);
final Duration pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG));
final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreConsumer, pollTime, userStateRestoreListener, logContext);
Producer<byte[], byte[]> threadProducer = null;
final boolean eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG));
if (!eosEnabled) {
final Map<String, Object> producerConfigs = config.getProducerConfigs(getThreadProducerClientId(threadId));
log.info("Creating shared producer client");
threadProducer = clientSupplier.getProducer(producerConfigs);
}
final StoreChangelogReader changelogReader = new StoreChangelogReader(config, logContext, restoreConsumer, userStateRestoreListener);
final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);
final AbstractTaskCreator<StreamTask> activeTaskCreator = new TaskCreator(
final TaskCreator activeTaskCreator = new TaskCreator(
builder,
config,
streamsMetrics,
@ -514,10 +558,9 @@ public class StreamThread extends Thread {
cache,
time,
clientSupplier,
threadProducer,
threadId,
log);
final AbstractTaskCreator<StandbyTask> standbyTaskCreator = new StandbyTaskCreator(
final StandbyTaskCreator standbyTaskCreator = new StandbyTaskCreator(
builder,
config,
streamsMetrics,
@ -530,18 +573,18 @@ public class StreamThread extends Thread {
changelogReader,
processId,
logPrefix,
restoreConsumer,
streamsMetadataState,
activeTaskCreator,
standbyTaskCreator,
adminClient,
new AssignedStreamsTasks(logContext),
new AssignedStandbyTasks(logContext));
builder,
adminClient
);
log.info("Creating consumer client");
final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
final Map<String, Object> consumerConfigs = config.getMainConsumerConfigs(applicationId, getConsumerClientId(threadId), threadIdx);
consumerConfigs.put(StreamsConfig.InternalConfig.TASK_MANAGER_FOR_PARTITION_ASSIGNOR, taskManager);
consumerConfigs.put(StreamsConfig.InternalConfig.STREAMS_METADATA_STATE_FOR_PARTITION_ASSIGNOR, streamsMetadataState);
consumerConfigs.put(StreamsConfig.InternalConfig.STREAMS_ADMIN_CLIENT, adminClient);
final AtomicInteger assignmentErrorCode = new AtomicInteger();
consumerConfigs.put(StreamsConfig.InternalConfig.ASSIGNMENT_ERROR_CODE, assignmentErrorCode);
String originalReset = null;
@ -551,29 +594,35 @@ public class StreamThread extends Thread {
}
final Consumer<byte[], byte[]> consumer = clientSupplier.getConsumer(consumerConfigs);
changelogReader.setMainConsumer(consumer);
taskManager.setConsumer(consumer);
return new StreamThread(
final StreamThread streamThread = new StreamThread(
time,
config,
threadProducer,
activeTaskCreator.threadProducer,
adminClient,
restoreConsumer,
consumer,
changelogReader,
originalReset,
taskManager,
streamsMetrics,
builder,
threadId,
logContext,
assignmentErrorCode)
.updateThreadMetadata(getSharedAdminClientId(clientId));
assignmentErrorCode);
return streamThread.updateThreadMetadata(getSharedAdminClientId(clientId));
}
public StreamThread(final Time time,
final StreamsConfig config,
final Producer<byte[], byte[]> producer,
final Admin adminClient,
final Consumer<byte[], byte[]> restoreConsumer,
final Consumer<byte[], byte[]> consumer,
final ChangelogReader changelogReader,
final String originalReset,
final TaskManager taskManager,
final StreamsMetricsImpl streamsMetrics,
@ -582,10 +631,9 @@ public class StreamThread extends Thread {
final LogContext logContext,
final AtomicInteger assignmentErrorCode) {
super(threadId);
this.stateLock = new Object();
this.standbyRecords = new HashMap<>();
this.adminClient = adminClient;
this.streamsMetrics = streamsMetrics;
this.commitSensor = ThreadMetrics.commitSensor(threadId, streamsMetrics);
this.pollSensor = ThreadMetrics.pollSensor(threadId, streamsMetrics);
@ -610,16 +658,17 @@ public class StreamThread extends Thread {
this.log = logContext.logger(StreamThread.class);
this.rebalanceListener = new StreamsRebalanceListener(time, taskManager, this, this.log);
this.taskManager = taskManager;
this.producer = producer;
this.restoreConsumer = restoreConsumer;
this.consumer = consumer;
this.producer = producer;
this.changelogReader = changelogReader;
this.originalReset = originalReset;
this.assignmentErrorCode = assignmentErrorCode;
this.pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG));
final int dummyThreadIdx = 1;
this.maxPollTimeMs = new InternalConsumerConfig(config.getMainConsumerConfigs("dummyGroupId", "dummyClientId", dummyThreadIdx))
.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG);
.getInt(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG);
this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG);
this.numIterations = 1;
@ -671,7 +720,7 @@ public class StreamThread extends Thread {
cleanRun = true;
} catch (final KafkaException e) {
log.error("Encountered the following unexpected Kafka exception during processing, " +
"this usually indicate Streams internal errors:", e);
"this usually indicate Streams internal errors:", e);
throw e;
} catch (final Exception e) {
// we have caught all Kafka related exceptions, and other runtime exceptions
@ -702,9 +751,8 @@ public class StreamThread extends Thread {
}
} catch (final TaskMigratedException ignoreAndRejoinGroup) {
log.warn("Detected task {} that got migrated to another thread. " +
"This implies that this thread missed a rebalance and dropped out of the consumer group. " +
"Will try to rejoin the consumer group. Below is the detailed description of the task:\n{}",
ignoreAndRejoinGroup.migratedTask().id(), ignoreAndRejoinGroup.migratedTask().toString(">"));
"This implies that this thread missed a rebalance and dropped out of the consumer group. " +
"Will try to rejoin the consumer group.", ignoreAndRejoinGroup.migratedTaskId());
enforceRebalance();
}
@ -770,19 +818,26 @@ public class StreamThread extends Thread {
addRecordsToTasks(records);
}
// we can always let changelog reader to try restoring in order to initialize the changelogs;
// if there's no active restoring or standby updating it would not try to fetch any data
changelogReader.restore();
// only try to initialize the assigned tasks
// if the state is still in PARTITION_ASSIGNED after the poll call
if (state == State.PARTITIONS_ASSIGNED) {
if (taskManager.updateNewAndRestoringTasks()) {
if (taskManager.checkForCompletedRestoration()) {
changelogReader.transitToUpdateStandby();
setState(State.RUNNING);
} else {
// transit to restore active is idempotent so we can call it multiple times
changelogReader.transitToRestoreActive();
}
}
advanceNowAndComputeLatency();
// TODO: we will process some tasks even if the state is not RUNNING, i.e. some other
// tasks are still being restored.
if (taskManager.hasActiveRunningTasks()) {
if (state == State.RUNNING) {
/*
* Within an iteration, after N (N initialized as 1 upon start up) round of processing one-record-each on the applicable tasks, check the current time:
* 1. If it is time to commit, do it;
@ -826,12 +881,9 @@ public class StreamThread extends Thread {
numIterations++;
}
} while (processed > 0);
maybeCommit();
}
// update standby tasks and maybe commit the standby tasks as well
maybeUpdateStandbyTasks();
maybeCommit();
}
/**
@ -914,7 +966,7 @@ public class StreamThread extends Thread {
private void addRecordsToTasks(final ConsumerRecords<byte[], byte[]> records) {
for (final TopicPartition partition : records.partitions()) {
final StreamTask task = taskManager.activeTask(partition);
final Task task = taskManager.taskForInputPartition(partition);
if (task == null) {
log.error(
@ -923,10 +975,6 @@ public class StreamThread extends Thread {
taskManager.toString(">")
);
throw new NullPointerException("Task was unexpectedly missing for partition " + partition);
} else if (task.isClosed()) {
log.info("Stream task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
"Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
task.addRecords(partition, records.records(partition));
@ -960,7 +1008,7 @@ public class StreamThread extends Thread {
if (now - lastCommitMs > commitTimeMs) {
if (log.isTraceEnabled()) {
log.trace("Committing all active tasks {} and standby tasks {} since {}ms has elapsed (commit interval is {}ms)",
taskManager.activeTaskIds(), taskManager.standbyTaskIds(), now - lastCommitMs, commitTimeMs);
taskManager.activeTaskIds(), taskManager.standbyTaskIds(), now - lastCommitMs, commitTimeMs);
}
committed = taskManager.commitAll();
@ -969,11 +1017,11 @@ public class StreamThread extends Thread {
commitSensor.record(intervalCommitLatency / (double) committed, now);
// try to purge the committed records for repartition topics if possible
taskManager.maybePurgeCommitedRecords();
taskManager.maybePurgeCommittedRecords();
if (log.isDebugEnabled()) {
log.debug("Committed all active tasks {} and standby tasks {} in {}ms",
taskManager.activeTaskIds(), taskManager.standbyTaskIds(), intervalCommitLatency);
taskManager.activeTaskIds(), taskManager.standbyTaskIds(), intervalCommitLatency);
}
}
@ -982,8 +1030,6 @@ public class StreamThread extends Thread {
} else {
lastCommitMs = now;
}
processStandbyRecords = true;
} else {
committed = taskManager.maybeCommitActiveTasksPerUserRequested();
if (committed > 0) {
@ -995,93 +1041,6 @@ public class StreamThread extends Thread {
return committed > 0;
}
private void maybeUpdateStandbyTasks() {
if (state == State.RUNNING && taskManager.hasStandbyRunningTasks()) {
if (processStandbyRecords) {
if (!standbyRecords.isEmpty()) {
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> remainingStandbyRecords = new HashMap<>();
for (final Map.Entry<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> entry : standbyRecords.entrySet()) {
final TopicPartition partition = entry.getKey();
List<ConsumerRecord<byte[], byte[]>> remaining = entry.getValue();
if (remaining != null) {
final StandbyTask task = taskManager.standbyTask(partition);
if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
"Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
remaining = task.update(partition, remaining);
if (!remaining.isEmpty()) {
remainingStandbyRecords.put(partition, remaining);
} else {
restoreConsumer.resume(singleton(partition));
}
}
}
standbyRecords = remainingStandbyRecords;
if (log.isDebugEnabled()) {
log.debug("Updated standby tasks {} in {}ms", taskManager.standbyTaskIds(), time.milliseconds() - now);
}
}
processStandbyRecords = false;
}
try {
// poll(0): Since this is during the normal processing, not during restoration.
// We can afford to have slower restore (because we don't wait inside poll for results).
// Instead, we want to proceed to the next iteration to call the main consumer#poll()
// as soon as possible so as to not be kicked out of the group.
final ConsumerRecords<byte[], byte[]> records = restoreConsumer.poll(Duration.ZERO);
if (!records.isEmpty()) {
for (final TopicPartition partition : records.partitions()) {
final StandbyTask task = taskManager.standbyTask(partition);
if (task == null) {
throw new StreamsException(logPrefix + "Missing standby task for partition " + partition);
}
if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
"Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
final List<ConsumerRecord<byte[], byte[]>> remaining = task.update(partition, records.records(partition));
if (!remaining.isEmpty()) {
restoreConsumer.pause(singleton(partition));
standbyRecords.put(partition, remaining);
}
}
}
} catch (final InvalidOffsetException recoverableException) {
log.warn("Updating StandbyTasks failed. Deleting StandbyTasks stores to recreate from scratch.", recoverableException);
final Set<TopicPartition> partitions = recoverableException.partitions();
for (final TopicPartition partition : partitions) {
final StandbyTask task = taskManager.standbyTask(partition);
if (task.isClosed()) {
log.info("Standby task {} is already closed, probably because it got unexpectedly migrated to another thread already. " +
"Notifying the thread to trigger a new rebalance immediately.", task.id());
throw new TaskMigratedException(task);
}
log.info("Reinitializing StandbyTask {} from changelogs {}", task, recoverableException.partitions());
task.reinitializeStateStoresForPartitions(recoverableException.partitions());
}
restoreConsumer.seekToBeginning(partitions);
}
// update now if the standby restoration indeed executed
advanceNowAndComputeLatency();
}
}
/**
* Compute the latency based on the current marked timestamp, and update the marked timestamp
* with the current system timestamp.
@ -1123,6 +1082,11 @@ public class StreamThread extends Thread {
} catch (final Throwable e) {
log.error("Failed to close task manager due to the following error:", e);
}
try {
changelogReader.clear();
} catch (final Throwable e) {
log.error("Failed to close changelog reader due to the following error:", e);
}
try {
consumer.close();
} catch (final Throwable e) {
@ -1139,12 +1103,6 @@ public class StreamThread extends Thread {
log.info("Shutdown complete");
}
void clearStandbyRecords(final List<TopicPartition> partitions) {
for (final TopicPartition tp : partitions) {
standbyRecords.remove(tp);
}
}
/**
* Return information about the current {@link StreamThread}.
*
@ -1170,17 +1128,17 @@ public class StreamThread extends Thread {
return this;
}
private void updateThreadMetadata(final Map<TaskId, StreamTask> activeTasks,
final Map<TaskId, StandbyTask> standbyTasks) {
private void updateThreadMetadata(final Map<TaskId, Task> activeTasks,
final Map<TaskId, Task> standbyTasks) {
final Set<String> producerClientIds = new HashSet<>();
final Set<TaskMetadata> activeTasksMetadata = new HashSet<>();
for (final Map.Entry<TaskId, StreamTask> task : activeTasks.entrySet()) {
activeTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().partitions()));
for (final Map.Entry<TaskId, Task> task : activeTasks.entrySet()) {
activeTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().inputPartitions()));
producerClientIds.add(getTaskProducerClientId(getName(), task.getKey()));
}
final Set<TaskMetadata> standbyTasksMetadata = new HashSet<>();
for (final Map.Entry<TaskId, StandbyTask> task : standbyTasks.entrySet()) {
standbyTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().partitions()));
for (final Map.Entry<TaskId, Task> task : standbyTasks.entrySet()) {
standbyTasksMetadata.add(new TaskMetadata(task.getKey().toString(), task.getValue().inputPartitions()));
}
final String adminClientId = threadMetadata.adminClientId();
@ -1195,27 +1153,12 @@ public class StreamThread extends Thread {
standbyTasksMetadata);
}
public Map<TaskId, StreamTask> activeTasks() {
return taskManager.activeTasks();
}
public List<StreamTask> allStreamsTasks() {
return taskManager.allStreamsTasks();
}
public List<StandbyTask> allStandbyTasks() {
return taskManager.allStandbyTasks();
}
public Set<TaskId> restoringTaskIds() {
return taskManager.restoringTaskIds();
public List<Task> activeTasks() {
return taskManager.activeTaskIterable();
}
public Map<TaskId, Task> allTasks() {
final Map<TaskId, Task> result = new TreeMap<>();
result.putAll(taskManager.standbyTasks());
result.putAll(taskManager.activeTasks());
return result;
return taskManager.tasks();
}
/**
@ -1250,8 +1193,8 @@ public class StreamThread extends Thread {
// When EOS is turned on, each task will have its own producer client
// and the producer object passed in here will be null. We would then iterate through
// all the active tasks and add their metrics to the output metrics map.
for (final StreamTask task: taskManager.activeTasks().values()) {
final Map<MetricName, ? extends Metric> taskProducerMetrics = task.getProducer().metrics();
for (final StreamTask task : taskManager.fixmeStreamTasks().values()) {
final Map<MetricName, ? extends Metric> taskProducerMetrics = ((RecordCollectorImpl) task.recordCollector()).producer().metrics();
result.putAll(taskProducerMetrics);
}
}
@ -1268,7 +1211,7 @@ public class StreamThread extends Thread {
}
public Map<MetricName, Metric> adminClientMetrics() {
final Map<MetricName, ? extends Metric> adminClientMetrics = taskManager.adminClient().metrics();
final Map<MetricName, ? extends Metric> adminClientMetrics = adminClient.metrics();
return new LinkedHashMap<>(adminClientMetrics);
}
@ -1281,15 +1224,11 @@ public class StreamThread extends Thread {
return taskManager;
}
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> standbyRecords() {
return standbyRecords;
}
int currentNumIterations() {
return numIterations;
}
public StreamThread.StateListener stateListener() {
return stateListener;
Throwable rebalanceException() {
return rebalanceException;
}
}

View File

@ -161,6 +161,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
private int numStandbyReplicas;
private TaskManager taskManager;
private StreamsMetadataState streamsMetadataState;
@SuppressWarnings("deprecation")
private org.apache.kafka.streams.processor.PartitionGrouper partitionGrouper;
private AtomicInteger assignmentErrorCode;
@ -171,14 +172,6 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
private CopartitionedTopicsEnforcer copartitionedTopicsEnforcer;
private RebalanceProtocol rebalanceProtocol;
protected String userEndPoint() {
return userEndPoint;
}
protected TaskManager taskManger() {
return taskManager;
}
/**
* We need to have the PartitionAssignor and its StreamThread to be mutually accessible since the former needs
* later's cached metadata while sending subscriptions, and the latter needs former's returned assignment when
@ -195,6 +188,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
usedSubscriptionMetadataVersion = assignorConfiguration
.configuredMetadataVersion(usedSubscriptionMetadataVersion);
taskManager = assignorConfiguration.getTaskManager();
streamsMetadataState = assignorConfiguration.getStreamsMetadataState();
assignmentErrorCode = assignorConfiguration.getAssignmentErrorCode(configs);
numStandbyReplicas = assignorConfiguration.getNumStandbyReplicas();
partitionGrouper = assignorConfiguration.getPartitionGrouper();
@ -225,7 +219,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
// 1. Client UUID (a unique id assigned to an instance of KafkaStreams)
// 2. Task ids of previously running tasks
// 3. Task ids of valid local states on the client's state directory.
final Set<TaskId> standbyTasks = taskManager.cachedTasksIds();
final Set<TaskId> standbyTasks = taskManager.tasksOnLocalStorage();
final Set<TaskId> activeTasks = prepareForSubscription(taskManager,
topics,
standbyTasks,
@ -251,7 +245,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
switch (rebalanceProtocol) {
case EAGER:
// In eager, onPartitionsRevoked is called first and we must get the previously saved running task ids
activeTasks = taskManager.previousRunningTaskIds();
activeTasks = taskManager.activeTaskIds();
standbyTasks.removeAll(activeTasks);
break;
case COOPERATIVE:
@ -263,8 +257,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
throw new IllegalStateException("Streams partition assignor's rebalance protocol is unknown");
}
taskManager.updateSubscriptionsFromMetadata(topics);
taskManager.setRebalanceInProgress(true);
taskManager.handleRebalanceStart(topics);
return activeTasks;
}
@ -395,7 +388,8 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
// parse the topology to determine the repartition source topics,
// making sure they are created with the number of partitions as
// the maximum of the depending sub-topologies source topics' number of partitions
final Map<Integer, InternalTopologyBuilder.TopicsInfo> topicGroups = taskManager.builder().topicGroups();
final Map<Integer, InternalTopologyBuilder.TopicsInfo> topicGroups =
taskManager.builder().topicGroups();
final Map<String, InternalTopicConfig> repartitionTopicMetadata = new HashMap<>();
for (final InternalTopologyBuilder.TopicsInfo topicsInfo : topicGroups.values()) {
@ -500,7 +494,6 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
}
final Cluster fullMetadata = metadata.withPartitions(allRepartitionTopicPartitions);
taskManager.setClusterMetadata(fullMetadata);
log.debug("Created repartition topics {} from the parsed topology.", allRepartitionTopicPartitions.values());
@ -646,7 +639,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
}
}
}
taskManager.setHostPartitionMappings(partitionsByHost, standbyPartitionsByHost);
streamsMetadataState.onChange(partitionsByHost, standbyPartitionsByHost, fullMetadata);
final Map<String, Assignment> assignment;
if (versionProbing) {
@ -968,9 +961,9 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
* @return set of consumer(s) that previously owned the partitions in this task
* empty set signals that it is a new task, or its previous owner is no longer in the group
*/
Set<String> previousConsumersOfTaskPartitions(final Set<TopicPartition> taskPartitions,
final Map<TopicPartition, String> clientOwnedPartitions,
final Set<TopicPartition> allOwnedPartitions) {
private Set<String> previousConsumersOfTaskPartitions(final Set<TopicPartition> taskPartitions,
final Map<TopicPartition, String> clientOwnedPartitions,
final Set<TopicPartition> allOwnedPartitions) {
// this "foreignConsumer" indicates a partition was owned by someone from another client -- we don't really care who
final String foreignConsumer = "";
final Set<String> previousConsumers = new HashSet<>();
@ -1125,32 +1118,39 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
}
// version 1 field
final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
final Map<TaskId, Set<TopicPartition>> activeTasks;
// version 2 fields
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo;
final Map<HostInfo, Set<TopicPartition>> partitionsByHost;
final Map<HostInfo, Set<TopicPartition>> standbyPartitionsByHost;
final Map<TopicPartition, TaskId> partitionsToTaskId = new HashMap<>();
switch (receivedAssignmentMetadataVersion) {
case 1:
processVersionOneAssignment(logPrefix, info, partitions, activeTasks, partitionsToTaskId);
validateActiveTaskEncoding(partitions, info, logPrefix);
activeTasks = getActiveTasks(partitions, info);
partitionsByHost = Collections.emptyMap();
standbyPartitionsByHost = Collections.emptyMap();
topicToPartitionInfo = Collections.emptyMap();
break;
case 2:
case 3:
case 4:
case 5:
processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId);
validateActiveTaskEncoding(partitions, info, logPrefix);
activeTasks = getActiveTasks(partitions, info);
partitionsByHost = info.partitionsByHost();
standbyPartitionsByHost = Collections.emptyMap();
topicToPartitionInfo = getTopicPartitionInfo(partitionsByHost);
break;
case 6:
processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId);
validateActiveTaskEncoding(partitions, info, logPrefix);
activeTasks = getActiveTasks(partitions, info);
partitionsByHost = info.partitionsByHost();
standbyPartitionsByHost = info.standbyPartitionByHost();
topicToPartitionInfo = getTopicPartitionInfo(partitionsByHost);
break;
default:
throw new IllegalStateException(
@ -1159,50 +1159,27 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
);
}
taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo));
taskManager.setHostPartitionMappings(partitionsByHost, standbyPartitionsByHost);
taskManager.setPartitionsToTaskId(partitionsToTaskId);
taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks());
taskManager.updateSubscriptionsFromAssignment(partitions);
taskManager.setRebalanceInProgress(false);
final Cluster fakeCluster = Cluster.empty().withPartitions(topicToPartitionInfo);
streamsMetadataState.onChange(partitionsByHost, standbyPartitionsByHost, fakeCluster);
// we do not capture any exceptions but just let the exception thrown from consumer.poll directly
// since when stream thread captures it, either we close all tasks as dirty or we close thread
taskManager.handleAssignment(activeTasks, info.standbyTasks());
}
private static void processVersionOneAssignment(final String logPrefix,
final AssignmentInfo info,
final List<TopicPartition> partitions,
final Map<TaskId, Set<TopicPartition>> activeTasks,
final Map<TopicPartition, TaskId> partitionsToTaskId) {
// the number of assigned partitions should be the same as number of active tasks, which
// could be duplicated if one task has more than one assigned partitions
if (partitions.size() != info.activeTasks().size()) {
throw new TaskAssignmentException(
String.format(
"%sNumber of assigned partitions %d is not equal to "
+ "the number of active taskIds %d, assignmentInfo=%s",
logPrefix, partitions.size(),
info.activeTasks().size(), info.toString()
)
);
}
// protected for upgrade test
protected static Map<TaskId, Set<TopicPartition>> getActiveTasks(final List<TopicPartition> partitions, final AssignmentInfo info) {
final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
for (int i = 0; i < partitions.size(); i++) {
final TopicPartition partition = partitions.get(i);
final TaskId id = info.activeTasks().get(i);
activeTasks.computeIfAbsent(id, k -> new HashSet<>()).add(partition);
partitionsToTaskId.put(partition, id);
activeTasks.computeIfAbsent(id, k1 -> new HashSet<>()).add(partition);
}
return activeTasks;
}
public static void processVersionTwoAssignment(final String logPrefix,
final AssignmentInfo info,
final List<TopicPartition> partitions,
final Map<TaskId, Set<TopicPartition>> activeTasks,
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo,
final Map<TopicPartition, TaskId> partitionsToTaskId) {
processVersionOneAssignment(logPrefix, info, partitions, activeTasks, partitionsToTaskId);
// process partitions by host
final Map<HostInfo, Set<TopicPartition>> partitionsByHost = info.partitionsByHost();
private static Map<TopicPartition, PartitionInfo> getTopicPartitionInfo(final Map<HostInfo, Set<TopicPartition>> partitionsByHost) {
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
for (final Set<TopicPartition> value : partitionsByHost.values()) {
for (final TopicPartition topicPartition : value) {
topicToPartitionInfo.put(
@ -1217,6 +1194,22 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
);
}
}
return topicToPartitionInfo;
}
private static void validateActiveTaskEncoding(final List<TopicPartition> partitions, final AssignmentInfo info, final String logPrefix) {
// the number of assigned partitions should be the same as number of active tasks, which
// could be duplicated if one task has more than one assigned partitions
if (partitions.size() != info.activeTasks().size()) {
throw new TaskAssignmentException(
String.format(
"%sNumber of assigned partitions %d is not equal to "
+ "the number of active taskIds %d, assignmentInfo=%s",
logPrefix, partitions.size(),
info.activeTasks().size(), info.toString()
)
);
}
}
/**
@ -1287,4 +1280,11 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
this.internalTopicManager = internalTopicManager;
}
protected String userEndPoint() {
return userEndPoint;
}
protected TaskManager taskManger() {
return taskManager;
}
}

View File

@ -16,18 +16,15 @@
*/
package org.apache.kafka.streams.processor.internals;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.StreamThread.State;
import org.apache.kafka.streams.processor.internals.assignment.AssignorError;
import org.slf4j.Logger;
import java.util.Collection;
public class StreamsRebalanceListener implements ConsumerRebalanceListener {
private final Time time;
@ -46,74 +43,33 @@ public class StreamsRebalanceListener implements ConsumerRebalanceListener {
}
@Override
public void onPartitionsAssigned(final Collection<TopicPartition> assignedPartitions) {
log.debug("Current state {}: assigned partitions {} at the end of consumer rebalance.\n" +
"\tpreviously assigned active tasks: {}\n" +
"\tpreviously assigned standby tasks: {}\n",
streamThread.state(),
assignedPartitions,
taskManager.previousActiveTaskIds(),
taskManager.previousStandbyTaskIds());
public void onPartitionsAssigned(final Collection<TopicPartition> partitions) {
// NB: all task management is already handled by:
// org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.onAssignment
if (streamThread.getAssignmentErrorCode() == AssignorError.INCOMPLETE_SOURCE_TOPIC_METADATA.code()) {
log.error("Received error code {} - shutdown", streamThread.getAssignmentErrorCode());
streamThread.shutdown();
return;
}
} else {
taskManager.handleRebalanceComplete();
final long start = time.milliseconds();
List<TopicPartition> revokedStandbyPartitions = null;
try {
if (streamThread.setState(State.PARTITIONS_ASSIGNED) == null) {
log.debug(
"Skipping task creation in rebalance because we are already in {} state.",
streamThread.state());
} else {
// Close non-reassigned tasks before initializing new ones as we may have suspended active
// tasks that become standbys or vice versa
revokedStandbyPartitions = taskManager.closeRevokedStandbyTasks();
taskManager.closeRevokedSuspendedTasks();
taskManager.createTasks(assignedPartitions);
}
} catch (final Throwable t) {
log.error(
"Error caught during partition assignment, " +
"will abort the current process and re-throw at the end of rebalance", t);
streamThread.setRebalanceException(t);
} finally {
if (revokedStandbyPartitions != null) {
streamThread.clearStandbyRecords(revokedStandbyPartitions);
}
log.info("partition assignment took {} ms.\n" +
"\tcurrently assigned active tasks: {}\n" +
"\tcurrently assigned standby tasks: {}\n" +
"\trevoked active tasks: {}\n" +
"\trevoked standby tasks: {}\n",
time.milliseconds() - start,
taskManager.activeTaskIds(),
taskManager.standbyTaskIds(),
taskManager.revokedActiveTaskIds(),
taskManager.revokedStandbyTaskIds());
streamThread.setState(State.PARTITIONS_ASSIGNED);
}
}
@Override
public void onPartitionsRevoked(final Collection<TopicPartition> revokedPartitions) {
public void onPartitionsRevoked(final Collection<TopicPartition> partitions) {
log.debug("Current state {}: revoked partitions {} because of consumer rebalance.\n" +
"\tcurrently assigned active tasks: {}\n" +
"\tcurrently assigned standby tasks: {}\n",
streamThread.state(),
revokedPartitions,
taskManager.activeTaskIds(),
taskManager.standbyTaskIds());
"\tcurrently assigned active tasks: {}\n" +
"\tcurrently assigned standby tasks: {}\n",
streamThread.state(),
partitions,
taskManager.activeTaskIds(),
taskManager.standbyTaskIds());
Set<TaskId> suspendedTasks = new HashSet<>();
if (streamThread.setState(State.PARTITIONS_REVOKED) != null && !revokedPartitions.isEmpty()) {
if (streamThread.setState(State.PARTITIONS_REVOKED) != null && !partitions.isEmpty()) {
final long start = time.milliseconds();
try {
// suspend only the active tasks, reassigned standby tasks will be closed in onPartitionsAssigned
suspendedTasks = taskManager.suspendActiveTasksAndState(revokedPartitions);
taskManager.handleRevocation(partitions);
} catch (final Throwable t) {
log.error(
"Error caught during partition revocation, " +
@ -122,29 +78,25 @@ public class StreamsRebalanceListener implements ConsumerRebalanceListener {
);
streamThread.setRebalanceException(t);
} finally {
log.info("partition revocation took {} ms.\n" +
"\tcurrent suspended active tasks: {}\n",
time.milliseconds() - start,
suspendedTasks);
log.info("partition revocation took {} ms.", time.milliseconds() - start);
}
}
}
@Override
public void onPartitionsLost(final Collection<TopicPartition> lostPartitions) {
public void onPartitionsLost(final Collection<TopicPartition> partitions) {
log.info("at state {}: partitions {} lost due to missed rebalance.\n" +
"\tlost active tasks: {}\n" +
"\tlost assigned standby tasks: {}\n",
streamThread.state(),
lostPartitions,
taskManager.activeTaskIds(),
taskManager.standbyTaskIds());
"\tlost active tasks: {}\n" +
"\tlost assigned standby tasks: {}\n",
streamThread.state(),
partitions,
taskManager.activeTaskIds(),
taskManager.standbyTaskIds());
Set<TaskId> lostTasks = new HashSet<>();
final long start = time.milliseconds();
try {
// close all active tasks as lost but don't try to commit offsets as we no longer own them
lostTasks = taskManager.closeLostTasks();
taskManager.handleLostAll();
} catch (final Throwable t) {
log.error(
"Error caught during partitions lost, " +
@ -153,10 +105,7 @@ public class StreamsRebalanceListener implements ConsumerRebalanceListener {
);
streamThread.setRebalanceException(t);
} finally {
log.info("partitions lost took {} ms.\n" +
"\tclosed lost active tasks: {}\n",
time.milliseconds() - start,
lostTasks);
log.info("partitions lost took {} ms.", time.milliseconds() - start);
}
}

View File

@ -16,56 +16,178 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.streams.errors.LockException;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.TaskId;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
public interface Task {
void initializeMetadata();
long LATEST_OFFSET = -2L;
/**
* Initialize the task and return {@code true} if the task is ready to run, i.e, it has no state stores
* @return true if this task has no state stores that may need restoring.
* @throws IllegalStateException If store gets registered after initialized is already finished
* @throws StreamsException if the store's change log does not contain the partition
/*
*
* <pre>
* +-------------+
* +<---- | Created (0) |
* | +-----+-------+
* | |
* | v
* | +-----+-------+
* +<---- | Restoring(1)|<---------------+
* | +-----+-------+ |
* | | |
* | +--------------------+ |
* | | | |
* | v v |
* | +-----+-------+ +----+---+----+
* | | Running (2) | ----> | Suspended(3)| * //TODO Suspended(3) could be removed after we've stable on KIP-429
* | +-----+-------+ +------+------+
* | | |
* | | |
* | v |
* | +-----+-------+ |
* +----> | Closing (4) | <------------+
* +-----+-------+
* |
* v
* +-----+-------+
* | Closed (5) |
* +-------------+
* </pre>
*/
boolean initializeStateStores();
enum State {
CREATED(1, 4), // 0
RESTORING(2, 3, 4), // 1
RUNNING(3, 4), // 2
SUSPENDED(1, 4), // 3
CLOSING(4, 5), // 4, we allow CLOSING to transit to itself to make close idempotent
CLOSED; // 5
boolean commitNeeded();
private final Set<Integer> validTransitions = new HashSet<>();
void initializeTopology();
State(final Integer... validTransitions) {
this.validTransitions.addAll(Arrays.asList(validTransitions));
}
void commit();
public boolean isValidTransition(final State newState) {
return validTransitions.contains(newState.ordinal());
}
}
void resume();
enum TaskType {
ACTIVE("ACTIVE"),
void close(final boolean clean,
final boolean isZombie);
STANDBY("STANDBY"),
StateStore getStore(final String name);
GLOBAL("GLOBAL");
String applicationId();
public final String name;
ProcessorTopology topology();
ProcessorContext context();
TaskType(final String name) {
this.name = name;
}
}
TaskId id();
Set<TopicPartition> partitions();
State state();
boolean isActive();
boolean isClosed();
/**
* @throws LockException could happen when multi-threads within the single instance, could retry
* @throws StreamsException fatal error, should close the thread
*/
void initializeIfNeeded();
/**
* @throws StreamsException fatal error, should close the thread
*/
void completeRestoration();
void addRecords(TopicPartition partition, Iterable<ConsumerRecord<byte[], byte[]>> records);
boolean commitNeeded();
/**
* @throws TaskMigratedException all the task has been migrated
* @throws StreamsException fatal error, should close the thread
*/
void commit();
/**
* @throws TaskMigratedException all the task has been migrated
* @throws StreamsException fatal error, should close the thread
*/
void suspend();
/**
* @throws StreamsException fatal error, should close the thread
*/
void resume();
/**
* Close a task that we still own. Commit all progress and close the task gracefully.
* Throws an exception if this couldn't be done.
*
* @throws TaskMigratedException all the task has been migrated
* @throws StreamsException fatal error, should close the thread
*/
void closeClean();
/**
* Close a task that we may not own. Discard any uncommitted progress and close the task.
* Never throws an exception, but just makes all attempts to release resources while closing.
*/
void closeDirty();
StateStore getStore(final String name);
Set<TopicPartition> inputPartitions();
/**
* @return any changelog partitions associated with this task
*/
Collection<TopicPartition> changelogPartitions();
boolean hasStateStores();
/**
* @return the offsets of all the changelog partitions associated with this task,
* indicating the current positions of the logged state stores of the task.
*/
Map<TopicPartition, Long> changelogOffsets();
default Map<TopicPartition, Long> purgableOffsets() {
return Collections.emptyMap();
}
default boolean process(final long wallClockTime) {
return false;
}
default boolean commitRequested() {
return false;
}
default boolean maybePunctuateStreamTime() {
return false;
}
default boolean maybePunctuateSystemTime() {
return false;
}
String toString(final String indent);
}

View File

@ -17,6 +17,7 @@
package org.apache.kafka.streams.processor.internals.assignment;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.ConfigException;
@ -24,6 +25,7 @@ import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.internals.QuietStreamsConfig;
import org.apache.kafka.streams.processor.internals.InternalTopicManager;
import org.apache.kafka.streams.processor.internals.StreamsMetadataState;
import org.apache.kafka.streams.processor.internals.TaskManager;
import org.slf4j.Logger;
@ -42,6 +44,7 @@ public final class AssignorConfiguration {
private final org.apache.kafka.streams.processor.PartitionGrouper partitionGrouper;
private final String userEndPoint;
private final TaskManager taskManager;
private final StreamsMetadataState streamsMetadataState;
private final InternalTopicManager internalTopicManager;
private final CopartitionedTopicsEnforcer copartitionedTopicsEnforcer;
private final StreamsConfig streamsConfig;
@ -87,24 +90,63 @@ public final class AssignorConfiguration {
userEndPoint = null;
}
final Object o = configs.get(StreamsConfig.InternalConfig.TASK_MANAGER_FOR_PARTITION_ASSIGNOR);
if (o == null) {
final KafkaException fatalException = new KafkaException("TaskManager is not specified");
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
{
final Object o = configs.get(StreamsConfig.InternalConfig.TASK_MANAGER_FOR_PARTITION_ASSIGNOR);
if (o == null) {
final KafkaException fatalException = new KafkaException("TaskManager is not specified");
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
if (!(o instanceof TaskManager)) {
final KafkaException fatalException = new KafkaException(
String.format("%s is not an instance of %s", o.getClass().getName(), TaskManager.class.getName())
);
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
taskManager = (TaskManager) o;
}
if (!(o instanceof TaskManager)) {
final KafkaException fatalException = new KafkaException(
String.format("%s is not an instance of %s", o.getClass().getName(), TaskManager.class.getName())
);
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
{
final Object o = configs.get(StreamsConfig.InternalConfig.STREAMS_METADATA_STATE_FOR_PARTITION_ASSIGNOR);
if (o == null) {
final KafkaException fatalException = new KafkaException("StreamsMetadataState is not specified");
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
if (!(o instanceof StreamsMetadataState)) {
final KafkaException fatalException = new KafkaException(
String.format("%s is not an instance of %s", o.getClass().getName(), StreamsMetadataState.class.getName())
);
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
streamsMetadataState = (StreamsMetadataState) o;
}
taskManager = (TaskManager) o;
{
final Object o = configs.get(StreamsConfig.InternalConfig.STREAMS_ADMIN_CLIENT);
if (o == null) {
final KafkaException fatalException = new KafkaException("Admin is not specified");
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
if (!(o instanceof Admin)) {
final KafkaException fatalException = new KafkaException(
String.format("%s is not an instance of %s", o.getClass().getName(), Admin.class.getName())
);
log.error(fatalException.getMessage(), fatalException);
throw fatalException;
}
internalTopicManager = new InternalTopicManager((Admin) o, streamsConfig);
}
internalTopicManager = new InternalTopicManager(taskManager.adminClient(), streamsConfig);
copartitionedTopicsEnforcer = new CopartitionedTopicsEnforcer(logPrefix);
}
@ -131,6 +173,10 @@ public final class AssignorConfiguration {
return taskManager;
}
public StreamsMetadataState getStreamsMetadataState() {
return streamsMetadataState;
}
public RebalanceProtocol rebalanceProtocol() {
final String upgradeFrom = streamsConfig.getString(StreamsConfig.UPGRADE_FROM_CONFIG);
if (upgradeFrom != null) {

View File

@ -221,6 +221,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, byte[]>, BulkLoadingSt
}
}
@Override
public void init(final ProcessorContext context,
final StateStore root) {
// open the DB dir

View File

@ -30,10 +30,10 @@ import org.apache.kafka.streams.state.TimestampedWindowStore;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
public class StreamThreadStateStoreProvider {
@ -56,17 +56,17 @@ public class StreamThreadStateStoreProvider {
}
final StreamThread.State state = streamThread.state();
if (storeQueryParams.staleStoresEnabled() ? state.isAlive() : state == StreamThread.State.RUNNING) {
final Map<TaskId, ? extends Task> tasks = storeQueryParams.staleStoresEnabled() ? streamThread.allTasks() : streamThread.activeTasks();
final Iterable<? extends Task> tasks = storeQueryParams.staleStoresEnabled() ? streamThread.allTasks().values() : streamThread.activeTasks();
final List<T> stores = new ArrayList<>();
for (final Task streamTask : tasks.values()) {
if (keyTaskId != null && !keyTaskId.equals(streamTask.id())) {
for (final Task task : tasks) {
if (keyTaskId != null && !keyTaskId.equals(task.id())) {
continue;
}
final StateStore store = streamTask.getStore(storeName);
final StateStore store = task.getStore(storeName);
if (store != null && queryableStoreType.accepts(store)) {
if (!store.isOpen()) {
throw new InvalidStateStoreException(
"Cannot get state store " + storeName + " for task " + streamTask +
"Cannot get state store " + storeName + " for task " + task +
" because the store is not open. " +
"The state store may have migrated to another instances.");
}
@ -92,14 +92,14 @@ public class StreamThreadStateStoreProvider {
return null;
}
final List<String> sourceTopics = internalTopologyBuilder.stateStoreNameToSourceTopics().get(storeName);
final Set<String> sourceTopicsSet = sourceTopics.stream().collect(Collectors.toSet());
final Set<String> sourceTopicsSet = new HashSet<>(sourceTopics);
final Map<Integer, InternalTopologyBuilder.TopicsInfo> topicGroups = internalTopologyBuilder.topicGroups();
for (final Map.Entry<Integer, InternalTopologyBuilder.TopicsInfo> topicGroup : topicGroups.entrySet()) {
if (topicGroup.getValue().sourceTopics.containsAll(sourceTopicsSet)) {
return new TaskId(topicGroup.getKey(), partition.intValue());
return new TaskId(topicGroup.getKey(), partition);
}
}
throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the requested partition " + partition + "is" +
"not available on this instance");
throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the requested partition " +
partition + " is not available on this instance");
}
}

View File

@ -81,6 +81,7 @@ import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.easymock.EasyMock.anyInt;
import static org.easymock.EasyMock.anyLong;
@ -319,9 +320,8 @@ public class KafkaStreamsTest {
return null;
}).anyTimes();
EasyMock.expect(thread.allStandbyTasks()).andStubReturn(Collections.emptyList());
EasyMock.expect(thread.restoringTaskIds()).andStubReturn(Collections.emptySet());
EasyMock.expect(thread.allStreamsTasks()).andStubReturn(Collections.emptyList());
EasyMock.expect(thread.activeTasks()).andStubReturn(emptyList());
EasyMock.expect(thread.allTasks()).andStubReturn(Collections.emptyMap());
}
@Test

View File

@ -41,6 +41,7 @@ import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.StreamsTestUtils;
import org.apache.kafka.test.TestUtils;
import org.junit.Before;
@ -69,6 +70,7 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({IntegrationTest.class})
public class EosIntegrationTest {
private static final int NUM_BROKERS = 3;
@ -204,8 +206,7 @@ public class EosIntegrationTest {
}
}
private void checkResultPerKey(final List<KeyValue<Long, Long>> result,
final List<KeyValue<Long, Long>> expectedResult) {
private void checkResultPerKey(final List<KeyValue<Long, Long>> result, final List<KeyValue<Long, Long>> expectedResult) {
final Set<Long> allKeys = new HashSet<>();
addAllKeys(allKeys, result);
addAllKeys(allKeys, expectedResult);
@ -216,15 +217,13 @@ public class EosIntegrationTest {
}
private void addAllKeys(final Set<Long> allKeys,
final List<KeyValue<Long, Long>> records) {
private void addAllKeys(final Set<Long> allKeys, final List<KeyValue<Long, Long>> records) {
for (final KeyValue<Long, Long> record : records) {
allKeys.add(record.key);
}
}
private List<KeyValue<Long, Long>> getAllRecordPerKey(final Long key,
final List<KeyValue<Long, Long>> records) {
private List<KeyValue<Long, Long>> getAllRecordPerKey(final Long key, final List<KeyValue<Long, Long>> records) {
final List<KeyValue<Long, Long>> recordsPerKey = new ArrayList<>(records.size());
for (final KeyValue<Long, Long> record : records) {
@ -590,8 +589,7 @@ public class EosIntegrationTest {
}
@Override
public KeyValue<Long, Long> transform(final Long key,
final Long value) {
public KeyValue<Long, Long> transform(final Long key, final Long value) {
if (gcInjected.compareAndSet(true, false)) {
while (doGC) {
try {
@ -769,5 +767,4 @@ public class EosIntegrationTest {
assertTrue(expectedStoreContent.isEmpty());
}
}

View File

@ -17,11 +17,11 @@
package org.apache.kafka.streams.integration;
import kafka.utils.MockTime;
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.common.utils.Utils;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
@ -38,6 +38,7 @@ import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockProcessorSupplier;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.ClassRule;
@ -53,6 +54,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.Assert.assertEquals;
/**
* This test asserts that when Kafka Streams is closing and shuts
* down a StreamThread the closing of the GlobalStreamThread happens
@ -67,8 +69,6 @@ public class GlobalThreadShutDownOrderTest {
private static final int NUM_BROKERS = 1;
private static final Properties BROKER_CONFIG;
private final AtomicInteger closeCounter = new AtomicInteger(0);
private final int expectedCloseCount = 1;
static {
BROKER_CONFIG = new Properties();
@ -76,6 +76,8 @@ public class GlobalThreadShutDownOrderTest {
BROKER_CONFIG.put("transaction.state.log.min.isr", 1);
}
private final AtomicInteger closeCounter = new AtomicInteger(0);
@ClassRule
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG);
@ -123,14 +125,13 @@ public class GlobalThreadShutDownOrderTest {
}
@After
public void whenShuttingDown() throws Exception {
public void after() throws Exception {
if (kafkaStreams != null) {
kafkaStreams.close();
}
IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
}
@Test
public void shouldFinishGlobalStoreOperationOnShutDown() throws Exception {
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration);
@ -148,7 +149,7 @@ public class GlobalThreadShutDownOrderTest {
final List<Long> expectedRetrievedValues = Arrays.asList(1L, 2L, 3L, 4L);
assertEquals(expectedRetrievedValues, retrievedValuesList);
assertEquals(expectedCloseCount, closeCounter.get());
assertEquals(1, closeCounter.get());
}

View File

@ -303,9 +303,10 @@ public class LagFetchIntegrationTest {
assertThat(fullLagInfo.endOffsetPosition(), equalTo(5L));
assertThat(fullLagInfo.offsetLag(), equalTo(5L));
assertThat(zeroLagInfo, equalTo(restoreEndLagInfo.get(stateStoreName).get(0)));
assertThat(restoreEndLagInfo.get(stateStoreName).get(0), equalTo(zeroLagInfo));
} finally {
streams.close();
streams.cleanUp();
}
}
}

View File

@ -18,9 +18,7 @@ package org.apache.kafka.streams.integration;
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
@ -31,7 +29,6 @@ import java.util.List;
import java.util.Properties;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.kafka.clients.consumer.ConsumerConfig;
@ -88,39 +85,6 @@ public class OptimizedKTableIntegrationTest {
}
}
@Test
public void standbyShouldNotPerformRestoreAtStartup() throws Exception {
final int numMessages = 10;
final int key = 1;
final Semaphore semaphore = new Semaphore(0);
final StreamsBuilder builder = new StreamsBuilder();
builder
.table(INPUT_TOPIC_NAME, Consumed.with(Serdes.Integer(), Serdes.Integer()),
Materialized.<Integer, Integer, KeyValueStore<Bytes, byte[]>>as(TABLE_NAME)
.withCachingDisabled())
.toStream()
.peek((k, v) -> semaphore.release());
final KafkaStreams kafkaStreams1 = createKafkaStreams(builder, streamsConfiguration());
final KafkaStreams kafkaStreams2 = createKafkaStreams(builder, streamsConfiguration());
final List<KafkaStreams> kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2);
produceValueRange(key, 0, 10);
final AtomicLong restoreStartOffset = new AtomicLong(-1);
kafkaStreamsList.forEach(kafkaStreams -> {
kafkaStreams.setGlobalStateRestoreListener(createTrackingRestoreListener(restoreStartOffset, new AtomicLong()));
});
startApplicationAndWaitUntilRunning(kafkaStreamsList, Duration.ofSeconds(60));
// Assert that all messages in the first batch were processed in a timely manner
assertThat(semaphore.tryAcquire(numMessages, 60, TimeUnit.SECONDS), is(equalTo(true)));
// Assert that no restore occurred
assertThat(restoreStartOffset.get(), is(equalTo(-1L)));
}
@Test
public void shouldApplyUpdatesToStandbyStore() throws Exception {
final int batch1NumMessages = 100;
@ -139,11 +103,10 @@ public class OptimizedKTableIntegrationTest {
final KafkaStreams kafkaStreams1 = createKafkaStreams(builder, streamsConfiguration());
final KafkaStreams kafkaStreams2 = createKafkaStreams(builder, streamsConfiguration());
final List<KafkaStreams> kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2);
final TrackingStateRestoreListener listener = new TrackingStateRestoreListener();
final AtomicLong restoreStartOffset = new AtomicLong(-1L);
final AtomicLong restoreEndOffset = new AtomicLong(-1L);
kafkaStreamsList.forEach(kafkaStreams -> {
kafkaStreams.setGlobalStateRestoreListener(createTrackingRestoreListener(restoreStartOffset, restoreEndOffset));
kafkaStreams.setGlobalStateRestoreListener(listener);
});
startApplicationAndWaitUntilRunning(kafkaStreamsList, Duration.ofSeconds(60));
@ -171,7 +134,8 @@ public class OptimizedKTableIntegrationTest {
// Assert that no restore has occurred, ensures that when we check later that the restore
// notification actually came from after the rebalance.
assertThat(restoreStartOffset.get(), is(equalTo(-1L)));
assertThat(listener.startOffset, is(equalTo(0L)));
assertThat(listener.totalNumRestored, is(equalTo(0L)));
// Assert that the current value in store reflects all messages being processed
assertThat(kafkaStreams1WasFirstActive ? store1.get(key) : store2.get(key), is(equalTo(batch1NumMessages - 1)));
@ -182,8 +146,7 @@ public class OptimizedKTableIntegrationTest {
kafkaStreams2.close();
}
final ReadOnlyKeyValueStore<Integer, Integer> newActiveStore =
kafkaStreams1WasFirstActive ? store2 : store1;
final ReadOnlyKeyValueStore<Integer, Integer> newActiveStore = kafkaStreams1WasFirstActive ? store2 : store1;
TestUtils.retryOnExceptionWithTimeout(100, 60 * 1000, () -> {
// Assert that after failover we have recovered to the last store write
assertThat(newActiveStore.get(key), is(equalTo(batch1NumMessages - 1)));
@ -196,12 +159,6 @@ public class OptimizedKTableIntegrationTest {
// Assert that all messages in the second batch were processed in a timely manner
assertThat(semaphore.tryAcquire(batch2NumMessages, 60, TimeUnit.SECONDS), is(equalTo(true)));
// Assert that either restore was unnecessary or we restored from an offset later than 0
assertThat(restoreStartOffset.get(), is(anyOf(greaterThan(0L), equalTo(-1L))));
// Assert that either restore was unnecessary or we restored to the last offset before we closed the kafkaStreams
assertThat(restoreEndOffset.get(), is(anyOf(equalTo(batch1NumMessages - 1L), equalTo(-1L))));
// Assert that the current value in store reflects all messages being processed
assertThat(newActiveStore.get(key), is(equalTo(totalNumMessages - 1)));
}
@ -227,30 +184,32 @@ public class OptimizedKTableIntegrationTest {
return streams;
}
private StateRestoreListener createTrackingRestoreListener(final AtomicLong restoreStartOffset,
final AtomicLong restoreEndOffset) {
return new StateRestoreListener() {
@Override
public void onRestoreStart(final TopicPartition topicPartition,
final String storeName,
final long startingOffset,
final long endingOffset) {
restoreStartOffset.set(startingOffset);
restoreEndOffset.set(endingOffset);
}
private class TrackingStateRestoreListener implements StateRestoreListener {
long startOffset = -1L;
long endOffset = -1L;
long totalNumRestored = 0L;
@Override
public void onBatchRestored(final TopicPartition topicPartition, final String storeName,
final long batchEndOffset, final long numRestored) {
@Override
public void onRestoreStart(final TopicPartition topicPartition,
final String storeName,
final long startingOffset,
final long endingOffset) {
startOffset = startingOffset;
endOffset = endingOffset;
}
}
@Override
public void onBatchRestored(final TopicPartition topicPartition,
final String storeName,
final long batchEndOffset,
final long numRestored) {
totalNumRestored += numRestored;
}
@Override
public void onRestoreEnd(final TopicPartition topicPartition, final String storeName,
final long totalRestored) {
@Override
public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) {
}
};
}
}
private Properties streamsConfiguration() {

View File

@ -93,7 +93,7 @@ public class ResetPartitionTimeIntegrationTest {
}
@Test
public void shouldPreservePartitionTimeOnKafkaStreamRestart() throws Exception {
public void shouldPreservePartitionTimeOnKafkaStreamRestart() {
final String appId = "appId";
final String input = "input";
final String outputRaw = "output-raw";

View File

@ -129,10 +129,11 @@ public class RestoreIntegrationTest {
setCommittedOffset(INPUT_STREAM, offsetLimitDelta);
final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true);
// note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1
new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 0)), ".checkpoint"))
.write(Collections.singletonMap(new TopicPartition(INPUT_STREAM, 0), (long) offsetCheckpointed));
.write(Collections.singletonMap(new TopicPartition(INPUT_STREAM, 0), (long) offsetCheckpointed - 1));
new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 1)), ".checkpoint"))
.write(Collections.singletonMap(new TopicPartition(INPUT_STREAM, 1), (long) offsetCheckpointed));
.write(Collections.singletonMap(new TopicPartition(INPUT_STREAM, 1), (long) offsetCheckpointed - 1));
final CountDownLatch startupLatch = new CountDownLatch(1);
final CountDownLatch shutdownLatch = new CountDownLatch(1);
@ -191,10 +192,11 @@ public class RestoreIntegrationTest {
createStateForRestoration(INPUT_STREAM);
final StateDirectory stateDirectory = new StateDirectory(new StreamsConfig(props), new MockTime(), true);
// note here the checkpointed offset is the last processed record's offset, so without control message we should write this offset - 1
new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 0)), ".checkpoint"))
.write(Collections.singletonMap(new TopicPartition(APPID + "-store-changelog", 0), (long) offsetCheckpointed));
.write(Collections.singletonMap(new TopicPartition(APPID + "-store-changelog", 0), (long) offsetCheckpointed - 1));
new OffsetCheckpoint(new File(stateDirectory.directoryForTask(new TaskId(0, 1)), ".checkpoint"))
.write(Collections.singletonMap(new TopicPartition(APPID + "-store-changelog", 1), (long) offsetCheckpointed));
.write(Collections.singletonMap(new TopicPartition(APPID + "-store-changelog", 1), (long) offsetCheckpointed - 1));
final CountDownLatch startupLatch = new CountDownLatch(1);
final CountDownLatch shutdownLatch = new CountDownLatch(1);

View File

@ -22,6 +22,7 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.apache.kafka.streams.tests.SmokeTestClient;
import org.apache.kafka.streams.tests.SmokeTestDriver;
import org.apache.kafka.test.IntegrationTest;
import org.junit.Assert;
import org.junit.ClassRule;
import org.junit.Test;
@ -142,5 +143,4 @@ public class SmokeTestDriverIntegrationTest {
}
Assert.assertTrue(driver.result().result(), driver.result().passed());
}
}

View File

@ -239,15 +239,18 @@ public class StoreQueryIntegrationTest {
assertThat(semaphore.tryAcquire(batch1NumMessages, 60, TimeUnit.SECONDS), is(equalTo(true)));
final QueryableStoreType<ReadOnlyKeyValueStore<Integer, Integer>> queryableStoreType = QueryableStoreTypes.keyValueStore();
final ReadOnlyKeyValueStore<Integer, Integer> store1 = kafkaStreams1
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores());
final ReadOnlyKeyValueStore<Integer, Integer> store2 = kafkaStreams2
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores());
// Assert that both active and standby are able to query for a key
assertThat(store1.get(key), is(notNullValue()));
assertThat(store2.get(key), is(notNullValue()));
TestUtils.waitForCondition(() -> {
final ReadOnlyKeyValueStore<Integer, Integer> store1 = kafkaStreams1
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores());
return store1.get(key) != null;
}, "store1 cannot find results for key");
TestUtils.waitForCondition(() -> {
final ReadOnlyKeyValueStore<Integer, Integer> store2 = kafkaStreams2
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores());
return store2.get(key) != null;
}, "store2 cannot find results for key");
}
@Test
@ -281,16 +284,18 @@ public class StoreQueryIntegrationTest {
//key doesn't belongs to this partition
final int keyDontBelongPartition = (keyPartition == 0) ? 1 : 0;
final QueryableStoreType<ReadOnlyKeyValueStore<Integer, Integer>> queryableStoreType = QueryableStoreTypes.keyValueStore();
final ReadOnlyKeyValueStore<Integer, Integer> store1 = kafkaStreams1
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores().withPartition(keyPartition));
final ReadOnlyKeyValueStore<Integer, Integer> store2 = kafkaStreams2
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores().withPartition(keyPartition));
// Assert that both active and standby are able to query for a key
assertThat(store1.get(key), is(notNullValue()));
assertThat(store2.get(key), is(notNullValue()));
TestUtils.waitForCondition(() -> {
final ReadOnlyKeyValueStore<Integer, Integer> store1 = kafkaStreams1
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores().withPartition(keyPartition));
return store1.get(key) != null;
}, "store1 cannot find results for key");
TestUtils.waitForCondition(() -> {
final ReadOnlyKeyValueStore<Integer, Integer> store2 = kafkaStreams2
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores().withPartition(keyPartition));
return store2.get(key) != null;
}, "store2 cannot find results for key");
final ReadOnlyKeyValueStore<Integer, Integer> store3 = kafkaStreams1
.store(StoreQueryParams.fromNameAndType(TABLE_NAME, queryableStoreType).enableStaleStores().withPartition(keyDontBelongPartition));

View File

@ -18,7 +18,6 @@ package org.apache.kafka.streams.integration;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.State;
import org.apache.kafka.streams.StoreQueryParams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
@ -31,6 +30,7 @@ import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.test.TestRecord;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockMapper;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -87,7 +87,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), STREAMS_CONFIG)) {
kafkaStreams.setStateListener((newState, oldState) -> {
if (newState == State.RUNNING) {
if (newState == KafkaStreams.State.RUNNING) {
latch.countDown();
}
});
@ -99,7 +99,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testInner() throws Exception {
public void testInner() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
@ -140,7 +140,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testInnerRepartitioned() throws Exception {
public void testInnerRepartitioned() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-repartitioned");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
@ -184,7 +184,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testLeft() throws Exception {
public void testLeft() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-left");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
@ -225,7 +225,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testLeftRepartitioned() throws Exception {
public void testLeftRepartitioned() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-left-repartitioned");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
@ -269,7 +269,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testOuter() throws Exception {
public void testOuter() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-outer");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
@ -310,7 +310,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testOuterRepartitioned() throws Exception {
public void testOuterRepartitioned() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-outer");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
@ -354,7 +354,7 @@ public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest
}
@Test
public void testMultiInner() throws Exception {
public void testMultiInner() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-multi-inner");
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(

View File

@ -36,6 +36,7 @@ import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.common.utils.Utils.mkProperties;
import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION;
import static org.apache.kafka.test.TestUtils.retryOnExceptionWithTimeout;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;

View File

@ -174,8 +174,8 @@ public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest {
if (cacheEnabled) {
runTestWithDriver(expectedFinalMultiJoinResult, storeName);
} else {
// FIXME: the duplicate below for all the multi-joins
// are due to KAFKA-6443, should be updated once it is fixed.
// TODO K6443: the duplicate below for all the multi-joins are due to
// KAFKA-6443, should be updated once it is fixed.
final List<List<TestRecord<Long, String>>> expectedResult = Arrays.asList(
null,
null,

View File

@ -280,8 +280,6 @@ public class SessionWindowedCogroupedKStreamImplTest {
}
@Test
public void sessionWindowMixAggregatorsManyWindowsTest() {
final KTable<Windowed<String>, String> customers = windowedCogroupedStream.aggregate(

View File

@ -1,242 +0,0 @@
/*
* 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.processor.internals;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.common.TopicPartition;
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.StreamsConfig;
import org.apache.kafka.streams.errors.LockException;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.MockRestoreCallback;
import org.apache.kafka.test.MockStateRestoreListener;
import org.apache.kafka.test.TestUtils;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import static org.apache.kafka.streams.processor.internals.ProcessorTopologyFactories.withLocalStores;
import static org.easymock.EasyMock.expect;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class AbstractTaskTest {
private final TaskId id = new TaskId(0, 0);
private StateDirectory stateDirectory = EasyMock.createMock(StateDirectory.class);
private final TopicPartition storeTopicPartition1 = new TopicPartition("t1", 0);
private final TopicPartition storeTopicPartition2 = new TopicPartition("t2", 0);
private final TopicPartition storeTopicPartition3 = new TopicPartition("t3", 0);
private final TopicPartition storeTopicPartition4 = new TopicPartition("t4", 0);
private final Set<TopicPartition> storeTopicPartitions =
Utils.mkSet(storeTopicPartition1, storeTopicPartition2, storeTopicPartition3, storeTopicPartition4);
@Before
public void before() {
expect(stateDirectory.directoryForTask(id)).andReturn(TestUtils.tempDirectory());
}
@Test
public void shouldThrowLockExceptionIfFailedToLockStateDirectoryWhenTopologyHasStores() throws IOException {
final Consumer consumer = EasyMock.createNiceMock(Consumer.class);
final StateStore store = EasyMock.createNiceMock(StateStore.class);
expect(store.name()).andReturn("dummy-store-name").anyTimes();
EasyMock.replay(store);
expect(stateDirectory.lock(id)).andReturn(false);
EasyMock.replay(stateDirectory);
final AbstractTask task = createTask(consumer, Collections.singletonMap(store, "dummy"));
try {
task.registerStateStores();
fail("Should have thrown LockException");
} catch (final LockException e) {
// ok
}
}
@Test
public void shouldNotAttemptToLockIfNoStores() {
final Consumer consumer = EasyMock.createNiceMock(Consumer.class);
EasyMock.replay(stateDirectory);
final AbstractTask task = createTask(consumer, Collections.<StateStore, String>emptyMap());
task.registerStateStores();
// should fail if lock is called
EasyMock.verify(stateDirectory);
}
@Test
public void shouldDeleteAndRecreateStoreDirectoryOnReinitialize() throws IOException {
final StreamsConfig streamsConfig = new StreamsConfig(new Properties() {
{
put(StreamsConfig.APPLICATION_ID_CONFIG, "app-id");
put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath());
}
});
final Consumer consumer = EasyMock.createNiceMock(Consumer.class);
final StateStore store1 = EasyMock.createNiceMock(StateStore.class);
final StateStore store2 = EasyMock.createNiceMock(StateStore.class);
final StateStore store3 = EasyMock.createNiceMock(StateStore.class);
final StateStore store4 = EasyMock.createNiceMock(StateStore.class);
final String storeName1 = "storeName1";
final String storeName2 = "storeName2";
final String storeName3 = "storeName3";
final String storeName4 = "storeName4";
expect(store1.name()).andReturn(storeName1).anyTimes();
EasyMock.replay(store1);
expect(store2.name()).andReturn(storeName2).anyTimes();
EasyMock.replay(store2);
expect(store3.name()).andReturn(storeName3).anyTimes();
EasyMock.replay(store3);
expect(store4.name()).andReturn(storeName4).anyTimes();
EasyMock.replay(store4);
final StateDirectory stateDirectory = new StateDirectory(streamsConfig, new MockTime(), true);
final AbstractTask task = createTask(
consumer,
new HashMap<StateStore, String>() {
{
put(store1, storeTopicPartition1.topic());
put(store2, storeTopicPartition2.topic());
put(store3, storeTopicPartition3.topic());
put(store4, storeTopicPartition4.topic());
}
},
stateDirectory);
final String taskDir = stateDirectory.directoryForTask(task.id).getAbsolutePath();
final File storeDirectory1 = new File(taskDir
+ File.separator + "rocksdb"
+ File.separator + storeName1);
final File storeDirectory2 = new File(taskDir
+ File.separator + "rocksdb"
+ File.separator + storeName2);
final File storeDirectory3 = new File(taskDir
+ File.separator + storeName3);
final File storeDirectory4 = new File(taskDir
+ File.separator + storeName4);
final File testFile1 = new File(storeDirectory1.getAbsolutePath() + File.separator + "testFile");
final File testFile2 = new File(storeDirectory2.getAbsolutePath() + File.separator + "testFile");
final File testFile3 = new File(storeDirectory3.getAbsolutePath() + File.separator + "testFile");
final File testFile4 = new File(storeDirectory4.getAbsolutePath() + File.separator + "testFile");
storeDirectory1.mkdirs();
storeDirectory2.mkdirs();
storeDirectory3.mkdirs();
storeDirectory4.mkdirs();
testFile1.createNewFile();
assertTrue(testFile1.exists());
testFile2.createNewFile();
assertTrue(testFile2.exists());
testFile3.createNewFile();
assertTrue(testFile3.exists());
testFile4.createNewFile();
assertTrue(testFile4.exists());
task.processorContext = new InternalMockProcessorContext(stateDirectory.directoryForTask(task.id), streamsConfig);
task.stateMgr.register(store1, new MockRestoreCallback());
task.stateMgr.register(store2, new MockRestoreCallback());
task.stateMgr.register(store3, new MockRestoreCallback());
task.stateMgr.register(store4, new MockRestoreCallback());
// only reinitialize store1 and store3 -- store2 and store4 should be untouched
task.reinitializeStateStoresForPartitions(Utils.mkSet(storeTopicPartition1, storeTopicPartition3));
assertFalse(testFile1.exists());
assertTrue(testFile2.exists());
assertFalse(testFile3.exists());
assertTrue(testFile4.exists());
}
private AbstractTask createTask(final Consumer consumer,
final Map<StateStore, String> stateStoresToChangelogTopics) {
return createTask(consumer, stateStoresToChangelogTopics, stateDirectory);
}
@SuppressWarnings("unchecked")
private AbstractTask createTask(final Consumer consumer,
final Map<StateStore, String> stateStoresToChangelogTopics,
final StateDirectory stateDirectory) {
final Properties properties = new Properties();
properties.put(StreamsConfig.APPLICATION_ID_CONFIG, "app");
properties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummyhost:9092");
final StreamsConfig config = new StreamsConfig(properties);
final Map<String, String> storeNamesToChangelogTopics = new HashMap<>(stateStoresToChangelogTopics.size());
for (final Map.Entry<StateStore, String> e : stateStoresToChangelogTopics.entrySet()) {
storeNamesToChangelogTopics.put(e.getKey().name(), e.getValue());
}
return new AbstractTask(id,
storeTopicPartitions,
withLocalStores(new ArrayList<>(stateStoresToChangelogTopics.keySet()),
storeNamesToChangelogTopics),
consumer,
new StoreChangelogReader(consumer,
Duration.ZERO,
new MockStateRestoreListener(),
new LogContext("stream-task-test ")),
false,
stateDirectory,
config) {
@Override
public void initializeMetadata() {}
@Override
public void resume() {}
@Override
public void commit() {}
@Override
public void close(final boolean clean, final boolean isZombie) {}
@Override
public boolean initializeStateStores() {
return false;
}
@Override
public void initializeTopology() {}
};
}
}

View File

@ -1,705 +0,0 @@
/*
* 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.processor.internals;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.MetricConfig;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
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.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.test.MockSourceNode;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import org.junit.function.ThrowingRunnable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsEqual.equalTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class AssignedStreamsTasksTest {
private final StreamTask t1 = EasyMock.createMock(StreamTask.class);
private final StreamTask t2 = EasyMock.createMock(StreamTask.class);
private final TopicPartition tp1 = new TopicPartition("t1", 0);
private final TopicPartition tp2 = new TopicPartition("t2", 0);
private final TopicPartition changeLog1 = new TopicPartition("cl1", 0);
private final TopicPartition changeLog2 = new TopicPartition("cl2", 0);
private final TaskId taskId1 = new TaskId(0, 0);
private final TaskId taskId2 = new TaskId(1, 0);
private AssignedStreamsTasks assignedTasks;
private final List<TopicPartition> revokedChangelogs = new ArrayList<>();
@Before
public void before() {
assignedTasks = new AssignedStreamsTasks(new LogContext("log "));
EasyMock.expect(t1.id()).andReturn(taskId1).anyTimes();
EasyMock.expect(t2.id()).andReturn(taskId2).anyTimes();
revokedChangelogs.clear();
}
@Test
public void shouldInitializeNewTasks() {
t1.initializeMetadata();
EasyMock.expect(t1.initializeStateStores()).andReturn(false);
EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()).anyTimes();
EasyMock.replay(t1);
addAndInitTask();
EasyMock.verify(t1);
}
@Test
public void shouldMoveInitializedTasksNeedingRestoreToRestoring() {
t1.initializeMetadata();
EasyMock.expect(t1.initializeStateStores()).andReturn(false);
t1.initializeTopology();
EasyMock.expectLastCall().once();
EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()).anyTimes();
t2.initializeMetadata();
EasyMock.expect(t2.initializeStateStores()).andReturn(true);
t2.initializeTopology();
EasyMock.expectLastCall().once();
final Set<TopicPartition> t2partitions = Collections.singleton(tp2);
EasyMock.expect(t2.partitions()).andReturn(t2partitions).anyTimes();
EasyMock.expect(t2.changelogPartitions()).andReturn(Collections.emptyList()).anyTimes();
EasyMock.replay(t1, t2);
assignedTasks.addNewTask(t1);
assignedTasks.addNewTask(t2);
assignedTasks.initializeNewTasks();
final Collection<StreamTask> restoring = assignedTasks.restoringTasks();
assertThat(restoring.size(), equalTo(1));
assertSame(restoring.iterator().next(), t1);
}
@Test
public void shouldMoveInitializedTasksThatDontNeedRestoringToRunning() {
t2.initializeMetadata();
EasyMock.expect(t2.initializeStateStores()).andReturn(true);
t2.initializeTopology();
EasyMock.expectLastCall().once();
EasyMock.expect(t2.partitions()).andReturn(Collections.singleton(tp2)).anyTimes();
EasyMock.expect(t2.changelogPartitions()).andReturn(Collections.emptyList()).anyTimes();
EasyMock.replay(t2);
assignedTasks.addNewTask(t2);
assignedTasks.initializeNewTasks();
assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.singleton(taskId2)));
}
@Test
public void shouldTransitionFullyRestoredTasksToRunning() {
final Set<TopicPartition> task1Partitions = Utils.mkSet(tp1);
t1.initializeMetadata();
EasyMock.expect(t1.initializeStateStores()).andReturn(false);
EasyMock.expect(t1.partitions()).andReturn(task1Partitions).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andReturn(Utils.mkSet(changeLog1, changeLog2)).anyTimes();
EasyMock.expect(t1.hasStateStores()).andReturn(true).anyTimes();
t1.initializeTopology();
EasyMock.expectLastCall().once();
EasyMock.replay(t1);
addAndInitTask();
assignedTasks.updateRestored(Utils.mkSet(changeLog1));
assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.<TaskId>emptySet()));
assignedTasks.updateRestored(Utils.mkSet(changeLog2));
assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.singleton(taskId1)));
}
@Test
public void shouldSuspendRunningTasks() {
mockRunningTaskSuspension();
EasyMock.replay(t1);
assertThat(suspendTask(), nullValue());
assertThat(assignedTasks.suspendedTaskIds(), equalTo(Collections.singleton(taskId1)));
EasyMock.verify(t1);
}
@Test
public void shouldCloseRestoringTasks() {
t1.initializeMetadata();
EasyMock.expect(t1.initializeStateStores()).andReturn(false);
EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptySet()).anyTimes();
t1.closeStateManager(true);
EasyMock.expectLastCall();
EasyMock.replay(t1);
addAndInitTask();
assertThat(assignedTasks.closeRestoringTasks(assignedTasks.restoringTaskIds(), revokedChangelogs), nullValue());
EasyMock.verify(t1);
}
@Test
public void shouldCloseUnInitializedTasksOnSuspend() {
EasyMock.expect(t1.partitions()).andAnswer(Collections::emptySet).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andAnswer(Collections::emptyList).anyTimes();
t1.close(false, false);
EasyMock.expectLastCall();
EasyMock.replay(t1);
assignedTasks.addNewTask(t1);
assertThat(assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs), nullValue());
EasyMock.verify(t1);
}
@Test
public void shouldNotSuspendSuspendedTasks() {
mockRunningTaskSuspension();
EasyMock.replay(t1);
assertThat(suspendTask(), nullValue());
assertThat(assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs), nullValue());
EasyMock.verify(t1);
}
@Test
public void shouldCloseTaskOnSuspendWhenRuntimeException() {
mockTaskInitialization();
t1.suspend();
EasyMock.expectLastCall().andThrow(new RuntimeException("KABOOM!"));
t1.close(false, false);
EasyMock.expectLastCall();
EasyMock.replay(t1);
assertThat(suspendTask(), not(nullValue()));
assertTrue(assignedTasks.runningTaskIds().isEmpty());
assertTrue(assignedTasks.suspendedTaskIds().isEmpty());
EasyMock.verify(t1);
}
@Test
public void shouldCloseTaskOnSuspendIfTaskMigratedException() {
mockTaskInitialization();
t1.suspend();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
t1.close(false, true);
EasyMock.expectLastCall().andThrow(new RuntimeException("any exception"));
EasyMock.replay(t1);
assertThat(suspendTask(), nullValue());
assertTrue(assignedTasks.runningTaskIds().isEmpty());
EasyMock.verify(t1);
}
@Test
public void shouldCloseUncleanAndThenRethrowOnShutdownIfRuntimeException() {
mockTaskInitialization();
t1.close(true, false);
EasyMock.expectLastCall().andThrow(new RuntimeException("any first exception"));
t1.close(false, false);
EasyMock.expectLastCall().andThrow(new RuntimeException("any second exception"));
EasyMock.replay(t1);
addAndInitTask();
try {
assignedTasks.shutdown(true);
fail("expected a runtime exception");
} catch (final RuntimeException e) {
assertThat(e.getMessage(), is("any first exception"));
}
}
@Test
public void shouldCloseWithoutExceptionOnShutdownIfTaskMigratedException() {
mockTaskInitialization();
t1.close(true, false);
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
t1.close(false, true);
EasyMock.expectLastCall().andThrow(new RuntimeException("any second exception"));
EasyMock.replay(t1);
addAndInitTask();
assignedTasks.shutdown(true);
}
@Test
public void shouldResumeMatchingSuspendedTasks() {
mockRunningTaskSuspension();
t1.resume();
EasyMock.expectLastCall();
t1.initializeTopology();
EasyMock.expectLastCall().once();
EasyMock.replay(t1);
assertThat(suspendTask(), nullValue());
assertTrue(assignedTasks.maybeResumeSuspendedTask(taskId1, Collections.singleton(tp1)));
assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.singleton(taskId1)));
EasyMock.verify(t1);
}
@Test
public void shouldNotCloseTaskWithinResumeSuspendedIfTaskMigratedException() {
mockRunningTaskSuspension();
t1.resume();
t1.initializeTopology();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
EasyMock.replay(t1);
assertThat(suspendTask(), nullValue());
verifyTaskMigratedExceptionDoesNotCloseTask(
() -> assignedTasks.maybeResumeSuspendedTask(taskId1, Collections.singleton(tp1)));
}
private void mockTaskInitialization() {
t1.initializeMetadata();
EasyMock.expect(t1.initializeStateStores()).andReturn(true);
t1.initializeTopology();
EasyMock.expectLastCall().once();
EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptyList()).anyTimes();
}
@Test
public void shouldCommitRunningTasks() {
mockTaskInitialization();
EasyMock.expect(t1.commitNeeded()).andReturn(true);
t1.commit();
EasyMock.expectLastCall();
EasyMock.replay(t1);
addAndInitTask();
assignedTasks.commit();
EasyMock.verify(t1);
}
@Test
public void shouldNotCloseTaskWithinCommitIfTaskMigratedException() {
mockTaskInitialization();
EasyMock.expect(t1.commitNeeded()).andReturn(true);
t1.commit();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
EasyMock.replay(t1);
addAndInitTask();
verifyTaskMigratedExceptionDoesNotCloseTask(
() -> assignedTasks.commit());
}
@Test
public void shouldThrowExceptionOnCommitWhenNotCommitFailedOrProducerFenced() {
mockTaskInitialization();
EasyMock.expect(t1.commitNeeded()).andReturn(true);
t1.commit();
EasyMock.expectLastCall().andThrow(new RuntimeException(""));
EasyMock.replay(t1);
addAndInitTask();
try {
assignedTasks.commit();
fail("Should have thrown exception");
} catch (final Exception e) {
// ok
}
assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.singleton(taskId1)));
EasyMock.verify(t1);
}
@Test
public void shouldCommitRunningTasksIfNeeded() {
mockTaskInitialization();
EasyMock.expect(t1.commitRequested()).andReturn(true);
EasyMock.expect(t1.commitNeeded()).andReturn(true);
t1.commit();
EasyMock.expectLastCall();
EasyMock.replay(t1);
addAndInitTask();
assertThat(assignedTasks.maybeCommitPerUserRequested(), equalTo(1));
EasyMock.verify(t1);
}
@Test
public void shouldNotCloseTaskWithinMaybeCommitIfTaskMigratedException() {
mockTaskInitialization();
EasyMock.expect(t1.commitRequested()).andReturn(true);
EasyMock.expect(t1.commitNeeded()).andReturn(true);
t1.commit();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
EasyMock.replay(t1);
addAndInitTask();
verifyTaskMigratedExceptionDoesNotCloseTask(
() -> assignedTasks.maybeCommitPerUserRequested());
}
@Test
public void shouldNotCloseTaskWithinProcessIfTaskMigratedException() {
mockTaskInitialization();
EasyMock.expect(t1.isProcessable(0L)).andReturn(true);
t1.process();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
EasyMock.replay(t1);
addAndInitTask();
verifyTaskMigratedExceptionDoesNotCloseTask(
() -> assignedTasks.process(0L));
}
@Test
public void shouldNotProcessUnprocessableTasks() {
mockTaskInitialization();
EasyMock.expect(t1.isProcessable(0L)).andReturn(false);
EasyMock.replay(t1);
addAndInitTask();
assertThat(assignedTasks.process(0L), equalTo(0));
EasyMock.verify(t1);
}
@Test
public void shouldAlwaysProcessProcessableTasks() {
mockTaskInitialization();
EasyMock.expect(t1.isProcessable(0L)).andReturn(true);
EasyMock.expect(t1.process()).andReturn(true).once();
EasyMock.replay(t1);
addAndInitTask();
assertThat(assignedTasks.process(0L), equalTo(1));
EasyMock.verify(t1);
}
@Test
public void shouldPunctuateRunningTasks() {
mockTaskInitialization();
EasyMock.expect(t1.maybePunctuateStreamTime()).andReturn(true);
EasyMock.expect(t1.maybePunctuateSystemTime()).andReturn(true);
EasyMock.replay(t1);
addAndInitTask();
assertThat(assignedTasks.punctuate(), equalTo(2));
EasyMock.verify(t1);
}
@Test
public void shouldNotCloseTaskWithinMaybePunctuateStreamTimeIfTaskMigratedException() {
mockTaskInitialization();
t1.maybePunctuateStreamTime();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
EasyMock.replay(t1);
addAndInitTask();
verifyTaskMigratedExceptionDoesNotCloseTask(
() -> assignedTasks.punctuate());
}
@Test
public void shouldNotloseTaskWithinMaybePunctuateSystemTimeIfTaskMigratedException() {
mockTaskInitialization();
EasyMock.expect(t1.maybePunctuateStreamTime()).andReturn(true);
t1.maybePunctuateSystemTime();
EasyMock.expectLastCall().andThrow(new TaskMigratedException());
EasyMock.replay(t1);
addAndInitTask();
try {
assignedTasks.punctuate();
fail("Should have thrown TaskMigratedException.");
} catch (final TaskMigratedException expected) {
assertThat(assignedTasks.runningTaskIds(), equalTo(Collections.singleton(taskId1)));
}
EasyMock.verify(t1);
}
@Test
public void shouldReturnNumberOfPunctuations() {
mockTaskInitialization();
EasyMock.expect(t1.maybePunctuateStreamTime()).andReturn(true);
EasyMock.expect(t1.maybePunctuateSystemTime()).andReturn(false);
EasyMock.replay(t1);
addAndInitTask();
assertThat(assignedTasks.punctuate(), equalTo(1));
EasyMock.verify(t1);
}
@Test
public void shouldCloseCleanlyWithSuspendedTaskAndEOS() {
final String topic = "topic";
final Deserializer<byte[]> deserializer = Serdes.ByteArray().deserializer();
final Serializer<byte[]> serializer = Serdes.ByteArray().serializer();
final MockConsumer<byte[], byte[]> consumer =
new MockConsumer<>(OffsetResetStrategy.EARLIEST);
final MockProducer<byte[], byte[]> producer =
new MockProducer<>(false, serializer, serializer);
final MockSourceNode<byte[], byte[]> source = new MockSourceNode<>(
new String[] {"topic"},
deserializer,
deserializer);
final ChangelogReader changelogReader = new MockChangelogReader();
final ProcessorTopology topology = new ProcessorTopology(
Collections.singletonList(source),
Collections.singletonMap(topic, source),
Collections.emptyMap(),
Collections.emptyList(),
Collections.emptyList(),
Collections.emptyMap(),
Collections.emptySet());
final Set<TopicPartition> partitions = Collections.singleton(
new TopicPartition(topic, 1));
final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.DEBUG));
final StreamsMetricsImpl streamsMetrics = new MockStreamsMetrics(metrics);
final MockTime time = new MockTime();
final StateDirectory stateDirectory = new StateDirectory(
StreamTaskTest.createConfig(true),
time,
true);
final StreamTask task = new StreamTask(
new TaskId(0, 0),
partitions,
topology,
consumer,
changelogReader,
StreamTaskTest.createConfig(true),
streamsMetrics,
stateDirectory,
null,
time,
() -> producer);
assignedTasks.addNewTask(task);
assignedTasks.initializeNewTasks();
assertNull(assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs));
assignedTasks.shutdown(true);
}
@Test
public void shouldClearZombieCreatedTasks() {
new TaskTestSuite() {
@Override
public void additionalSetup(final StreamTask task) {
task.close(false, true);
}
@Override
public void action(final StreamTask task) {
assignedTasks.addNewTask(task);
}
@Override
public Set<TaskId> taskIds() {
return assignedTasks.created.keySet();
}
}.createTaskAndClear();
}
@Test
public void shouldClearZombieRestoringTasks() {
new TaskTestSuite() {
@Override
public void additionalSetup(final StreamTask task) {
task.closeStateManager(false);
}
@Override
public void action(final StreamTask task) {
assignedTasks.addTaskToRestoring(task);
}
@Override
public Set<TaskId> taskIds() {
return assignedTasks.restoringTaskIds();
}
}.createTaskAndClear();
}
@Test
public void shouldClearZombieRunningTasks() {
new TaskTestSuite() {
@Override
public void additionalSetup(final StreamTask task) {
task.initializeTopology();
task.close(false, true);
}
@Override
public void action(final StreamTask task) {
assignedTasks.transitionToRunning(task);
}
@Override
public Set<TaskId> taskIds() {
return assignedTasks.runningTaskIds();
}
}.createTaskAndClear();
}
@Test
public void shouldClearZombieSuspendedTasks() {
new TaskTestSuite() {
@Override
public void additionalSetup(final StreamTask task) {
task.initializeTopology();
task.suspend();
task.closeSuspended(false, null);
}
@Override
public void action(final StreamTask task) {
assignedTasks.transitionToRunning(task);
final List<TopicPartition> revokedChangelogs = new ArrayList<>();
final List<TaskId> ids = Collections.singletonList(task.id());
assignedTasks.suspendOrCloseTasks(new HashSet<>(ids), revokedChangelogs);
assertEquals(clearingPartitions, revokedChangelogs);
}
@Override
public Set<TaskId> taskIds() {
return assignedTasks.suspendedTaskIds();
}
}.createTaskAndClear();
}
abstract class TaskTestSuite {
TaskId clearingTaskId = new TaskId(0, 0);
List<TopicPartition> clearingPartitions = Collections.singletonList(new TopicPartition("topic", 0));
abstract void additionalSetup(final StreamTask task);
abstract void action(final StreamTask task);
abstract Set<TaskId> taskIds();
void createTaskAndClear() {
final StreamTask task = EasyMock.createMock(StreamTask.class);
EasyMock.expect(task.id()).andReturn(clearingTaskId).anyTimes();
EasyMock.expect(task.partitions()).andReturn(Collections.emptySet()).anyTimes();
EasyMock.expect(task.changelogPartitions()).andReturn(clearingPartitions).anyTimes();
EasyMock.expect(task.toString(EasyMock.anyString())).andReturn("task").anyTimes();
additionalSetup(task);
EasyMock.replay(task);
action(task);
final Set<TaskId> ids = new HashSet<>(Collections.singleton(task.id()));
assertEquals(ids, taskIds());
assignedTasks.closeAllTasksAsZombies();
assertEquals(Collections.emptySet(), taskIds());
}
}
private void addAndInitTask() {
assignedTasks.addNewTask(t1);
assignedTasks.initializeNewTasks();
}
private RuntimeException suspendTask() {
addAndInitTask();
return assignedTasks.suspendOrCloseTasks(assignedTasks.allAssignedTaskIds(), revokedChangelogs);
}
private void mockRunningTaskSuspension() {
t1.initializeMetadata();
EasyMock.expect(t1.initializeStateStores()).andReturn(true);
t1.initializeTopology();
EasyMock.expectLastCall().once();
EasyMock.expect(t1.hasStateStores()).andReturn(false).anyTimes();
EasyMock.expect(t1.partitions()).andReturn(Collections.singleton(tp1)).anyTimes();
EasyMock.expect(t1.changelogPartitions()).andReturn(Collections.emptyList()).anyTimes();
t1.suspend();
EasyMock.expectLastCall();
}
private void verifyTaskMigratedExceptionDoesNotCloseTask(final ThrowingRunnable action) {
final Set<TaskId> expectedRunningTaskIds = Collections.singleton(taskId1);
// This action is expected to throw a TaskMigratedException
assertThrows(TaskMigratedException.class, action);
// This task should be closed as a zombie with all the other tasks during onPartitionsLost
assertThat(assignedTasks.runningTaskIds(), equalTo(expectedRunningTaskIds));
EasyMock.verify(t1);
}
}

View File

@ -168,7 +168,7 @@ public class GlobalStateManagerImplTest {
final Map<TopicPartition, Long> expected = writeCheckpoint();
stateManager.initialize();
final Map<TopicPartition, Long> offsets = stateManager.checkpointed();
final Map<TopicPartition, Long> offsets = stateManager.changelogOffsets();
assertEquals(expected, offsets);
}
@ -226,7 +226,7 @@ public class GlobalStateManagerImplTest {
stateManager.initialize();
try {
stateManager.register(new NoOpReadOnlyStore<>("not-in-topology"), stateRestoreCallback);
stateManager.registerStore(new NoOpReadOnlyStore<>("not-in-topology"), stateRestoreCallback);
fail("should have raised an illegal argument exception as store is not in the topology");
} catch (final IllegalArgumentException e) {
// pass
@ -237,9 +237,9 @@ public class GlobalStateManagerImplTest {
public void shouldThrowIllegalArgumentExceptionIfAttemptingToRegisterStoreTwice() {
stateManager.initialize();
initializeConsumer(2, 0, t1);
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
try {
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
fail("should have raised an illegal argument exception as store has already been registered");
} catch (final IllegalArgumentException e) {
// pass
@ -250,7 +250,7 @@ public class GlobalStateManagerImplTest {
public void shouldThrowStreamsExceptionIfNoPartitionsFoundForStore() {
stateManager.initialize();
try {
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
fail("Should have raised a StreamsException as there are no partition for the store");
} catch (final StreamsException e) {
// pass
@ -262,7 +262,7 @@ public class GlobalStateManagerImplTest {
initializeConsumer(1, 0, t1);
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
final KeyValue<byte[], byte[]> restoredRecord = stateRestoreCallback.restored.get(0);
assertEquals(3, restoredRecord.key.length);
@ -274,7 +274,7 @@ public class GlobalStateManagerImplTest {
initializeConsumer(1, 0, t1);
stateManager.initialize();
stateManager.register(
stateManager.registerStore(
new WrappedStateStore<NoOpReadOnlyStore<Object, Object>, Object, Object>(store1) {
},
stateRestoreCallback
@ -290,7 +290,7 @@ public class GlobalStateManagerImplTest {
initializeConsumer(1, 0, t2);
stateManager.initialize();
stateManager.register(store2, stateRestoreCallback);
stateManager.registerStore(store2, stateRestoreCallback);
final KeyValue<byte[], byte[]> restoredRecord = stateRestoreCallback.restored.get(0);
assertEquals(3, restoredRecord.key.length);
@ -302,7 +302,7 @@ public class GlobalStateManagerImplTest {
initializeConsumer(1, 0, t2);
stateManager.initialize();
stateManager.register(
stateManager.registerStore(
new WrappedStateStore<NoOpReadOnlyStore<Object, Object>, Object, Object>(store2) {
},
stateRestoreCallback
@ -319,7 +319,7 @@ public class GlobalStateManagerImplTest {
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
assertEquals(2, stateRestoreCallback.restored.size());
}
@ -334,7 +334,7 @@ public class GlobalStateManagerImplTest {
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
assertEquals(2, stateRestoreCallback.restored.size());
}
@ -343,7 +343,7 @@ public class GlobalStateManagerImplTest {
initializeConsumer(5, 1, t1);
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
assertThat(stateRestoreListener.restoreStartOffset, equalTo(1L));
assertThat(stateRestoreListener.restoreEndOffset, equalTo(6L));
@ -364,7 +364,7 @@ public class GlobalStateManagerImplTest {
offsetCheckpoint.write(Collections.singletonMap(t1, 5L));
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
assertEquals(5, stateRestoreCallback.restored.size());
}
@ -374,9 +374,9 @@ public class GlobalStateManagerImplTest {
stateManager.initialize();
// register the stores
initializeConsumer(1, 0, t1);
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
initializeConsumer(1, 0, t2);
stateManager.register(store2, stateRestoreCallback);
stateManager.registerStore(store2, stateRestoreCallback);
stateManager.flush();
assertTrue(store1.flushed);
@ -388,7 +388,7 @@ public class GlobalStateManagerImplTest {
stateManager.initialize();
// register the stores
initializeConsumer(1, 0, t1);
stateManager.register(new NoOpReadOnlyStore(store1.name()) {
stateManager.registerStore(new NoOpReadOnlyStore(store1.name()) {
@Override
public void flush() {
throw new RuntimeException("KABOOM!");
@ -403,11 +403,11 @@ public class GlobalStateManagerImplTest {
stateManager.initialize();
// register the stores
initializeConsumer(1, 0, t1);
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
initializeConsumer(1, 0, t2);
stateManager.register(store2, stateRestoreCallback);
stateManager.registerStore(store2, stateRestoreCallback);
stateManager.close(true);
stateManager.close();
assertFalse(store1.isOpen());
assertFalse(store2.isOpen());
}
@ -416,21 +416,21 @@ public class GlobalStateManagerImplTest {
public void shouldThrowProcessorStateStoreExceptionIfStoreCloseFailed() throws IOException {
stateManager.initialize();
initializeConsumer(1, 0, t1);
stateManager.register(new NoOpReadOnlyStore(store1.name()) {
stateManager.registerStore(new NoOpReadOnlyStore(store1.name()) {
@Override
public void close() {
throw new RuntimeException("KABOOM!");
}
}, stateRestoreCallback);
stateManager.close(true);
stateManager.close();
}
@Test
public void shouldThrowIllegalArgumentExceptionIfCallbackIsNull() {
stateManager.initialize();
try {
stateManager.register(store1, null);
stateManager.registerStore(store1, null);
fail("should have thrown due to null callback");
} catch (final IllegalArgumentException e) {
//pass
@ -440,7 +440,7 @@ public class GlobalStateManagerImplTest {
@Test
public void shouldUnlockGlobalStateDirectoryOnClose() throws IOException {
stateManager.initialize();
stateManager.close(true);
stateManager.close();
final StateDirectory stateDir = new StateDirectory(streamsConfig, new MockTime(), true);
try {
// should be able to get the lock now as it should've been released in close
@ -454,7 +454,7 @@ public class GlobalStateManagerImplTest {
public void shouldNotCloseStoresIfCloseAlreadyCalled() throws IOException {
stateManager.initialize();
initializeConsumer(1, 0, t1);
stateManager.register(new NoOpReadOnlyStore("t1-store") {
stateManager.registerStore(new NoOpReadOnlyStore("t1-store") {
@Override
public void close() {
if (!isOpen()) {
@ -463,9 +463,9 @@ public class GlobalStateManagerImplTest {
super.close();
}
}, stateRestoreCallback);
stateManager.close(true);
stateManager.close();
stateManager.close(true);
stateManager.close();
}
@Test
@ -479,13 +479,13 @@ public class GlobalStateManagerImplTest {
throw new RuntimeException("KABOOM!");
}
};
stateManager.register(store, stateRestoreCallback);
stateManager.registerStore(store, stateRestoreCallback);
initializeConsumer(1, 0, t2);
stateManager.register(store2, stateRestoreCallback);
stateManager.registerStore(store2, stateRestoreCallback);
try {
stateManager.close(true);
stateManager.close();
} catch (final ProcessorStateException e) {
// expected
}
@ -519,21 +519,21 @@ public class GlobalStateManagerImplTest {
final Map<TopicPartition, Long> result = readOffsetsCheckpoint();
assertThat(result, equalTo(offsets));
assertThat(stateManager.checkpointed(), equalTo(offsets));
assertThat(stateManager.changelogOffsets(), equalTo(offsets));
}
@Test
public void shouldNotRemoveOffsetsOfUnUpdatedTablesDuringCheckpoint() {
stateManager.initialize();
initializeConsumer(10, 0, t1);
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
initializeConsumer(20, 0, t2);
stateManager.register(store2, stateRestoreCallback);
stateManager.registerStore(store2, stateRestoreCallback);
final Map<TopicPartition, Long> initialCheckpoint = stateManager.checkpointed();
final Map<TopicPartition, Long> initialCheckpoint = stateManager.changelogOffsets();
stateManager.checkpoint(Collections.singletonMap(t1, 101L));
final Map<TopicPartition, Long> updatedCheckpoint = stateManager.checkpointed();
final Map<TopicPartition, Long> updatedCheckpoint = stateManager.changelogOffsets();
assertThat(updatedCheckpoint.get(t2), equalTo(initialCheckpoint.get(t2)));
assertThat(updatedCheckpoint.get(t1), equalTo(101L));
}
@ -554,7 +554,7 @@ public class GlobalStateManagerImplTest {
consumer.addRecord(new ConsumerRecord<>(t1.topic(), t1.partition(), 2, expectedKey, expectedValue));
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
final KeyValue<byte[], byte[]> restoredKv = stateRestoreCallback.restored.get(0);
assertThat(stateRestoreCallback.restored, equalTo(Collections.singletonList(KeyValue.pair(restoredKv.key, restoredKv.value))));
}
@ -563,11 +563,11 @@ public class GlobalStateManagerImplTest {
public void shouldCheckpointRestoredOffsetsToFile() throws IOException {
stateManager.initialize();
initializeConsumer(10, 0, t1);
stateManager.register(store1, stateRestoreCallback);
stateManager.registerStore(store1, stateRestoreCallback);
stateManager.checkpoint(Collections.emptyMap());
stateManager.close(true);
stateManager.close();
final Map<TopicPartition, Long> checkpointMap = stateManager.checkpointed();
final Map<TopicPartition, Long> checkpointMap = stateManager.changelogOffsets();
assertThat(checkpointMap, equalTo(Collections.singletonMap(t1, 10L)));
assertThat(readOffsetsCheckpoint(), equalTo(checkpointMap));
}
@ -576,8 +576,8 @@ public class GlobalStateManagerImplTest {
public void shouldSkipGlobalInMemoryStoreOffsetsToFile() throws IOException {
stateManager.initialize();
initializeConsumer(10, 0, t3);
stateManager.register(store3, stateRestoreCallback);
stateManager.close(true);
stateManager.registerStore(store3, stateRestoreCallback);
stateManager.close();
assertThat(readOffsetsCheckpoint(), equalTo(Collections.emptyMap()));
}
@ -678,68 +678,6 @@ public class GlobalStateManagerImplTest {
}
}
@Test
public void shouldDeleteAndRecreateStoreDirectoryOnReinitialize() throws IOException {
final File storeDirectory1 = new File(stateDirectory.globalStateDir().getAbsolutePath()
+ File.separator + "rocksdb"
+ File.separator + storeName1);
final File storeDirectory2 = new File(stateDirectory.globalStateDir().getAbsolutePath()
+ File.separator + "rocksdb"
+ File.separator + storeName2);
final File storeDirectory3 = new File(stateDirectory.globalStateDir().getAbsolutePath()
+ File.separator + storeName3);
final File storeDirectory4 = new File(stateDirectory.globalStateDir().getAbsolutePath()
+ File.separator + storeName4);
final File testFile1 = new File(storeDirectory1.getAbsolutePath() + File.separator + "testFile");
final File testFile2 = new File(storeDirectory2.getAbsolutePath() + File.separator + "testFile");
final File testFile3 = new File(storeDirectory3.getAbsolutePath() + File.separator + "testFile");
final File testFile4 = new File(storeDirectory4.getAbsolutePath() + File.separator + "testFile");
consumer.updatePartitions(t1.topic(), Collections.singletonList(new PartitionInfo(t1.topic(), t1.partition(), null, null, null)));
consumer.updatePartitions(t2.topic(), Collections.singletonList(new PartitionInfo(t2.topic(), t2.partition(), null, null, null)));
consumer.updatePartitions(t3.topic(), Collections.singletonList(new PartitionInfo(t3.topic(), t3.partition(), null, null, null)));
consumer.updatePartitions(t4.topic(), Collections.singletonList(new PartitionInfo(t4.topic(), t4.partition(), null, null, null)));
consumer.updateBeginningOffsets(new HashMap<TopicPartition, Long>() {
{
put(t1, 0L);
put(t2, 0L);
put(t3, 0L);
put(t4, 0L);
}
});
consumer.updateEndOffsets(new HashMap<TopicPartition, Long>() {
{
put(t1, 0L);
put(t2, 0L);
put(t3, 0L);
put(t4, 0L);
}
});
stateManager.initialize();
stateManager.register(store1, stateRestoreCallback);
stateManager.register(store2, stateRestoreCallback);
stateManager.register(store3, stateRestoreCallback);
stateManager.register(store4, stateRestoreCallback);
testFile1.createNewFile();
assertTrue(testFile1.exists());
testFile2.createNewFile();
assertTrue(testFile2.exists());
testFile3.createNewFile();
assertTrue(testFile3.exists());
testFile4.createNewFile();
assertTrue(testFile4.exists());
// only delete and recreate store 1 and 3 -- 2 and 4 must be untouched
stateManager.reinitializeStateStoresForPartitions(asList(t1, t3), processorContext);
assertFalse(testFile1.exists());
assertTrue(testFile2.exists());
assertFalse(testFile3.exists());
assertTrue(testFile4.exists());
}
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

@ -211,7 +211,7 @@ public class GlobalStateTaskTest {
globalStateTask.initialize();
globalStateTask.update(new ConsumerRecord<>(topic1, 1, 51, "foo".getBytes(), "foo".getBytes()));
globalStateTask.flushState();
assertEquals(expectedOffsets, stateMgr.checkpointed());
assertEquals(expectedOffsets, stateMgr.changelogOffsets());
}
@Test
@ -222,7 +222,7 @@ public class GlobalStateTaskTest {
globalStateTask.initialize();
globalStateTask.update(new ConsumerRecord<>(topic1, 1, 101, "foo".getBytes(), "foo".getBytes()));
globalStateTask.flushState();
assertThat(stateMgr.checkpointed(), equalTo(expectedOffsets));
assertThat(stateMgr.changelogOffsets(), equalTo(expectedOffsets));
}
}

View File

@ -690,7 +690,7 @@ public class InternalTopologyBuilderTest {
updatedTopics.add("topic-3");
updatedTopics.add("topic-A");
builder.updateSubscribedTopics(updatedTopics, null);
builder.addSubscribedTopicsFromMetadata(updatedTopics, null);
builder.setApplicationId("test-id");
final Map<Integer, InternalTopologyBuilder.TopicsInfo> topicGroups = builder.topicGroups();
@ -772,7 +772,7 @@ public class InternalTopologyBuilderTest {
updatedTopics.add("topic-3");
updatedTopics.add("topic-A");
builder.updateSubscribedTopics(updatedTopics, "test-thread");
builder.addSubscribedTopicsFromMetadata(updatedTopics, "test-thread");
builder.setApplicationId("test-app");
final Map<String, List<String>> stateStoreAndTopics = builder.stateStoreNameToSourceTopics();

View File

@ -16,7 +16,6 @@
*/
package org.apache.kafka.streams.processor.internals;
import java.util.List;
import org.apache.kafka.common.TopicPartition;
import java.util.Collection;
@ -26,46 +25,60 @@ import java.util.Map;
import java.util.Set;
public class MockChangelogReader implements ChangelogReader {
private final Set<TopicPartition> registered = new HashSet<>();
private final Set<TopicPartition> restoringPartitions = new HashSet<>();
private Map<TopicPartition, Long> restoredOffsets = Collections.emptyMap();
@Override
public void register(final StateRestorer restorer) {
registered.add(restorer.partition());
public boolean isPartitionRegistered(final TopicPartition partition) {
return restoringPartitions.contains(partition);
}
@Override
public Collection<TopicPartition> restore(final RestoringTasks active) {
return registered;
public void register(final TopicPartition partition, final ProcessorStateManager stateManager) {
restoringPartitions.add(partition);
}
@Override
public Map<TopicPartition, Long> restoredOffsets() {
return restoredOffsets;
public void restore() {
// do nothing
}
void setRestoredOffsets(final Map<TopicPartition, Long> restoredOffsets) {
this.restoredOffsets = restoredOffsets;
@Override
public void transitToRestoreActive() {
// do nothing
}
@Override
public void transitToUpdateStandby() {
// do nothing
}
@Override
public void updateLimitOffsets() {
// do nothing
}
@Override
public Set<TopicPartition> completedChangelogs() {
// assuming all restoring partitions are completed
return restoringPartitions;
}
@Override
public void clear() {
registered.clear();
restoringPartitions.clear();
}
@Override
public void remove(final List<TopicPartition> revokedPartitions) {
for (final TopicPartition partition : revokedPartitions) {
public void remove(final Collection<TopicPartition> partitions) {
restoringPartitions.removeAll(partitions);
for (final TopicPartition partition : partitions) {
restoredOffsets.remove(partition);
}
}
@Override
public boolean isEmpty() {
return restoredOffsets.isEmpty() && registered.isEmpty();
}
public boolean wasRegistered(final TopicPartition partition) {
return registered.contains(partition);
return restoredOffsets.isEmpty() && restoringPartitions.isEmpty();
}
}

View File

@ -113,7 +113,6 @@ public class ProcessorContextImplTest {
expect(stateManager.getGlobalStore("GlobalTimestampedWindowStore")).andReturn(timestampedWindowStoreMock());
expect(stateManager.getGlobalStore("GlobalSessionStore")).andReturn(sessionStoreMock());
expect(stateManager.getGlobalStore(anyString())).andReturn(null);
expect(stateManager.getStore("LocalKeyValueStore")).andReturn(keyValueStoreMock());
expect(stateManager.getStore("LocalTimestampedKeyValueStore")).andReturn(timestampedKeyValueStoreMock());
expect(stateManager.getStore("LocalWindowStore")).andReturn(windowStoreMock());

View File

@ -46,8 +46,8 @@ public class PunctuationQueueTest {
final ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
@Override
public void punctuate(final ProcessorNode node, final long time, final PunctuationType type, final Punctuator punctuator) {
punctuator.punctuate(time);
public void punctuate(final ProcessorNode node, final long timestamp, final PunctuationType type, final Punctuator punctuator) {
punctuator.punctuate(timestamp);
}
};
@ -85,8 +85,8 @@ public class PunctuationQueueTest {
final ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
@Override
public void punctuate(final ProcessorNode node, final long time, final PunctuationType type, final Punctuator punctuator) {
punctuator.punctuate(time);
public void punctuate(final ProcessorNode node, final long timestamp, final PunctuationType type, final Punctuator punctuator) {
punctuator.punctuate(timestamp);
}
};
@ -124,8 +124,8 @@ public class PunctuationQueueTest {
final ProcessorNodePunctuator processorNodePunctuator = new ProcessorNodePunctuator() {
@Override
public void punctuate(final ProcessorNode node, final long time, final PunctuationType type, final Punctuator punctuator) {
punctuator.punctuate(time);
public void punctuate(final ProcessorNode node, final long timestamp, final PunctuationType type, final Punctuator punctuator) {
punctuator.punctuate(timestamp);
// simulate scheduler cancelled from within punctuator
cancellable.cancel();
}

View File

@ -18,8 +18,6 @@ package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.IntegerDeserializer;
@ -28,7 +26,6 @@ import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler;
import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
import org.apache.kafka.streams.errors.StreamsException;
@ -37,6 +34,7 @@ import org.apache.kafka.streams.processor.LogAndSkipOnInvalidTimestamp;
import org.apache.kafka.streams.processor.TimestampExtractor;
import org.apache.kafka.streams.state.StateSerdes;
import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.MockRecordCollector;
import org.apache.kafka.test.MockSourceNode;
import org.apache.kafka.test.MockTimestampExtractor;
import org.junit.After;
@ -56,16 +54,9 @@ public class RecordQueueTest {
private final TimestampExtractor timestampExtractor = new MockTimestampExtractor();
private final String[] topics = {"topic"};
private final Sensor droppedRecordsSensor = new Metrics().sensor("skipped-records");
final InternalMockProcessorContext context = new InternalMockProcessorContext(
StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class),
new RecordCollectorImpl(
null,
new LogContext("record-queue-test "),
new DefaultProductionExceptionHandler(),
droppedRecordsSensor
)
new MockRecordCollector()
);
private final MockSourceNode mockSourceNodeWithMetrics = new MockSourceNode<>(topics, intDeserializer, intDeserializer);
private final RecordQueue queue = new RecordQueue(

View File

@ -16,16 +16,15 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.state.StateSerdes;
import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.MockRecordCollector;
import org.junit.Before;
import org.junit.Test;
@ -35,19 +34,23 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.fail;
public class SinkNodeTest {
private final Serializer<byte[]> anySerializer = Serdes.ByteArray().serializer();
private final StateSerdes<Bytes, Bytes> anyStateSerde = StateSerdes.withBuiltinTypes("anyName", Bytes.class, Bytes.class);
private final RecordCollector recordCollector = new RecordCollectorImpl(
null,
new LogContext("sinknode-test "),
new DefaultProductionExceptionHandler(),
new Metrics().sensor("dropped-records")
);
private final Serializer<byte[]> anySerializer = Serdes.ByteArray().serializer();
private final RecordCollector recordCollector = new MockRecordCollector() {
@Override
public <K, V> void send(final String topic,
final K key,
final V value,
final Headers headers,
final Long timestamp,
final Serializer<K> keySerializer,
final Serializer<V> valueSerializer,
final StreamPartitioner<? super K, ? super V> partitioner) {
throw new ClassCastException("boom");
}
};
private final InternalMockProcessorContext context = new InternalMockProcessorContext(
anyStateSerde,
recordCollector
);
private final InternalMockProcessorContext context = new InternalMockProcessorContext(anyStateSerde, recordCollector);
private final SinkNode<byte[], byte[]> sink = new SinkNode<>("anyNodeName",
new StaticTopicNameExtractor<>("any-output-topic"), anySerializer, anySerializer, null);
@ -57,19 +60,15 @@ public class SinkNodeTest {
@Before
public void before() {
recordCollector.init(new MockProducer<>(true, anySerializer, anySerializer));
sink.init(context);
}
@Test
public void shouldThrowStreamsExceptionOnInputRecordWithInvalidTimestamp() {
final Bytes anyKey = new Bytes("any key".getBytes());
final Bytes anyValue = new Bytes("any value".getBytes());
// When/Then
context.setTime(-1); // ensures a negative timestamp is set for the record we send next
try {
illTypedSink.process(anyKey, anyValue);
illTypedSink.process("any key".getBytes(), "any value".getBytes());
fail("Should have thrown StreamsException");
} catch (final StreamsException ignored) {
// expected
@ -77,14 +76,11 @@ public class SinkNodeTest {
}
@Test
public void shouldThrowStreamsExceptionOnKeyValueTypeSerializerMismatch() {
final String keyOfDifferentTypeThanSerializer = "key with different type";
final String valueOfDifferentTypeThanSerializer = "value with different type";
public void shouldThrowStreamsExceptionWithClassCastFromRecordCollector() {
// When/Then
context.setTime(0);
try {
illTypedSink.process(keyOfDifferentTypeThanSerializer, valueOfDifferentTypeThanSerializer);
illTypedSink.process("key", "value");
fail("Should have thrown StreamsException");
} catch (final StreamsException e) {
assertThat(e.getCause(), instanceOf(ClassCastException.class));
@ -92,13 +88,11 @@ public class SinkNodeTest {
}
@Test
public void shouldHandleNullKeysWhenThrowingStreamsExceptionOnKeyValueTypeSerializerMismatch() {
final String invalidValueToTriggerSerializerMismatch = "";
public void shouldThrowStreamsExceptionNullKeyWithClassCastFromRecordCollector() {
// When/Then
context.setTime(1);
try {
illTypedSink.process(null, invalidValueToTriggerSerializerMismatch);
illTypedSink.process(null, "");
fail("Should have thrown StreamsException");
} catch (final StreamsException e) {
assertThat(e.getCause(), instanceOf(ClassCastException.class));
@ -107,13 +101,11 @@ public class SinkNodeTest {
}
@Test
public void shouldHandleNullValuesWhenThrowingStreamsExceptionOnKeyValueTypeSerializerMismatch() {
final String invalidKeyToTriggerSerializerMismatch = "";
public void shouldThrowStreamsExceptionNullValueWithClassCastFromRecordCollector() {
// When/Then
context.setTime(1);
try {
illTypedSink.process(invalidKeyToTriggerSerializerMismatch, null);
illTypedSink.process("", null);
fail("Should have thrown StreamsException");
} catch (final StreamsException e) {
assertThat(e.getCause(), instanceOf(ClassCastException.class));

View File

@ -16,11 +16,6 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
@ -29,111 +24,70 @@ import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.CumulativeSum;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.IntegerDeserializer;
import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.LongSerializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.Bytes;
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.LockException;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.kstream.Windowed;
import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder;
import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilderTest;
import org.apache.kafka.streams.kstream.internals.TimeWindow;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.TimestampedWindowStore;
import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.apache.kafka.streams.state.WindowStore;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.streams.state.internals.WindowKeySchema;
import org.apache.kafka.test.MockKeyValueStore;
import org.apache.kafka.test.MockKeyValueStoreBuilder;
import org.apache.kafka.test.MockRestoreConsumer;
import org.apache.kafka.test.MockStateRestoreListener;
import org.apache.kafka.test.MockTimestampExtractor;
import org.apache.kafka.test.TestUtils;
import org.easymock.EasyMock;
import org.easymock.EasyMockRunner;
import org.easymock.Mock;
import org.easymock.MockType;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import java.io.File;
import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import static java.time.Duration.ofMillis;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptySet;
import static java.util.Collections.singletonList;
import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.apache.kafka.common.utils.Utils.mkProperties;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertThrows;
@RunWith(EasyMockRunner.class)
public class StandbyTaskTest {
private final String threadName = "threadName";
private final String threadId = Thread.currentThread().getName();
private final TaskId taskId = new TaskId(0, 1);
private StandbyTask task;
private final Serializer<Integer> intSerializer = new IntegerSerializer();
private final TaskId taskId = new TaskId(0, 0);
private final String applicationId = "test-application";
private final String storeName1 = "store1";
private final String storeName2 = "store2";
private final String applicationId = "test-application";
private final String storeChangelogTopicName1 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName1);
private final String storeChangelogTopicName2 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName2);
private final String globalStoreName = "ktable1";
private final TopicPartition partition1 = new TopicPartition(storeChangelogTopicName1, 1);
private final TopicPartition partition2 = new TopicPartition(storeChangelogTopicName2, 1);
private final MockStateRestoreListener stateRestoreListener = new MockStateRestoreListener();
private final TopicPartition partition = new TopicPartition(storeChangelogTopicName1, 0);
private final MockKeyValueStore store1 = (MockKeyValueStore) new MockKeyValueStoreBuilder(storeName1, false).build();
private final MockKeyValueStore store2 = (MockKeyValueStore) new MockKeyValueStoreBuilder(storeName2, true).build();
private final Set<TopicPartition> topicPartitions = Collections.emptySet();
private final ProcessorTopology topology = ProcessorTopologyFactories.withLocalStores(
asList(new MockKeyValueStoreBuilder(storeName1, false).build(),
new MockKeyValueStoreBuilder(storeName2, true).build()),
mkMap(
mkEntry(storeName1, storeChangelogTopicName1),
mkEntry(storeName2, storeChangelogTopicName2)
)
);
private final TopicPartition globalTopicPartition = new TopicPartition(globalStoreName, 0);
private final Set<TopicPartition> ktablePartitions = Utils.mkSet(globalTopicPartition);
private final ProcessorTopology ktableTopology = ProcessorTopologyFactories.withLocalStores(
singletonList(new MockKeyValueStoreBuilder(globalTopicPartition.topic(), true)
.withLoggingDisabled().build()),
mkMap(
mkEntry(globalStoreName, globalTopicPartition.topic())
)
asList(store1, store2),
mkMap(mkEntry(storeName1, storeChangelogTopicName1), mkEntry(storeName2, storeChangelogTopicName2))
);
private final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(new Metrics(), threadName, StreamsConfig.METRICS_LATEST);
private File baseDir;
private StreamsConfig config;
private StateDirectory stateDirectory;
private StandbyTask task;
private StreamsConfig createConfig(final File baseDir) throws IOException {
return new StreamsConfig(mkProperties(mkMap(
@ -145,27 +99,18 @@ public class StandbyTaskTest {
)));
}
private final MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
private final MockRestoreConsumer<Integer, Integer> restoreStateConsumer = new MockRestoreConsumer<>(
new IntegerSerializer(),
new IntegerSerializer()
);
private final StoreChangelogReader changelogReader = new StoreChangelogReader(
restoreStateConsumer,
Duration.ZERO,
stateRestoreListener,
new LogContext("standby-task-test ")
);
private final byte[] recordValue = intSerializer.serialize(null, 10);
private final byte[] recordKey = intSerializer.serialize(null, 1);
private final String threadName = "threadName";
private final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(new Metrics(), threadName, StreamsConfig.METRICS_LATEST);
@Mock(type = MockType.NICE)
private ProcessorStateManager stateManager;
@Before
public void setup() throws Exception {
EasyMock.expect(stateManager.taskId()).andReturn(taskId).anyTimes();
restoreStateConsumer.reset();
restoreStateConsumer.updatePartitions(storeChangelogTopicName1, asList(
new PartitionInfo(storeChangelogTopicName1, 0, Node.noNode(), new Node[0], new Node[0]),
@ -179,615 +124,234 @@ public class StandbyTaskTest {
new PartitionInfo(storeChangelogTopicName2, 2, Node.noNode(), new Node[0], new Node[0])
));
baseDir = TestUtils.tempDirectory();
stateDirectory = new StateDirectory(createConfig(baseDir), new MockTime(), true);
config = createConfig(baseDir);
stateDirectory = new StateDirectory(config, new MockTime(), true);
}
@After
public void cleanup() throws IOException {
if (task != null && !task.isClosed()) {
task.close(true, false);
task.closeDirty();
task = null;
}
Utils.delete(baseDir);
}
@Test
public void testStorePartitions() throws IOException {
final StreamsConfig config = createConfig(baseDir);
task = new StandbyTask(taskId,
topicPartitions,
topology,
consumer,
changelogReader,
config,
streamsMetrics,
stateDirectory);
task.initializeStateStores();
assertEquals(Utils.mkSet(partition2, partition1), new HashSet<>(task.checkpointedOffsets().keySet()));
}
public void shouldThrowLockExceptionIfFailedToLockStateDirectory() throws IOException {
stateDirectory = EasyMock.createNiceMock(StateDirectory.class);
EasyMock.expect(stateDirectory.lock(taskId)).andReturn(false);
EasyMock.replay(stateDirectory);
@SuppressWarnings("unchecked")
@Test
public void testUpdateNonInitializedStore() throws IOException {
final StreamsConfig config = createConfig(baseDir);
task = new StandbyTask(taskId,
topicPartitions,
topology,
consumer,
changelogReader,
config,
streamsMetrics,
stateDirectory);
restoreStateConsumer.assign(new ArrayList<>(task.checkpointedOffsets().keySet()));
try {
task.update(partition1,
singletonList(
new ConsumerRecord<>(
partition1.topic(),
partition1.partition(),
10,
0L,
TimestampType.CREATE_TIME,
0L,
0,
0,
recordKey,
recordValue))
);
fail("expected an exception");
} catch (final NullPointerException npe) {
assertThat(npe.getMessage(), containsString("stateRestoreCallback must not be null"));
}
task = createStandbyTask();
assertThrows(LockException.class, task::initializeIfNeeded);
}
@Test
public void testUpdate() throws IOException {
final StreamsConfig config = createConfig(baseDir);
task = new StandbyTask(taskId,
topicPartitions,
topology,
consumer,
changelogReader,
config,
streamsMetrics,
stateDirectory);
task.initializeStateStores();
assertThat(task.checkpointedOffsets(),
equalTo(mkMap(mkEntry(partition1, -1L), mkEntry(partition2, -1L))));
final Set<TopicPartition> partition = Collections.singleton(partition2);
restoreStateConsumer.assign(partition);
public void shouldTransitToRunningAfterInitialization() {
stateManager.registerStore(store1, store1.stateRestoreCallback);
EasyMock.expectLastCall();
stateManager.registerStore(store2, store2.stateRestoreCallback);
EasyMock.expectLastCall();
for (final ConsumerRecord<Integer, Integer> record : asList(new ConsumerRecord<>(partition2.topic(),
partition2.partition(),
10,
0L,
TimestampType.CREATE_TIME,
0L,
0,
0,
1,
100),
new ConsumerRecord<>(partition2.topic(),
partition2.partition(),
20,
0L,
TimestampType.CREATE_TIME,
0L,
0,
0,
2,
100),
new ConsumerRecord<>(partition2.topic(),
partition2.partition(),
30,
0L,
TimestampType.CREATE_TIME,
0L,
0,
0,
3,
100))) {
restoreStateConsumer.bufferRecord(record);
}
EasyMock.replay(stateManager);
restoreStateConsumer.seekToBeginning(partition);
task.update(partition2, restoreStateConsumer.poll(ofMillis(100)).records(partition2));
assertThat(
task.checkpointedOffsets(),
equalTo(
mkMap(
mkEntry(partition1, -1L),
mkEntry(partition2, 31L /*the checkpoint should be 1+ the highest consumed offset*/)
)
)
);
final StandbyContextImpl context = (StandbyContextImpl) task.context();
final MockKeyValueStore store1 = (MockKeyValueStore) context.getStateMgr().getStore(storeName1);
final MockKeyValueStore store2 = (MockKeyValueStore) context.getStateMgr().getStore(storeName2);
task = createStandbyTask();
assertEquals(Collections.emptyList(), store1.keys);
assertEquals(asList(1, 2, 3), store2.keys);
assertEquals(Task.State.CREATED, task.state());
task.initializeIfNeeded();
assertEquals(Task.State.RUNNING, task.state());
// initialize should be idempotent
task.initializeIfNeeded();
assertEquals(Task.State.RUNNING, task.state());
EasyMock.verify(stateManager);
}
@Test
public void shouldRestoreToWindowedStores() throws IOException {
final String storeName = "windowed-store";
final String changelogName = applicationId + "-" + storeName + "-changelog";
public void shouldThrowIfCommittingOnIllegalState() {
task = createStandbyTask();
final TopicPartition topicPartition = new TopicPartition(changelogName, 1);
assertThrows(IllegalStateException.class, task::commit);
}
final Set<TopicPartition> partitions = Collections.singleton(topicPartition);
@Test
public void shouldFlushAndCheckpointStateManagerOnCommit() {
stateManager.flush();
EasyMock.expectLastCall();
stateManager.checkpoint(EasyMock.eq(Collections.emptyMap()));
EasyMock.replay(stateManager);
consumer.assign(partitions);
final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder().setApplicationId(applicationId);
final InternalStreamsBuilder builder = new InternalStreamsBuilder(internalTopologyBuilder);
builder
.stream(Collections.singleton("topic"), new ConsumedInternal<>())
.groupByKey()
.windowedBy(TimeWindows.of(ofMillis(60_000)).grace(ofMillis(0L)))
.count(Materialized.<Object, Long, WindowStore<Bytes, byte[]>>as(storeName).withRetention(ofMillis(120_000L)));
builder.buildAndOptimizeTopology();
task = new StandbyTask(
taskId,
partitions,
internalTopologyBuilder.build(0),
consumer,
new StoreChangelogReader(
restoreStateConsumer,
Duration.ZERO,
stateRestoreListener,
new LogContext("standby-task-test ")
),
createConfig(baseDir),
new MockStreamsMetrics(new Metrics()),
stateDirectory
);
task.initializeStateStores();
consumer.commitSync(mkMap(mkEntry(topicPartition, new OffsetAndMetadata(35L))));
task = createStandbyTask();
task.initializeIfNeeded();
task.commit();
final List<ConsumerRecord<byte[], byte[]>> remaining1 = task.update(
topicPartition,
asList(
makeWindowedConsumerRecord(changelogName, 10, 1, 0L, 60_000L),
makeWindowedConsumerRecord(changelogName, 20, 2, 60_000L, 120_000),
makeWindowedConsumerRecord(changelogName, 30, 3, 120_000L, 180_000),
makeWindowedConsumerRecord(changelogName, 40, 4, 180_000L, 240_000)
)
);
assertEquals(
asList(
new KeyValue<>(new Windowed<>(1, new TimeWindow(0, 60_000)), ValueAndTimestamp.make(100L, 60_000L)),
new KeyValue<>(new Windowed<>(2, new TimeWindow(60_000, 120_000)), ValueAndTimestamp.make(100L, 120_000L)),
new KeyValue<>(new Windowed<>(3, new TimeWindow(120_000, 180_000)), ValueAndTimestamp.make(100L, 180_000L))
),
getWindowedStoreContents(storeName, task)
);
consumer.commitSync(mkMap(mkEntry(topicPartition, new OffsetAndMetadata(45L))));
task.commit();
final List<ConsumerRecord<byte[], byte[]>> remaining2 = task.update(topicPartition, remaining1);
assertEquals(emptyList(), remaining2);
// the first record's window should have expired.
assertEquals(
asList(
new KeyValue<>(new Windowed<>(2, new TimeWindow(60_000, 120_000)), ValueAndTimestamp.make(100L, 120_000L)),
new KeyValue<>(new Windowed<>(3, new TimeWindow(120_000, 180_000)), ValueAndTimestamp.make(100L, 180_000L)),
new KeyValue<>(new Windowed<>(4, new TimeWindow(180_000, 240_000)), ValueAndTimestamp.make(100L, 240_000L))
),
getWindowedStoreContents(storeName, task)
);
}
private ConsumerRecord<byte[], byte[]> makeWindowedConsumerRecord(final String changelogName,
final int offset,
final int key,
final long start,
final long end) {
final Windowed<Integer> data = new Windowed<>(key, new TimeWindow(start, end));
final Bytes wrap = Bytes.wrap(new IntegerSerializer().serialize(null, data.key()));
final byte[] keyBytes = WindowKeySchema.toStoreKeyBinary(new Windowed<>(wrap, data.window()), 1).get();
return new ConsumerRecord<>(
changelogName,
1,
offset,
end,
TimestampType.CREATE_TIME,
0L,
0,
0,
keyBytes,
new LongSerializer().serialize(null, 100L)
);
EasyMock.verify(stateManager);
}
@Test
public void shouldWriteCheckpointFile() throws IOException {
final String storeName = "checkpoint-file-store";
final String changelogName = applicationId + "-" + storeName + "-changelog";
public void shouldReturnStateManagerChangelogOffsets() {
EasyMock.expect(stateManager.changelogOffsets()).andReturn(Collections.singletonMap(partition, 50L));
EasyMock.replay(stateManager);
final TopicPartition topicPartition = new TopicPartition(changelogName, 1);
final Set<TopicPartition> partitions = Collections.singleton(topicPartition);
task = createStandbyTask();
final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder().setApplicationId(applicationId);
assertEquals(Collections.singletonMap(partition, 50L), task.changelogOffsets());
final InternalStreamsBuilder builder = new InternalStreamsBuilder(internalTopologyBuilder);
builder.stream(Collections.singleton("topic"), new ConsumedInternal<>())
.groupByKey()
.count(Materialized.as(storeName));
builder.buildAndOptimizeTopology();
consumer.assign(partitions);
task = new StandbyTask(
taskId,
partitions,
internalTopologyBuilder.build(0),
consumer,
changelogReader,
createConfig(baseDir),
new MockStreamsMetrics(new Metrics()),
stateDirectory
);
task.initializeStateStores();
consumer.commitSync(mkMap(mkEntry(topicPartition, new OffsetAndMetadata(20L))));
task.commit();
task.update(
topicPartition,
singletonList(makeWindowedConsumerRecord(changelogName, 10, 1, 0L, 60_000L))
);
task.close(true, false);
final File taskDir = stateDirectory.directoryForTask(taskId);
final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(taskDir, StateManagerUtil.CHECKPOINT_FILE_NAME));
final Map<TopicPartition, Long> offsets = checkpoint.read();
assertEquals(1, offsets.size());
assertEquals(Long.valueOf(11L), offsets.get(topicPartition));
}
@SuppressWarnings("unchecked")
private List<KeyValue<Windowed<Integer>, ValueAndTimestamp<Long>>> getWindowedStoreContents(final String storeName,
final StandbyTask task) {
final StandbyContextImpl context = (StandbyContextImpl) task.context();
final List<KeyValue<Windowed<Integer>, ValueAndTimestamp<Long>>> result = new ArrayList<>();
try (final KeyValueIterator<Windowed<byte[]>, ValueAndTimestamp<Long>> iterator =
((TimestampedWindowStore) context.getStateMgr().getStore(storeName)).all()) {
while (iterator.hasNext()) {
final KeyValue<Windowed<byte[]>, ValueAndTimestamp<Long>> next = iterator.next();
final Integer deserializedKey = new IntegerDeserializer().deserialize(null, next.key.key());
result.add(new KeyValue<>(new Windowed<>(deserializedKey, next.key.window()), next.value));
}
}
return result;
EasyMock.verify(stateManager);
}
@Test
public void shouldRestoreToKTable() throws IOException {
consumer.assign(Collections.singletonList(globalTopicPartition));
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(0L))));
public void shouldDoNothingWithCreatedStateOnClose() {
stateManager.close();
EasyMock.expectLastCall().andThrow(new AssertionError("Close should not be called")).anyTimes();
stateManager.flush();
EasyMock.expectLastCall().andThrow(new AssertionError("Flush should not be called")).anyTimes();
stateManager.checkpoint(EasyMock.anyObject());
EasyMock.expectLastCall().andThrow(new AssertionError("Checkpoint should not be called")).anyTimes();
EasyMock.replay(stateManager);
final MetricName metricName = setupCloseTaskMetric();
task = new StandbyTask(
taskId,
ktablePartitions,
ktableTopology,
consumer,
changelogReader,
createConfig(baseDir),
streamsMetrics,
stateDirectory
);
task.initializeStateStores();
task = createStandbyTask();
task.closeClean();
// The commit offset is at 0L. Records should not be processed
List<ConsumerRecord<byte[], byte[]>> remaining = task.update(
globalTopicPartition,
asList(
makeConsumerRecord(globalTopicPartition, 10, 1),
makeConsumerRecord(globalTopicPartition, 20, 2),
makeConsumerRecord(globalTopicPartition, 30, 3),
makeConsumerRecord(globalTopicPartition, 40, 4),
makeConsumerRecord(globalTopicPartition, 50, 5)
)
);
assertEquals(5, remaining.size());
assertEquals(Task.State.CLOSED, task.state());
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(10L))));
task.commit(); // update offset limits
final double expectedCloseTaskMetric = 1.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
// The commit offset has not reached, yet.
remaining = task.update(globalTopicPartition, remaining);
assertEquals(5, remaining.size());
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(11L))));
task.commit(); // update offset limits
// one record should be processed.
remaining = task.update(globalTopicPartition, remaining);
assertEquals(4, remaining.size());
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(45L))));
task.commit(); // update offset limits
// The commit offset is now 45. All record except for the last one should be processed.
remaining = task.update(globalTopicPartition, remaining);
assertEquals(1, remaining.size());
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(50L))));
task.commit(); // update offset limits
// The commit offset is now 50. Still the last record remains.
remaining = task.update(globalTopicPartition, remaining);
assertEquals(1, remaining.size());
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(60L))));
task.commit(); // update offset limits
// The commit offset is now 60. No record should be left.
remaining = task.update(globalTopicPartition, remaining);
assertEquals(emptyList(), remaining);
}
private ConsumerRecord<byte[], byte[]> makeConsumerRecord(final TopicPartition topicPartition,
final long offset,
final int key) {
final IntegerSerializer integerSerializer = new IntegerSerializer();
return new ConsumerRecord<>(
topicPartition.topic(),
topicPartition.partition(),
offset,
0L,
TimestampType.CREATE_TIME,
0L,
0,
0,
integerSerializer.serialize(null, key),
integerSerializer.serialize(null, 100)
);
EasyMock.verify(stateManager);
}
@Test
public void shouldNotGetConsumerCommittedOffsetIfThereAreNoRecordUpdates() throws IOException {
final AtomicInteger committedCallCount = new AtomicInteger();
public void shouldNotCommitAndThrowOnCloseDirty() {
stateManager.close();
EasyMock.expectLastCall().andThrow(new ProcessorStateException("KABOOM!")).anyTimes();
stateManager.flush();
EasyMock.expectLastCall().andThrow(new AssertionError("Flush should not be called")).anyTimes();
stateManager.checkpoint(EasyMock.anyObject());
EasyMock.expectLastCall().andThrow(new AssertionError("Checkpoint should not be called")).anyTimes();
EasyMock.replay(stateManager);
final MetricName metricName = setupCloseTaskMetric();
final Consumer<byte[], byte[]> consumer = new MockConsumer<byte[], byte[]>(OffsetResetStrategy.EARLIEST) {
@Override
public synchronized Map<TopicPartition, OffsetAndMetadata> committed(final Set<TopicPartition> partitions) {
committedCallCount.getAndIncrement();
return super.committed(partitions);
}
};
task = createStandbyTask();
task.initializeIfNeeded();
task.closeDirty();
consumer.assign(Collections.singletonList(globalTopicPartition));
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(0L))));
assertEquals(Task.State.CLOSED, task.state());
task = new StandbyTask(
taskId,
ktablePartitions,
ktableTopology,
consumer,
changelogReader,
createConfig(baseDir),
streamsMetrics,
stateDirectory
);
task.initializeStateStores();
assertThat(committedCallCount.get(), equalTo(0));
final double expectedCloseTaskMetric = 1.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
task.update(globalTopicPartition, Collections.emptyList());
// We should not make a consumer.committed() call because there are no new records.
assertThat(committedCallCount.get(), equalTo(0));
EasyMock.verify(stateManager);
}
@Test
public void shouldGetConsumerCommittedOffsetsOncePerCommit() throws IOException {
final AtomicInteger committedCallCount = new AtomicInteger();
public void shouldCommitOnCloseClean() {
stateManager.close();
EasyMock.expectLastCall();
stateManager.flush();
EasyMock.expectLastCall();
stateManager.checkpoint(EasyMock.eq(Collections.emptyMap()));
EasyMock.expectLastCall();
EasyMock.replay(stateManager);
final MetricName metricName = setupCloseTaskMetric();
final Consumer<byte[], byte[]> consumer = new MockConsumer<byte[], byte[]>(OffsetResetStrategy.EARLIEST) {
@Override
public synchronized Map<TopicPartition, OffsetAndMetadata> committed(final Set<TopicPartition> partitions) {
committedCallCount.getAndIncrement();
return super.committed(partitions);
}
};
task = createStandbyTask();
task.initializeIfNeeded();
task.closeClean();
consumer.assign(Collections.singletonList(globalTopicPartition));
consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(0L))));
assertEquals(Task.State.CLOSED, task.state());
task = new StandbyTask(
taskId,
ktablePartitions,
ktableTopology,
consumer,
changelogReader,
createConfig(baseDir),
streamsMetrics,
stateDirectory
);
task.initializeStateStores();
final double expectedCloseTaskMetric = 1.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
task.update(
globalTopicPartition,
Collections.singletonList(
makeConsumerRecord(globalTopicPartition, 1, 1)
)
);
assertThat(committedCallCount.get(), equalTo(1));
task.update(
globalTopicPartition,
Collections.singletonList(
makeConsumerRecord(globalTopicPartition, 1, 1)
)
);
// We should not make another consumer.committed() call until we commit
assertThat(committedCallCount.get(), equalTo(1));
task.commit();
task.update(
globalTopicPartition,
Collections.singletonList(
makeConsumerRecord(globalTopicPartition, 1, 1)
)
);
// We committed so we're allowed to make another consumer.committed() call
assertThat(committedCallCount.get(), equalTo(2));
EasyMock.verify(stateManager);
}
@Test
public void shouldInitializeStateStoreWithoutException() throws IOException {
final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder());
builder.stream(Collections.singleton("topic"), new ConsumedInternal<>()).groupByKey().count();
public void shouldThrowOnCloseCleanError() {
stateManager.close();
EasyMock.expectLastCall().andThrow(new RuntimeException("KABOOM!")).anyTimes();
EasyMock.replay(stateManager);
final MetricName metricName = setupCloseTaskMetric();
initializeStandbyStores(builder);
task = createStandbyTask();
task.initializeIfNeeded();
assertThrows(RuntimeException.class, task::closeClean);
assertEquals(Task.State.CLOSING, task.state());
final double expectedCloseTaskMetric = 0.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
EasyMock.verify(stateManager);
EasyMock.reset(stateManager);
EasyMock.replay(stateManager);
}
@Test
public void shouldInitializeWindowStoreWithoutException() throws IOException {
final InternalStreamsBuilder builder = new InternalStreamsBuilder(new InternalTopologyBuilder());
builder.stream(Collections.singleton("topic"),
new ConsumedInternal<>()).groupByKey().windowedBy(TimeWindows.of(ofMillis(100))).count();
public void shouldThrowOnCloseCleanFlushError() {
stateManager.flush();
EasyMock.expectLastCall().andThrow(new RuntimeException("KABOOM!")).anyTimes();
EasyMock.replay(stateManager);
final MetricName metricName = setupCloseTaskMetric();
initializeStandbyStores(builder);
}
task = createStandbyTask();
task.initializeIfNeeded();
private void initializeStandbyStores(final InternalStreamsBuilder builder) throws IOException {
final StreamsConfig config = createConfig(baseDir);
builder.buildAndOptimizeTopology();
final InternalTopologyBuilder internalTopologyBuilder = InternalStreamsBuilderTest.internalTopologyBuilder(builder);
final ProcessorTopology topology = internalTopologyBuilder.setApplicationId(applicationId).build(0);
assertThrows(RuntimeException.class, task::closeClean);
assertEquals(Task.State.RUNNING, task.state());
task = new StandbyTask(
taskId,
emptySet(),
topology,
consumer,
changelogReader,
config,
new MockStreamsMetrics(new Metrics()),
stateDirectory
);
final double expectedCloseTaskMetric = 0.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
task.initializeStateStores();
assertTrue(task.hasStateStores());
EasyMock.verify(stateManager);
EasyMock.reset(stateManager);
EasyMock.replay(stateManager);
}
@Test
public void shouldCheckpointStoreOffsetsOnCommit() throws IOException {
consumer.assign(Collections.singletonList(globalTopicPartition));
final Map<TopicPartition, OffsetAndMetadata> committedOffsets = new HashMap<>();
committedOffsets.put(new TopicPartition(globalTopicPartition.topic(), globalTopicPartition.partition()),
new OffsetAndMetadata(100L));
consumer.commitSync(committedOffsets);
public void shouldThrowOnCloseCleanCheckpointError() {
stateManager.checkpoint(EasyMock.anyObject());
EasyMock.expectLastCall().andThrow(new RuntimeException("KABOOM!")).anyTimes();
EasyMock.replay(stateManager);
final MetricName metricName = setupCloseTaskMetric();
restoreStateConsumer.updatePartitions(
globalStoreName,
Collections.singletonList(new PartitionInfo(globalStoreName, 0, Node.noNode(), new Node[0], new Node[0]))
);
task = createStandbyTask();
task.initializeIfNeeded();
final TaskId taskId = new TaskId(0, 0);
final MockTime time = new MockTime();
final StreamsConfig config = createConfig(baseDir);
task = new StandbyTask(
taskId,
ktablePartitions,
ktableTopology,
consumer,
changelogReader,
config,
streamsMetrics,
stateDirectory
);
task.initializeStateStores();
assertThrows(RuntimeException.class, task::closeClean);
restoreStateConsumer.assign(new ArrayList<>(task.checkpointedOffsets().keySet()));
assertEquals(Task.State.RUNNING, task.state());
final byte[] serializedValue = Serdes.Integer().serializer().serialize("", 1);
task.update(
globalTopicPartition,
singletonList(new ConsumerRecord<>(globalTopicPartition.topic(),
globalTopicPartition.partition(),
50L,
serializedValue,
serializedValue))
);
time.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG));
task.commit();
final Map<TopicPartition, Long> checkpoint = new OffsetCheckpoint(
new File(stateDirectory.directoryForTask(taskId), StateManagerUtil.CHECKPOINT_FILE_NAME)
).read();
assertThat(checkpoint, equalTo(Collections.singletonMap(globalTopicPartition, 51L)));
final double expectedCloseTaskMetric = 0.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
EasyMock.verify(stateManager);
EasyMock.reset(stateManager);
EasyMock.replay(stateManager);
}
@Test
public void shouldCloseStateMangerOnTaskCloseWhenCommitFailed() throws Exception {
consumer.assign(Collections.singletonList(globalTopicPartition));
final Map<TopicPartition, OffsetAndMetadata> committedOffsets = new HashMap<>();
committedOffsets.put(new TopicPartition(globalTopicPartition.topic(), globalTopicPartition.partition()),
new OffsetAndMetadata(100L));
consumer.commitSync(committedOffsets);
public void shouldThrowIfClosingOnIllegalState() {
task = createStandbyTask();
restoreStateConsumer.updatePartitions(
globalStoreName,
Collections.singletonList(new PartitionInfo(globalStoreName, 0, Node.noNode(), new Node[0], new Node[0]))
);
task.closeClean();
final StreamsConfig config = createConfig(baseDir);
final AtomicBoolean closedStateManager = new AtomicBoolean(false);
task = new StandbyTask(
taskId,
ktablePartitions,
ktableTopology,
consumer,
changelogReader,
config,
streamsMetrics,
stateDirectory
) {
@Override
public void commit() {
throw new RuntimeException("KABOOM!");
}
// close call are not idempotent since we are already in closed
assertThrows(IllegalStateException.class, task::closeClean);
assertThrows(IllegalStateException.class, task::closeDirty);
}
@Override
void closeStateManager(final boolean clean) throws ProcessorStateException {
closedStateManager.set(true);
}
};
task.initializeStateStores();
try {
task.close(true, false);
fail("should have thrown exception");
} catch (final Exception e) {
// expected
task = null;
}
assertTrue(closedStateManager.get());
private StandbyTask createStandbyTask() {
return new StandbyTask(taskId, Collections.singleton(partition), topology, config, streamsMetrics, stateManager, stateDirectory);
}
private MetricName setupCloseTaskMetric() {
@ -797,33 +361,9 @@ public class StandbyTaskTest {
return metricName;
}
private void verifyCloseTaskMetric(final double expected,
final StreamsMetricsImpl streamsMetrics,
final MetricName metricName) {
private void verifyCloseTaskMetric(final double expected, final StreamsMetricsImpl streamsMetrics, final MetricName metricName) {
final KafkaMetric metric = (KafkaMetric) streamsMetrics.metrics().get(metricName);
final double totalCloses = metric.measurable().measure(metric.config(), System.currentTimeMillis());
assertThat(totalCloses, equalTo(expected));
}
@Test
public void shouldRecordTaskClosedMetricOnClose() throws IOException {
final MetricName metricName = setupCloseTaskMetric();
final StandbyTask task = new StandbyTask(
taskId,
ktablePartitions,
ktableTopology,
consumer,
changelogReader,
createConfig(baseDir),
streamsMetrics,
stateDirectory
);
final boolean clean = true;
final boolean isZombie = false;
task.close(clean, isZombie);
final double expectedCloseTaskMetric = 1.0;
verifyCloseTaskMetric(expectedCloseTaskMetric, streamsMetrics, metricName);
}
}

View File

@ -46,13 +46,13 @@ public class StateConsumerTest {
private final Map<TopicPartition, Long> partitionOffsets = new HashMap<>();
private final LogContext logContext = new LogContext("test ");
private GlobalStreamThread.StateConsumer stateConsumer;
private StateMaintainerStub stateMaintainer;
private TaskStub stateMaintainer;
@Before
public void setUp() {
partitionOffsets.put(topicOne, 20L);
partitionOffsets.put(topicTwo, 30L);
stateMaintainer = new StateMaintainerStub(partitionOffsets);
stateMaintainer = new TaskStub(partitionOffsets);
stateConsumer = new GlobalStreamThread.StateConsumer(logContext, consumer, stateMaintainer, time, Duration.ofMillis(10L), FLUSH_INTERVAL);
}
@ -121,13 +121,13 @@ public class StateConsumerTest {
}
private static class StateMaintainerStub implements GlobalStateMaintainer {
private static class TaskStub implements GlobalStateMaintainer {
private final Map<TopicPartition, Long> partitionOffsets;
private final Map<TopicPartition, Integer> updatedPartitions = new HashMap<>();
private boolean flushed;
private boolean closed;
StateMaintainerStub(final Map<TopicPartition, Long> partitionOffsets) {
TaskStub(final Map<TopicPartition, Long> partitionOffsets) {
this.partitionOffsets = partitionOffsets;
}

View File

@ -22,8 +22,6 @@ import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateStore;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
public class StateManagerStub implements StateManager {
@ -34,23 +32,14 @@ public class StateManagerStub implements StateManager {
}
@Override
public void register(final StateStore store,
final StateRestoreCallback stateRestoreCallback) {}
@Override
public void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext) {}
public void registerStore(final StateStore store,
final StateRestoreCallback stateRestoreCallback) {}
@Override
public void flush() {}
@Override
public void close(final boolean clean) throws IOException {}
@Override
public StateStore getGlobalStore(final String name) {
return null;
}
public void close() {}
@Override
public StateStore getStore(final String name) {
@ -58,7 +47,12 @@ public class StateManagerStub implements StateManager {
}
@Override
public Map<TopicPartition, Long> checkpointed() {
public StateStore getGlobalStore(final String name) {
return null;
}
@Override
public Map<TopicPartition, Long> changelogOffsets() {
return null;
}

View File

@ -1,111 +0,0 @@
/*
* 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.processor.internals;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.test.MockRestoreCallback;
import org.apache.kafka.test.MockStateRestoreListener;
import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
import static org.apache.kafka.streams.state.internals.RecordConverters.identity;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.assertTrue;
public class StateRestorerTest {
private static final long OFFSET_LIMIT = 50;
private final MockRestoreCallback callback = new MockRestoreCallback();
private final MockStateRestoreListener reportingListener = new MockStateRestoreListener();
private final CompositeRestoreListener compositeRestoreListener = new CompositeRestoreListener(callback);
private final StateRestorer restorer = new StateRestorer(
new TopicPartition("topic", 1),
compositeRestoreListener,
null,
OFFSET_LIMIT,
true,
"storeName",
identity());
@Before
public void setUp() {
compositeRestoreListener.setUserRestoreListener(reportingListener);
}
@Test
public void shouldCallRestoreOnRestoreCallback() {
restorer.restore(Collections.singletonList(new ConsumerRecord<>("", 0, 0L, new byte[0], new byte[0])));
assertThat(callback.restored.size(), equalTo(1));
}
@Test
public void shouldBeCompletedIfRecordOffsetGreaterThanEndOffset() {
assertTrue(restorer.hasCompleted(11, 10));
}
@Test
public void shouldBeCompletedIfRecordOffsetGreaterThanOffsetLimit() {
assertTrue(restorer.hasCompleted(51, 100));
}
@Test
public void shouldBeCompletedIfEndOffsetAndRecordOffsetAreZero() {
assertTrue(restorer.hasCompleted(0, 0));
}
@Test
public void shouldBeCompletedIfOffsetAndOffsetLimitAreZero() {
final StateRestorer restorer = new StateRestorer(
new TopicPartition("topic", 1),
compositeRestoreListener,
null,
0,
true,
"storeName",
identity());
assertTrue(restorer.hasCompleted(0, 10));
}
@Test
public void shouldSetRestoredOffsetToMinOfLimitAndOffset() {
restorer.setRestoredOffset(20);
assertThat(restorer.restoredOffset(), equalTo(20L));
restorer.setRestoredOffset(100);
assertThat(restorer.restoredOffset(), equalTo(OFFSET_LIMIT));
}
@Test
public void shouldSetStartingOffsetToMinOfLimitAndOffset() {
restorer.setStartingOffset(20);
assertThat(restorer.startingOffset(), equalTo(20L));
restorer.setRestoredOffset(100);
assertThat(restorer.restoredOffset(), equalTo(OFFSET_LIMIT));
}
@Test
public void shouldReturnCorrectNumRestoredRecords() {
restorer.setStartingOffset(20);
restorer.setRestoredOffset(40);
assertThat(restorer.restoredNumRecords(), equalTo(20L));
restorer.setRestoredOffset(100);
assertThat(restorer.restoredNumRecords(), equalTo(OFFSET_LIMIT - 20));
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
@ -165,12 +166,15 @@ public class StreamsPartitionAssignorTest {
private static final String APPLICATION_ID = "stream-partition-assignor-test";
private TaskManager taskManager;
private StreamsMetadataState streamsMetadataState;
private Map<String, Object> configProps() {
final Map<String, Object> configurationMap = new HashMap<>();
configurationMap.put(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID);
configurationMap.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, USER_END_POINT);
configurationMap.put(StreamsConfig.InternalConfig.TASK_MANAGER_FOR_PARTITION_ASSIGNOR, taskManager);
configurationMap.put(StreamsConfig.InternalConfig.STREAMS_METADATA_STATE_FOR_PARTITION_ASSIGNOR, streamsMetadataState);
configurationMap.put(StreamsConfig.InternalConfig.STREAMS_ADMIN_CLIENT, EasyMock.createNiceMock(Admin.class));
configurationMap.put(StreamsConfig.InternalConfig.ASSIGNMENT_ERROR_CODE, new AtomicInteger());
return configurationMap;
}
@ -182,6 +186,7 @@ public class StreamsPartitionAssignorTest {
}
private void configureDefault() {
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager();
partitionAssignor.configure(configProps());
}
@ -199,11 +204,9 @@ public class StreamsPartitionAssignorTest {
final UUID processId,
final InternalTopologyBuilder builder) {
taskManager = EasyMock.createNiceMock(TaskManager.class);
EasyMock.expect(taskManager.adminClient()).andReturn(null).anyTimes();
EasyMock.expect(taskManager.builder()).andReturn(builder).anyTimes();
EasyMock.expect(taskManager.previousRunningTaskIds()).andReturn(prevTasks).anyTimes();
EasyMock.expect(taskManager.activeTaskIds()).andReturn(prevTasks).anyTimes();
EasyMock.expect(taskManager.cachedTasksIds()).andReturn(cachedTasks).anyTimes();
EasyMock.expect(taskManager.tasksOnLocalStorage()).andReturn(cachedTasks).anyTimes();
EasyMock.expect(taskManager.processId()).andReturn(processId).anyTimes();
}
@ -239,6 +242,7 @@ public class StreamsPartitionAssignorTest {
@Test
public void shouldUseEagerRebalancingProtocol() {
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager();
final Map<String, Object> props = configProps();
props.put(StreamsConfig.UPGRADE_FROM_CONFIG, StreamsConfig.UPGRADE_FROM_23);
@ -251,6 +255,7 @@ public class StreamsPartitionAssignorTest {
@Test
public void shouldUseCooperativeRebalancingProtocol() {
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager();
final Map<String, Object> props = configProps();
partitionAssignor.configure(props);
@ -434,6 +439,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(prevTasks, cachedTasks, processId, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
partitionAssignor.setRebalanceProtocol(RebalanceProtocol.EAGER);
@ -468,6 +474,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(prevTasks, cachedTasks, processId, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
final Set<String> topics = mkSet("topic1", "topic2");
@ -506,6 +513,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(prevTasks10, standbyTasks10, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer));
@ -598,6 +606,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(new HashSet<>(), new HashSet<>(), uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer));
@ -648,6 +657,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(Collections.singletonMap(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, SingleGroupPartitionGrouperStub.class));
partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer));
@ -687,6 +697,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(prevTasks10, standbyTasks10, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
subscriptions.put("consumer10",
@ -744,6 +755,7 @@ public class StreamsPartitionAssignorTest {
final UUID uuid2 = UUID.randomUUID();
createMockTaskManager(prevTasks10, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer));
@ -810,6 +822,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer));
@ -904,6 +917,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(prevTasks00, standbyTasks01, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(Collections.singletonMap(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1));
partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer));
@ -976,12 +990,15 @@ public class StreamsPartitionAssignorTest {
@Test
public void testOnAssignment() {
createMockTaskManager();
final InternalTopologyBuilder internalTopologyBuilder =
TopologyWrapper.getInternalTopologyBuilder(new StreamsBuilder().build());
internalTopologyBuilder.setApplicationId(APPLICATION_ID);
taskManager = EasyMock.createStrictMock(TaskManager.class);
final Map<HostInfo, Set<TopicPartition>> hostState = Collections.singletonMap(
new HostInfo("localhost", 9090),
mkSet(t3p0, t3p3));
taskManager.setHostPartitionMappings(hostState, emptyMap());
EasyMock.expectLastCall();
final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
activeTasks.put(task0_0, mkSet(t3p0));
@ -989,15 +1006,18 @@ public class StreamsPartitionAssignorTest {
final Map<TaskId, Set<TopicPartition>> standbyTasks = new HashMap<>();
standbyTasks.put(task0_1, mkSet(t3p1));
standbyTasks.put(task0_2, mkSet(t3p2));
taskManager.setAssignmentMetadata(activeTasks, standbyTasks);
EasyMock.expectLastCall();
final Capture<Cluster> capturedCluster = EasyMock.newCapture();
taskManager.setClusterMetadata(EasyMock.capture(capturedCluster));
taskManager.handleAssignment(activeTasks, standbyTasks);
EasyMock.expectLastCall();
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createStrictMock(StreamsMetadataState.class);
final Capture<Cluster> capturedCluster = EasyMock.newCapture();
streamsMetadataState.onChange(EasyMock.eq(hostState), EasyMock.anyObject(), EasyMock.capture(capturedCluster));
EasyMock.expectLastCall();
EasyMock.replay(streamsMetadataState);
configurePartitionAssignor(emptyMap());
final List<TaskId> activeTaskList = asList(task0_0, task0_3);
final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTaskList, standbyTasks, hostState, emptyMap(), 0);
@ -1005,6 +1025,7 @@ public class StreamsPartitionAssignorTest {
partitionAssignor.onAssignment(assignment, null);
EasyMock.verify(streamsMetadataState);
EasyMock.verify(taskManager);
assertEquals(Collections.singleton(t3p0.topic()), capturedCluster.getValue().topics());
@ -1026,6 +1047,7 @@ public class StreamsPartitionAssignorTest {
final UUID uuid1 = UUID.randomUUID();
createMockTaskManager(emptyTasks, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
final MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer);
partitionAssignor.setInternalTopicManager(internalTopicManager);
@ -1062,6 +1084,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
final MockInternalTopicManager internalTopicManager =
new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer);
@ -1110,6 +1133,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), internalTopologyBuilder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
final MockInternalTopicManager mockInternalTopicManager = new MockInternalTopicManager(
@ -1167,6 +1191,7 @@ public class StreamsPartitionAssignorTest {
final UUID uuid1 = UUID.randomUUID();
createMockTaskManager(emptyTasks, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(Collections.singletonMap(StreamsConfig.APPLICATION_SERVER_CONFIG, USER_END_POINT));
final Set<String> topics = mkSet("input");
final ByteBuffer userData = partitionAssignor.subscriptionUserData(topics);
@ -1187,6 +1212,7 @@ public class StreamsPartitionAssignorTest {
final UUID uuid1 = UUID.randomUUID();
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager(emptyTasks, emptyTasks, uuid1, builder);
EasyMock.replay(taskManager);
configurePartitionAssignor(Collections.singletonMap(StreamsConfig.APPLICATION_SERVER_CONFIG, USER_END_POINT));
@ -1283,6 +1309,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), internalTopologyBuilder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
final MockInternalTopicManager mockInternalTopicManager = new MockInternalTopicManager(
@ -1309,11 +1336,9 @@ public class StreamsPartitionAssignorTest {
final Map<HostInfo, Set<TopicPartition>> hostState = Collections.singletonMap(
new HostInfo("localhost", 9090), mkSet(partitionOne, partitionTwo));
final StreamsBuilder builder = new StreamsBuilder();
final InternalTopologyBuilder internalTopologyBuilder = TopologyWrapper.getInternalTopologyBuilder(builder.build());
internalTopologyBuilder.setApplicationId(APPLICATION_ID);
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), internalTopologyBuilder);
createMockTaskManager();
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(emptyMap());
partitionAssignor.onAssignment(createAssignment(hostState), null);
@ -1334,6 +1359,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, uuid, internalTopologyBuilder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
final Map<String, Object> props = new HashMap<>();
props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, USER_END_POINT);
@ -1399,6 +1425,7 @@ public class StreamsPartitionAssignorTest {
@Test
public void shouldThrowKafkaExceptionAssignmentErrorCodeNotConfigured() {
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager();
final Map<String, Object> config = configProps();
config.remove(StreamsConfig.InternalConfig.ASSIGNMENT_ERROR_CODE);
@ -1413,6 +1440,7 @@ public class StreamsPartitionAssignorTest {
@Test
public void shouldThrowKafkaExceptionIfVersionProbingFlagConfigIsNotAtomicInteger() {
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager();
final Map<String, Object> config = configProps();
config.put(StreamsConfig.InternalConfig.ASSIGNMENT_ERROR_CODE, "i am not an AtomicInteger");
@ -1457,6 +1485,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
partitionAssignor.configure(configProps());
final Map<String, ConsumerPartitionAssignor.Assignment> assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment();
@ -1467,6 +1496,7 @@ public class StreamsPartitionAssignorTest {
@Test
public void shouldDownGradeSubscriptionToVersion1() {
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
configurePartitionAssignor(Collections.singletonMap(StreamsConfig.UPGRADE_FROM_CONFIG, StreamsConfig.UPGRADE_FROM_0100));
@ -1505,6 +1535,7 @@ public class StreamsPartitionAssignorTest {
private void shouldDownGradeSubscriptionToVersion2(final Object upgradeFromValue) {
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
configurePartitionAssignor(Collections.singletonMap(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFromValue));
final Set<String> topics = mkSet("topic1");
@ -1539,6 +1570,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(allTasks, allTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
partitionAssignor.configure(configProps());
final Map<String, ConsumerPartitionAssignor.Assignment> assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment();
@ -1602,6 +1634,7 @@ public class StreamsPartitionAssignorTest {
Collections.singletonList(t1p2))
);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager(allTasks, allTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
final Map<String, Object> props = configProps();
@ -1662,6 +1695,7 @@ public class StreamsPartitionAssignorTest {
createMockTaskManager(allTasks, allTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
final Map<String, Object> props = configProps();
props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
partitionAssignor.configure(props);
@ -1710,7 +1744,7 @@ public class StreamsPartitionAssignorTest {
Collections.singletonList("topic1"),
encodeFutureSubscription())
);
streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class);
createMockTaskManager(emptyTasks, emptyTasks, UUID.randomUUID(), builder);
EasyMock.replay(taskManager);
partitionAssignor.configure(configProps());

View File

@ -16,6 +16,9 @@
*/
package org.apache.kafka.streams.processor.internals;
import org.apache.kafka.streams.integration.StandbyTaskCreationIntegrationTest;
import org.apache.kafka.streams.kstream.internals.metrics.TaskMetricsTest;
import org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignorTest;
import org.junit.runner.RunWith;
import org.junit.runners.Suite;
@ -27,11 +30,15 @@ import org.junit.runners.Suite;
*/
@RunWith(Suite.class)
@Suite.SuiteClasses({
AbstractTaskTest.class,
StreamTaskTest.class,
StandbyTaskTest.class,
AssignedStreamsTasksTest.class,
})
StreamTaskTest.class,
StandbyTaskTest.class,
GlobalStateTaskTest.class,
TaskManagerTest.class,
TaskMetricsTest.class,
StickyTaskAssignorTest.class,
StreamsPartitionAssignorTest.class,
StandbyTaskCreationIntegrationTest.class,
})
public class TaskSuite {
}

View File

@ -16,6 +16,9 @@
*/
package org.apache.kafka.streams.state;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.common.header.Headers;
@ -27,11 +30,12 @@ import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StreamPartitioner;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.processor.internals.RecordCollector;
import org.apache.kafka.streams.processor.internals.RecordCollectorImpl;
import org.apache.kafka.streams.state.internals.MeteredKeyValueStore;
@ -39,6 +43,7 @@ import org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreTest;
import org.apache.kafka.streams.state.internals.ThreadCache;
import org.apache.kafka.test.InternalMockProcessorContext;
import org.apache.kafka.test.MockTimestampExtractor;
import org.apache.kafka.test.StreamsTestUtils;
import org.apache.kafka.test.TestUtils;
import java.io.File;
@ -188,12 +193,15 @@ public class KeyValueStoreTestDriver<K, V> {
private KeyValueStoreTestDriver(final StateSerdes<K, V> serdes) {
final ByteArraySerializer rawSerializer = new ByteArraySerializer();
final Producer<byte[], byte[]> producer = new MockProducer<>(true, rawSerializer, rawSerializer);
final Consumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
final RecordCollector recordCollector = new RecordCollectorImpl(
"KeyValueStoreTestDriver",
new TaskId(0, 0),
new StreamsConfig(StreamsTestUtils.getStreamsConfig("test")),
new LogContext("KeyValueStoreTestDriver "),
new DefaultProductionExceptionHandler(),
new Metrics().sensor("dropped-records")
new MockStreamsMetrics(new Metrics()),
consumer,
id -> producer
) {
@Override
public <K1, V1> void send(final String topic,
@ -224,7 +232,6 @@ public class KeyValueStoreTestDriver<K, V> {
throw new UnsupportedOperationException();
}
};
recordCollector.init(producer);
final File stateDir = TestUtils.tempDirectory();
//noinspection ResultOfMethodCallIgnored

View File

@ -29,13 +29,17 @@ import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.TopologyWrapper;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.ProcessorTopology;
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
import org.apache.kafka.streams.processor.internals.ProcessorTopology;
import org.apache.kafka.streams.processor.internals.RecordCollector;
import org.apache.kafka.streams.processor.internals.RecordCollectorImpl;
import org.apache.kafka.streams.processor.internals.StateDirectory;
import org.apache.kafka.streams.processor.internals.StoreChangelogReader;
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
import org.apache.kafka.streams.processor.internals.StateDirectory;
import org.apache.kafka.streams.processor.internals.StreamTask;
import org.apache.kafka.streams.processor.internals.StreamThread;
import org.apache.kafka.streams.processor.internals.Task;
import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.state.ReadOnlyWindowStore;
@ -61,6 +65,8 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.instanceOf;
@ -75,7 +81,7 @@ public class StreamThreadStateStoreProviderTest {
private File stateDir;
private final String topicName = "topic";
private StreamThread threadMock;
private Map<TaskId, StreamTask> tasks;
private Map<TaskId, Task> tasks;
@Before
public void before() {
@ -138,7 +144,7 @@ public class StreamThreadStateStoreProviderTest {
clientSupplier,
processorTopology,
new TaskId(0, 0));
taskOne.initializeStateStores();
taskOne.initializeIfNeeded();
tasks.put(new TaskId(0, 0), taskOne);
final StreamTask taskTwo = createStreamsTask(
@ -146,7 +152,7 @@ public class StreamThreadStateStoreProviderTest {
clientSupplier,
processorTopology,
new TaskId(0, 1));
taskTwo.initializeStateStores();
taskTwo.initializeIfNeeded();
tasks.put(new TaskId(0, 1), taskTwo);
threadMock = EasyMock.createNiceMock(StreamThread.class);
@ -303,27 +309,45 @@ public class StreamThreadStateStoreProviderTest {
final ProcessorTopology topology,
final TaskId taskId) {
final Metrics metrics = new Metrics();
final LogContext logContext = new LogContext("test-stream-task ");
final Set<TopicPartition> partitions = Collections.singleton(new TopicPartition(topicName, taskId.partition));
final ProcessorStateManager stateManager = new ProcessorStateManager(
taskId,
partitions,
Task.TaskType.ACTIVE,
stateDirectory,
topology.storeToChangelogTopic(),
new StoreChangelogReader(
streamsConfig,
logContext,
clientSupplier.restoreConsumer,
new MockStateRestoreListener()),
logContext);
final RecordCollector recordCollector = new RecordCollectorImpl(
taskId,
streamsConfig,
logContext,
new MockStreamsMetrics(metrics),
clientSupplier.consumer,
id -> clientSupplier.getProducer(new HashMap<>()));
return new StreamTask(
taskId,
Collections.singleton(new TopicPartition(topicName, taskId.partition)),
partitions,
topology,
clientSupplier.consumer,
new StoreChangelogReader(
clientSupplier.restoreConsumer,
Duration.ZERO,
new MockStateRestoreListener(),
new LogContext("test-stream-task ")),
streamsConfig,
new MockStreamsMetrics(metrics),
stateDirectory,
null,
new MockTime(),
() -> clientSupplier.getProducer(new HashMap<>()));
stateManager,
recordCollector);
}
private void mockThread(final boolean initialized) {
EasyMock.expect(threadMock.isRunning()).andReturn(initialized);
EasyMock.expect(threadMock.activeTasks()).andStubReturn(tasks);
EasyMock.expect(threadMock.allTasks()).andStubReturn(tasks);
EasyMock.expect(threadMock.activeTasks()).andStubReturn(tasks.values().stream().collect(Collectors.toList()));
EasyMock.expect(threadMock.state()).andReturn(
initialized ? StreamThread.State.RUNNING : StreamThread.State.PARTITIONS_ASSIGNED
).anyTimes();

View File

@ -23,7 +23,6 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.utils.ByteBufferInputStream;
@ -43,7 +42,6 @@ import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo;
import org.apache.kafka.streams.processor.internals.assignment.AssignorError;
import org.apache.kafka.streams.processor.internals.assignment.LegacySubscriptionInfoSerde;
import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo;
import org.apache.kafka.streams.state.HostInfo;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
@ -149,7 +147,7 @@ public class StreamsUpgradeTest {
// 3. Task ids of valid local states on the client's state directory.
final TaskManager taskManager = taskManger();
final Set<TaskId> standbyTasks = taskManager.cachedTasksIds();
final Set<TaskId> standbyTasks = taskManager.tasksOnLocalStorage();
final Set<TaskId> activeTasks = prepareForSubscription(taskManager,
topics,
standbyTasks,
@ -213,24 +211,10 @@ public class StreamsUpgradeTest {
final List<TopicPartition> partitions = new ArrayList<>(assignment.partitions());
partitions.sort(PARTITION_COMPARATOR);
// version 1 field
final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
// version 2 fields
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
final Map<HostInfo, Set<TopicPartition>> partitionsByHost;
final Map<TopicPartition, TaskId> partitionsToTaskId = new HashMap<>();
processVersionTwoAssignment("test ", info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId);
partitionsByHost = info.partitionsByHost();
final Map<TaskId, Set<TopicPartition>> activeTasks = getActiveTasks(partitions, info);
final TaskManager taskManager = taskManger();
taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo));
taskManager.setHostPartitionMappings(partitionsByHost, info.standbyPartitionByHost());
taskManager.setPartitionsToTaskId(partitionsToTaskId);
taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks());
taskManager.updateSubscriptionsFromAssignment(partitions);
taskManager.setRebalanceInProgress(false);
taskManager.handleAssignment(activeTasks, info.standbyTasks());
usedSubscriptionMetadataVersionPeek.set(usedSubscriptionMetadataVersion);
}
@ -263,8 +247,8 @@ public class StreamsUpgradeTest {
final Subscription subscription = entry.getValue();
final SubscriptionInfo info = SubscriptionInfo.decode(subscription.userData()
.putInt(0, LATEST_SUPPORTED_VERSION)
.putInt(4, LATEST_SUPPORTED_VERSION));
.putInt(0, LATEST_SUPPORTED_VERSION)
.putInt(4, LATEST_SUPPORTED_VERSION));
downgradedSubscriptions.put(
entry.getKey(),
@ -327,7 +311,7 @@ public class StreamsUpgradeTest {
}
}
public ByteBuffer encode() {
private ByteBuffer encode() {
final ByteBuffer buf = encodeFutureVersion();
buf.rewind();
return buf;

View File

@ -23,8 +23,6 @@ import org.apache.kafka.streams.processor.internals.GlobalStateManager;
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
@ -49,25 +47,19 @@ public class GlobalStateManagerStub implements GlobalStateManager {
return storeNames;
}
@Override
public void reinitializeStateStoresForPartitions(final Collection<TopicPartition> partitions,
final InternalProcessorContext processorContext) {}
@Override
public File baseDir() {
return null;
}
@Override
public void register(final StateStore store,
final StateRestoreCallback stateRestoreCallback) {}
public void registerStore(final StateStore store, final StateRestoreCallback stateRestoreCallback) {}
@Override
public void flush() {}
@Override
public void close(final boolean clean) throws IOException {
this.offsets.putAll(offsets);
public void close() {
closed = true;
}
@ -76,18 +68,18 @@ public class GlobalStateManagerStub implements GlobalStateManager {
this.offsets.putAll(offsets);
}
@Override
public StateStore getGlobalStore(final String name) {
return null;
}
@Override
public StateStore getStore(final String name) {
return null;
}
@Override
public Map<TopicPartition, Long> checkpointed() {
public StateStore getGlobalStore(final String name) {
return null;
}
@Override
public Map<TopicPartition, Long> changelogOffsets() {
return offsets;
}
}

View File

@ -38,10 +38,6 @@ public class MockBatchingStateRestoreListener extends MockStateRestoreListener i
}
public void resetRestoredBatch() {
restoredRecords.clear();
}
public Collection<KeyValue<byte[], byte[]>> getRestoredRecords() {
return restoredRecords;
}

View File

@ -18,11 +18,9 @@ package org.apache.kafka.test;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.IntegerDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.StateRestoreCallback;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.internals.RecordCollector;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore;
@ -43,22 +41,11 @@ public class MockKeyValueStore implements KeyValueStore {
public boolean closed = true;
public final ArrayList<Integer> keys = new ArrayList<>();
public final ArrayList<byte[]> values = new ArrayList<>();
private final boolean simulateForwardOnFlush;
private RecordCollector collector;
public MockKeyValueStore(final String name,
final boolean persistent) {
this.name = name;
this.persistent = persistent;
simulateForwardOnFlush = false;
}
public MockKeyValueStore(final String name,
final boolean persistent,
final boolean simulateForwardOnFlush) {
this.name = name;
this.persistent = persistent;
this.simulateForwardOnFlush = simulateForwardOnFlush;
}
@Override
@ -70,18 +57,12 @@ public class MockKeyValueStore implements KeyValueStore {
public void init(final ProcessorContext context,
final StateStore root) {
context.register(root, stateRestoreCallback);
if (simulateForwardOnFlush) {
collector = ((RecordCollector.Supplier) context).recordCollector();
}
initialized = true;
closed = false;
}
@Override
public void flush() {
if (simulateForwardOnFlush) {
collector.send("any", "anykey", "anyvalue", null, 0, 0L, new StringSerializer(), new StringSerializer());
}
instanceLastFlushCount.set(GLOBAL_FLUSH_COUNTER.getAndIncrement());
flushed = true;
}
@ -117,9 +98,7 @@ public class MockKeyValueStore implements KeyValueStore {
};
@Override
public void put(final Object key, final Object value) {
}
public void put(final Object key, final Object value) {}
@Override
public Object putIfAbsent(final Object key, final Object value) {

View File

@ -16,7 +16,7 @@
*/
package org.apache.kafka.test;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
@ -36,6 +36,9 @@ public class MockRecordCollector implements RecordCollector {
// remember all records that are collected so far
private final List<ProducerRecord<Object, Object>> collected = new LinkedList<>();
// remember all commits that are submitted so far
private final List<Map<TopicPartition, OffsetAndMetadata>> committed = new LinkedList<>();
// remember if flushed is called
private boolean flushed = false;
@ -74,7 +77,9 @@ public class MockRecordCollector implements RecordCollector {
}
@Override
public void init(final Producer<byte[], byte[]> producer) {}
public void commit(final Map<TopicPartition, OffsetAndMetadata> offsets) {
committed.add(offsets);
}
@Override
public void flush() {
@ -93,6 +98,10 @@ public class MockRecordCollector implements RecordCollector {
return unmodifiableList(collected);
}
public List<Map<TopicPartition, OffsetAndMetadata>> committed() {
return unmodifiableList(committed);
}
public boolean flushed() {
return flushed;
}

View File

@ -48,8 +48,10 @@ public class MockStateRestoreListener extends AbstractNotifyingRestoreCallback {
}
@Override
public void onRestoreStart(final TopicPartition topicPartition, final String storeName,
final long startingOffset, final long endingOffset) {
public void onRestoreStart(final TopicPartition topicPartition,
final String storeName,
final long startingOffset,
final long endingOffset) {
restoreTopicPartition = topicPartition;
storeNameCalledStates.put(RESTORE_START, storeName);
restoreStartOffset = startingOffset;
@ -57,17 +59,19 @@ public class MockStateRestoreListener extends AbstractNotifyingRestoreCallback {
}
@Override
public void onBatchRestored(final TopicPartition topicPartition, final String storeName,
final long batchEndOffset, final long numRestored) {
public void onBatchRestored(final TopicPartition topicPartition,
final String storeName,
final long batchEndOffset,
final long numRestored) {
restoreTopicPartition = topicPartition;
storeNameCalledStates.put(RESTORE_BATCH, storeName);
restoredBatchOffset = batchEndOffset;
numBatchRestored = numRestored;
}
@Override
public void onRestoreEnd(final TopicPartition topicPartition, final String storeName,
public void onRestoreEnd(final TopicPartition topicPartition,
final String storeName,
final long totalRestored) {
restoreTopicPartition = topicPartition;
storeNameCalledStates.put(RESTORE_END, storeName);

View File

@ -114,15 +114,6 @@ public final class StreamsTestUtils {
return results;
}
public static <K, V> List<V> valuesToList(final Iterator<KeyValue<K, V>> iterator) {
final List<V> results = new ArrayList<>();
while (iterator.hasNext()) {
results.add(iterator.next().value);
}
return results;
}
public static <K, V> Set<KeyValue<K, V>> toSet(final Iterator<KeyValue<K, V>> iterator) {
final Set<KeyValue<K, V>> results = new HashSet<>();

View File

@ -43,6 +43,10 @@ import org.apache.kafka.streams.errors.TopologyException;
import org.apache.kafka.streams.internals.KeyValueStoreFacade;
import org.apache.kafka.streams.internals.QuietStreamsConfig;
import org.apache.kafka.streams.internals.WindowStoreFacade;
import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
import org.apache.kafka.streams.processor.internals.RecordCollector;
import org.apache.kafka.streams.processor.internals.RecordCollectorImpl;
import org.apache.kafka.streams.processor.internals.Task;
import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.PunctuationType;
@ -216,6 +220,17 @@ public class TopologyTestDriver implements Closeable {
private final Map<String, Queue<ProducerRecord<byte[], byte[]>>> outputRecordsByTopic = new HashMap<>();
private final boolean eosEnabled;
private final StateRestoreListener stateRestoreListener = new StateRestoreListener() {
@Override
public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) {}
@Override
public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) {}
@Override
public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) {}
};
/**
* Create a new test diver instance.
* Initialized the internally mocked wall-clock time with {@link System#currentTimeMillis() current system time}.
@ -312,16 +327,6 @@ public class TopologyTestDriver implements Closeable {
new LogContext("topology-test-driver "),
Math.max(0, streamsConfig.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG)),
streamsMetrics);
final StateRestoreListener stateRestoreListener = new StateRestoreListener() {
@Override
public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) {}
@Override
public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) {}
@Override
public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) {}
};
for (final InternalTopologyBuilder.TopicsInfo topicsInfo : internalTopologyBuilder.topicGroups().values()) {
internalTopics.addAll(topicsInfo.repartitionSourceTopics.keySet());
@ -377,24 +382,40 @@ public class TopologyTestDriver implements Closeable {
}
if (!partitionsByTopic.isEmpty()) {
final LogContext logContext = new LogContext("topology-test-driver ");
final ProcessorStateManager stateManager = new ProcessorStateManager(
TASK_ID,
new HashSet<>(partitionsByTopic.values()),
Task.TaskType.ACTIVE,
stateDirectory,
processorTopology.storeToChangelogTopic(),
new StoreChangelogReader(
streamsConfig,
logContext,
createRestoreConsumer(processorTopology.storeToChangelogTopic()),
stateRestoreListener),
logContext);
final RecordCollector recordCollector = new RecordCollectorImpl(
TASK_ID,
streamsConfig,
logContext,
streamsMetrics,
consumer,
taskId -> producer);
task = new StreamTask(
TASK_ID,
new HashSet<>(partitionsByTopic.values()),
processorTopology,
consumer,
new StoreChangelogReader(
createRestoreConsumer(processorTopology.storeToChangelogTopic()),
Duration.ZERO,
stateRestoreListener,
new LogContext("topology-test-driver ")),
streamsConfig,
streamsMetrics,
stateDirectory,
cache,
mockWallClockTime,
() -> producer);
task.initializeStateStores();
task.initializeTopology();
stateManager,
recordCollector);
task.initializeIfNeeded();
task.completeRestoration();
((InternalProcessorContext) task.context()).setRecordContext(new ProcessorRecordContext(
0L,
-1L,
@ -464,7 +485,7 @@ public class TopologyTestDriver implements Closeable {
headers)));
// Process the record ...
task.process();
task.process(mockWallClockTime.milliseconds());
task.maybePunctuateStreamTime();
task.commit();
captureOutputRecords();
@ -809,7 +830,7 @@ public class TopologyTestDriver implements Closeable {
}
if (globalStateManager != null) {
final StateStore stateStore = globalStateManager.getGlobalStore(name);
final StateStore stateStore = globalStateManager.getStore(name);
if (stateStore != null) {
if (throwForBuiltInStores) {
throwIfBuiltInStore(stateStore);
@ -978,7 +999,7 @@ public class TopologyTestDriver implements Closeable {
*/
public void close() {
if (task != null) {
task.close(true, false);
task.closeClean();
}
if (globalStateTask != null) {
try {

View File

@ -79,8 +79,6 @@ class StreamsBrokerCompatibility(Test):
@parametrize(broker_version=str(LATEST_1_1))
@parametrize(broker_version=str(LATEST_1_0))
@parametrize(broker_version=str(LATEST_0_11_0))
@parametrize(broker_version=str(LATEST_0_10_2))
@parametrize(broker_version=str(LATEST_0_10_1))
def test_compatible_brokers_eos_disabled(self, broker_version):
self.kafka.set_version(KafkaVersion(broker_version))
self.kafka.start()
@ -97,6 +95,8 @@ class StreamsBrokerCompatibility(Test):
self.consumer.stop()
self.kafka.stop()
@parametrize(broker_version=str(LATEST_0_10_2))
@parametrize(broker_version=str(LATEST_0_10_1))
@parametrize(broker_version=str(LATEST_0_10_0))
def test_fail_fast_on_incompatible_brokers(self, broker_version):
self.kafka.set_version(KafkaVersion(broker_version))
@ -106,9 +106,9 @@ class StreamsBrokerCompatibility(Test):
with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor:
processor.start()
monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support CREATE_TOPICS',
monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException',
timeout_sec=60,
err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support CREATE_TOPICS' error message " + str(processor.node.account))
err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException " + str(processor.node.account))
self.kafka.stop()