-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat: surface error to user when command topic deleted while server running #6240
Changes from 10 commits
2e06bfc
53a47a9
085317d
deb88e1
04bf50f
98e60fa
6e8a7e2
7f1cbfa
f63c88c
2d66b31
1ec87c0
0368e77
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,6 +22,7 @@ | |
import io.confluent.ksql.rest.server.state.ServerState; | ||
import io.confluent.ksql.rest.util.ClusterTerminator; | ||
import io.confluent.ksql.rest.util.TerminateCluster; | ||
import io.confluent.ksql.services.KafkaTopicClient; | ||
import io.confluent.ksql.util.Pair; | ||
import io.confluent.ksql.util.PersistentQueryMetadata; | ||
import io.confluent.ksql.util.RetryUtil; | ||
|
@@ -40,7 +41,9 @@ | |
import java.util.concurrent.atomic.AtomicReference; | ||
import java.util.function.Consumer; | ||
import java.util.function.Function; | ||
import java.util.function.Supplier; | ||
|
||
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException; | ||
import org.apache.kafka.common.errors.SerializationException; | ||
import org.apache.kafka.common.errors.WakeupException; | ||
import org.apache.kafka.common.serialization.Deserializer; | ||
|
@@ -81,6 +84,8 @@ public class CommandRunner implements Closeable { | |
private final Consumer<QueuedCommand> incompatibleCommandChecker; | ||
private final Errors errorHandler; | ||
private boolean incompatibleCommandDetected; | ||
private final Supplier<Boolean> commandTopicExists; | ||
private boolean commandTopicDeleted; | ||
private Status state = new Status(CommandRunnerStatus.RUNNING, CommandRunnerDegradedReason.NONE); | ||
|
||
public enum CommandRunnerStatus { | ||
|
@@ -91,8 +96,9 @@ public enum CommandRunnerStatus { | |
|
||
public enum CommandRunnerDegradedReason { | ||
NONE(errors -> ""), | ||
CORRUPTED(Errors:: commandRunnerDegradedBackupCorruptedErrorMessage), | ||
INCOMPATIBLE_COMMAND(Errors:: commandRunnerDegradedIncompatibleCommandsErrorMessage); | ||
CORRUPTED(Errors::commandRunnerDegradedBackupCorruptedErrorMessage), | ||
INCOMPATIBLE_COMMAND(Errors::commandRunnerDegradedIncompatibleCommandsErrorMessage), | ||
COMMAND_TOPIC_DELETED(Errors::commandRunnerDegradedCommandTopicDeletedErrorMessage); | ||
|
||
private final Function<Errors, String> msgFactory; | ||
|
||
|
@@ -110,8 +116,8 @@ public static class Status { | |
private final CommandRunnerDegradedReason degradedReason; | ||
|
||
public Status( | ||
final CommandRunnerStatus status, | ||
final CommandRunnerDegradedReason degradedReason | ||
final CommandRunnerStatus status, | ||
final CommandRunnerDegradedReason degradedReason | ||
) { | ||
this.status = status; | ||
this.degradedReason = degradedReason; | ||
|
@@ -137,7 +143,9 @@ public CommandRunner( | |
final Duration commandRunnerHealthTimeout, | ||
final String metricsGroupPrefix, | ||
final Deserializer<Command> commandDeserializer, | ||
final Errors errorHandler | ||
final Errors errorHandler, | ||
final KafkaTopicClient kafkaTopicClient, | ||
final String commandTopicName | ||
) { | ||
this( | ||
statementExecutor, | ||
|
@@ -156,7 +164,8 @@ public CommandRunner( | |
queuedCommand.getAndDeserializeCommand(commandDeserializer); | ||
}, | ||
commandDeserializer, | ||
errorHandler | ||
errorHandler, | ||
() -> kafkaTopicClient.isTopicExists(commandTopicName) | ||
); | ||
} | ||
|
||
|
@@ -176,7 +185,8 @@ public CommandRunner( | |
final Function<List<QueuedCommand>, List<QueuedCommand>> compactor, | ||
final Consumer<QueuedCommand> incompatibleCommandChecker, | ||
final Deserializer<Command> commandDeserializer, | ||
final Errors errorHandler | ||
final Errors errorHandler, | ||
final Supplier<Boolean> commandTopicExists | ||
) { | ||
// CHECKSTYLE_RULES.ON: ParameterNumberCheck | ||
this.statementExecutor = Objects.requireNonNull(statementExecutor, "statementExecutor"); | ||
|
@@ -199,7 +209,10 @@ public CommandRunner( | |
Objects.requireNonNull(commandDeserializer, "commandDeserializer"); | ||
this.errorHandler = | ||
Objects.requireNonNull(errorHandler, "errorHandler"); | ||
this.commandTopicExists = | ||
Objects.requireNonNull(commandTopicExists, "commandTopicExists"); | ||
this.incompatibleCommandDetected = false; | ||
this.commandTopicDeleted = false; | ||
} | ||
|
||
/** | ||
|
@@ -289,6 +302,9 @@ void fetchAndRunCommands() { | |
lastPollTime.set(clock.instant()); | ||
final List<QueuedCommand> commands = commandStore.getNewCommands(NEW_CMDS_TIMEOUT); | ||
if (commands.isEmpty()) { | ||
if (!commandTopicExists.get()) { | ||
commandTopicDeleted = true; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we can save this for another PR, but I think it makes sense to have this method return some status then to set a ton of flags and check them in the main loop There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll file a follow up issue for that refactoring improvement. |
||
} | ||
return; | ||
} | ||
|
||
|
@@ -428,6 +444,13 @@ public void run() { | |
CommandRunnerDegradedReason.CORRUPTED | ||
); | ||
closeEarly(); | ||
} else if (commandTopicDeleted) { | ||
LOG.warn("CommandRunner entering degraded state due to command topic deletion."); | ||
state = new Status( | ||
CommandRunnerStatus.DEGRADED, | ||
CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED | ||
); | ||
closeEarly(); | ||
} else { | ||
LOG.trace("Polling for new writes to command topic"); | ||
fetchAndRunCommands(); | ||
|
@@ -437,6 +460,13 @@ public void run() { | |
if (!closed) { | ||
throw wue; | ||
} | ||
} catch (final OffsetOutOfRangeException e) { | ||
LOG.warn("The command topic offset was reset. CommandRunner thread exiting."); | ||
state = new Status( | ||
CommandRunnerStatus.DEGRADED, | ||
CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. does offset reset always mean command topic deleted? it could mean the retention was configured incorrectly perhaps? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Every time we start up the server, we'd check/override the command topic properties such that the The REASON can be updated to |
||
); | ||
closeEarly(); | ||
} finally { | ||
commandStore.close(); | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I remember a discussion on the design doc indicated that maybe we don't want to differentiate between corrupted and deleted. What was the decision on that? (also good to document it here for posterity)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the main reason is right now, backups are optional, so if backups aren't enabled, then it wouldn't really make sense to have the degraded reason be CORRUPTED.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
After talking it over with Rohan a bit more, I've combined CORRUPTED and COMMAND_TOPIC_DELETED into CORRUPTED. The COMMAND_TOPIC_DELETED could show up as CORRUPTED after a restart and having different states for the same external action (deleting the command topic) is strange.
I've made the error message returned from the API generic enough so it address both the backup corruption and the command topic deleted/modified case