Skip to content

Commit

Permalink
HOTFIX: safely clear all active state in onPartitionsLost (apache#7691)
Browse files Browse the repository at this point in the history
After a number of last minute bugs were found stemming from the incremental closing of lost tasks in StreamsRebalanceListener#onPartitionsLost, a safer approach to this edge case seems warranted. We initially wanted to be as "future-proof" as possible, and avoid baking further protocol assumptions into the code that may be broken as the protocol evolves. This meant that rather than simply closing all active tasks and clearing all associated state in #onPartitionsLost(lostPartitions) we would loop through the lostPartitions/lost tasks and remove them one by one from the various data structures/assignments, then verify that everything was empty in the end. This verification in particular has caused us significant trouble, as it turns out to be nontrivial to determine what should in fact be empty, and if so whether it is also being correctly updated.

Therefore, before worrying about it being "future-proof" it seems we should make sure it is "present-day-proof" and implement this callback in the safest possible way, by blindly clearing and closing all active task state. We log all the relevant state (at debug level) before clearing it, so we can at least tell from the logs whether/which emptiness checks were being violated.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Andrew Choi <andchoi@linkedin.com>
  • Loading branch information
ableegoldman authored and bbejeck committed Nov 19, 2019
1 parent cca0225 commit 41a9e2c
Show file tree
Hide file tree
Showing 8 changed files with 156 additions and 170 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,8 @@ public interface ConsumerRebalanceListener {
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
*
* @param partitions The list of partitions that were assigned to the consumer and now have been reassigned
* to other consumers (may not include all currently assigned partitions, i.e. there may still
* to other consumers. With the current protocol this will always include all of the consumer's
* previously assigned partitions, but this may change in future protocols (ie there would still
* be some partitions left)
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
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;
Expand Down Expand Up @@ -79,7 +80,7 @@ List<TopicPartition> closeRevokedStandbyTasks(final Map<TaskId, Set<TopicPartiti
} catch (final RuntimeException e) {
log.error("Closing the standby task {} failed due to the following error:", task.id(), e);
} finally {
removeTaskFromRunning(task);
removeTaskFromAllStateMaps(task, Collections.emptyMap());
revokedChangelogs.addAll(task.changelogPartitions());
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.processor.internals;

import java.util.ArrayList;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.LogContext;
Expand Down Expand Up @@ -83,6 +84,15 @@ void closeTask(final StreamTask task, final boolean clean) {
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();
Expand Down Expand Up @@ -152,7 +162,7 @@ private RuntimeException suspendRunningTasks(final Set<TaskId> runningTasksToSus
id, f);
}
} finally {
removeTaskFromRunning(task);
removeTaskFromAllStateMaps(task, suspended);
taskChangelogs.addAll(task.changelogPartitions());
}
}
Expand Down Expand Up @@ -189,10 +199,8 @@ RuntimeException closeRestoringTasks(final Set<TaskId> restoringTasksToClose,
}

private RuntimeException closeRunning(final boolean isZombie,
final StreamTask task,
final List<TopicPartition> closedTaskChangelogs) {
removeTaskFromRunning(task);
closedTaskChangelogs.addAll(task.changelogPartitions());
final StreamTask task) {
removeTaskFromAllStateMaps(task, Collections.emptyMap());

try {
final boolean clean = !isZombie;
Expand All @@ -208,7 +216,7 @@ private RuntimeException closeRunning(final boolean isZombie,
private RuntimeException closeNonRunning(final boolean isZombie,
final StreamTask task,
final List<TopicPartition> closedTaskChangelogs) {
created.remove(task.id());
removeTaskFromAllStateMaps(task, Collections.emptyMap());
closedTaskChangelogs.addAll(task.changelogPartitions());

try {
Expand All @@ -221,10 +229,11 @@ private RuntimeException closeNonRunning(final boolean isZombie,
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) {
removeTaskFromRestoring(task);
removeTaskFromAllStateMaps(task, Collections.emptyMap());
closedTaskChangelogs.addAll(task.changelogPartitions());

try {
Expand All @@ -240,7 +249,7 @@ private RuntimeException closeRestoring(final boolean isZombie,

private RuntimeException closeSuspended(final boolean isZombie,
final StreamTask task) {
suspended.remove(task.id());
removeTaskFromAllStateMaps(task, Collections.emptyMap());

try {
final boolean clean = !isZombie;
Expand Down Expand Up @@ -269,37 +278,30 @@ RuntimeException closeNotAssignedSuspendedTasks(final Set<TaskId> revokedTasks)
return firstException.get();
}

RuntimeException closeZombieTasks(final Set<TaskId> lostTasks, final List<TopicPartition> lostTaskChangelogs) {
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 : lostTasks) {
if (suspended.containsKey(id)) {
log.debug("Closing the zombie suspended stream task {}.", id);
firstException.compareAndSet(null, closeSuspended(true, suspended.get(id)));
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), lostTaskChangelogs));
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), lostTaskChangelogs));
} else if (running.containsKey(id)) {
log.debug("Closing the zombie running stream task {}.", id);
firstException.compareAndSet(null, closeRunning(true, running.get(id), lostTaskChangelogs));
} else {
log.warn("Skipping closing the zombie stream task {} as it was already removed.", 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)));
}
}

// We always clear the prevActiveTasks and replace with current set of running tasks to encode in subscription
// We should exclude any tasks that were lost however, they will be counted as standbys for assignment purposes
prevActiveTasks.clear();
prevActiveTasks.addAll(running.keySet());
clear();

// With the current rebalance protocol, there should not be any running tasks left as they were all lost
if (!prevActiveTasks.isEmpty()) {
log.error("Found the still running stream tasks {} after closing all tasks lost as zombies", prevActiveTasks);
firstException.compareAndSet(null, new IllegalStateException("Not all lost tasks were closed as zombies"));
}
return firstException.get();
}

Expand All @@ -311,7 +313,7 @@ boolean maybeResumeSuspendedTask(final TaskId taskId,
if (suspended.containsKey(taskId)) {
final StreamTask task = suspended.get(taskId);
log.trace("Found suspended stream task {}", taskId);
suspended.remove(taskId);
removeTaskFromAllStateMaps(task, Collections.emptyMap());

if (task.partitions().equals(partitions)) {
task.resume();
Expand Down Expand Up @@ -346,8 +348,12 @@ void updateRestored(final Collection<TopicPartition> restored) {
if (restoredPartitions.containsAll(task.changelogPartitions())) {
transitionToRunning(task);
it.remove();
restoringByPartition.keySet().removeAll(task.partitions());
restoringByPartition.keySet().removeAll(task.changelogPartitions());
// 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());
Expand All @@ -372,6 +378,24 @@ void updateRestored(final Collection<TopicPartition> restored) {
}
}

@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()) {
Expand All @@ -382,16 +406,14 @@ void addTaskToRestoring(final StreamTask task) {
}
}

private void removeTaskFromRestoring(final StreamTask task) {
restoring.remove(task.id());
for (final TopicPartition topicPartition : task.partitions()) {
restoredPartitions.remove(topicPartition);
restoringByPartition.remove(topicPartition);
}
for (final TopicPartition topicPartition : task.changelogPartitions()) {
restoredPartitions.remove(topicPartition);
restoringByPartition.remove(topicPartition);
}
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());
}

/**
Expand Down Expand Up @@ -497,6 +519,7 @@ void clear() {
restoringByPartition.clear();
restoredPartitions.clear();
suspended.clear();
prevActiveTasks.clear();
}

@Override
Expand All @@ -511,26 +534,13 @@ public void shutdown(final boolean clean) {
super.shutdown(clean);
}

@Override
public boolean isEmpty() throws IllegalStateException {
if (restoring.isEmpty() && !restoringByPartition.isEmpty()) {
log.error("Assigned stream tasks in an inconsistent state: the set of restoring tasks is empty but the " +
"restoring by partitions map contained {}", restoringByPartition);
throw new IllegalStateException("Found inconsistent state: no tasks restoring but nonempty restoringByPartition");
} else {
return super.isEmpty()
&& restoring.isEmpty()
&& restoringByPartition.isEmpty()
&& restoredPartitions.isEmpty()
&& suspended.isEmpty();
}
}

public String toString(final String indent) {
final StringBuilder builder = new StringBuilder();
builder.append(super.toString(indent));
describe(builder, restoring.values(), indent, "Restoring:");
describe(builder, suspended.values(), indent, "Suspended:");
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();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,12 +69,17 @@ void initializeNewTasks() {
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
Expand Down Expand Up @@ -121,10 +126,25 @@ void transitionToRunning(final T task) {
}
}

void removeTaskFromRunning(final T task) {
running.remove(task.id());
runningByPartition.keySet().removeAll(task.partitions());
runningByPartition.keySet().removeAll(task.changelogPartitions());
/**
* 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) {
Expand All @@ -146,22 +166,34 @@ public String toString() {

public String toString(final String indent) {
final StringBuilder builder = new StringBuilder();
describe(builder, running.values(), indent, "Running:");
describe(builder, created.values(), indent, "New:");
describeTasks(builder, running.values(), indent, "Running:");
describePartitions(builder, runningByPartition.keySet(), indent, "Running Partitions:");
describeTasks(builder, created.values(), indent, "New:");
return builder.toString();
}

void describe(final StringBuilder builder,
final Collection<T> tasks,
final String indent,
final String name) {
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());
Expand All @@ -182,18 +214,6 @@ void clear() {
created.clear();
}

boolean isEmpty() throws IllegalStateException {
if (running.isEmpty() && !runningByPartition.isEmpty()) {
log.error("Assigned stream tasks in an inconsistent state: the set of running tasks is empty but the " +
"running by partitions map contained {}", runningByPartition);
throw new IllegalStateException("Found inconsistent state: no tasks running but nonempty runningByPartition");
} else {
return runningByPartition.isEmpty()
&& running.isEmpty()
&& created.isEmpty();
}
}

/**
* @throws TaskMigratedException if committing offsets failed (non-EOS)
* or if the task producer got fenced (EOS)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ private void startRestoration(final Set<TopicPartition> initialized,
restoreToOffsets.get(partition));
restorer.setStartingOffset(restoreConsumer.position(partition));

log.debug("Calling restorer for partition {} of task {}", partition, active.restoringTaskFor(partition));
log.debug("Calling restorer for partition {}", partition);
restorer.restoreStarted();
} else {
log.trace("Did not find checkpoint from changelog {} for store {}, rewinding to beginning.", partition, restorer.storeName());
Expand Down
Loading

0 comments on commit 41a9e2c

Please sign in to comment.