From 2e06bfc99fa013c42da60acbf6e6c0d1c7d759b4 Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Sun, 6 Sep 2020 19:56:16 -0700 Subject: [PATCH 1/9] test commit --- .../io/confluent/ksql/util/KsqlConfig.java | 7 ++++ .../ksql/rest/server/CommandTopicBackup.java | 4 ++ .../rest/server/CommandTopicBackupImpl.java | 42 ++++++++----------- .../rest/server/CommandTopicBackupNoOp.java | 10 +++++ .../ksql/rest/server/KsqlRestApplication.java | 26 ++++++++++-- .../rest/server/computation/CommandQueue.java | 5 +++ .../server/computation/CommandRunner.java | 7 +++- .../rest/server/computation/CommandStore.java | 30 ++++++------- .../computation/DistributingExecutor.java | 13 +++--- .../rest/server/resources/KsqlResource.java | 36 +++++++++++----- .../server/computation/CommandStoreTest.java | 6 ++- .../computation/DistributingExecutorTest.java | 13 +++--- .../rest/server/computation/RecoveryTest.java | 5 +++ .../server/resources/KsqlResourceTest.java | 15 +++++-- .../ksql/rest/DefaultErrorMessages.java | 19 +++++++++ .../io/confluent/ksql/rest/ErrorMessages.java | 2 + .../java/io/confluent/ksql/rest/Errors.java | 4 ++ 17 files changed, 172 insertions(+), 72 deletions(-) diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 39f2110732bb..cde3d62f1a9b 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -303,6 +303,13 @@ public class KsqlConfig extends AbstractConfig { public static final String KSQL_METASTORE_BACKUP_LOCATION_DOC = "Specify the directory where " + "KSQL metastore backup files are located."; + public static final String KSQL_METASTORE_BACKUP_HARD_FAIL = "ksql.metastore.backup.hard.fail"; + public static final Boolean KSQL_METASTORE_BACKUP_HARD_FAIL_DEFAULT = true; + public static final String KSQL_METASTORE_BACKUP_HARD_FAIL_DOC = + "Enable hard failing when the metastore backup and command topic are inconsistent. " + + "The server will enter a degraded state if the command topic isn't present but the metastore " + + "back up is present, or the command topic and the metastore have conflicting commands"; + public static final String KSQL_SUPPRESS_ENABLED = "ksql.suppress.enabled"; public static final Boolean KSQL_SUPPRESS_ENABLED_DEFAULT = false; public static final String KSQL_SUPPRESS_ENABLED_DOC = diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java index d12b23bb5d1b..87c3e3ecbc6f 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java @@ -22,5 +22,9 @@ public interface CommandTopicBackup { void writeRecord(ConsumerRecord record); + boolean commandTopicCorruption(); + + boolean backupExists(); + void close(); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java index b0fe42838c6c..6147777ccd6d 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java @@ -59,6 +59,7 @@ public long read() { private BackupReplayFile replayFile; private List> latestReplay; private int latestReplayIdx; + private boolean corruptionDetected; public CommandTopicBackupImpl(final String location, final String topicName) { this(location, topicName, CURRENT_MILLIS_TICKER); @@ -92,6 +93,7 @@ public void initialize() { } latestReplayIdx = 0; + corruptionDetected = false; LOG.info("Command topic will be backup on file: {}", replayFile.getPath()); } @@ -147,16 +149,20 @@ public void writeRecord(final ConsumerRecord record) { } void writeCommandToBackup(final ConsumerRecord record) { +// if (corruptionDetected) { +// LOG.warn("Failure to write command topic data to backup. Corruption detected in command topic."); +// } + if (isRestoring()) { if (isRecordInLatestReplay(record)) { // Ignore backup because record was already replayed return; } else { - LOG.info("Previous command topic backup does not match the new command topic data. " - + "A new backup file will be created."); - createNewBackupFile(); - latestReplay.clear(); - LOG.info("New backup file created: {}", replayFile.getPath()); + LOG.info("Previous command topic backup does not match the new command topic data."); + corruptionDetected = true; +// createNewBackupFile(); +// latestReplay.clear(); +// LOG.info("New backup file created: {}", replayFile.getPath()); } } else if (latestReplay.size() > 0) { // clear latest replay from memory @@ -173,26 +179,14 @@ void writeCommandToBackup(final ConsumerRecord record) { } } - private void createNewBackupFile() { - try { - replayFile.close(); - } catch (IOException e) { - LOG.warn("Couldn't close the current backup file {}. Error = {}", - replayFile.getPath(), e.getMessage()); - } - - replayFile = newReplayFile(); + @Override + public boolean commandTopicCorruption() { + return corruptionDetected; + } - if (latestReplay.size() > 0 && latestReplayIdx > 0) { - try { - replayFile.write(latestReplay.subList(0, latestReplayIdx)); - } catch (final IOException e) { - LOG.warn("Couldn't write the latest replayed commands to the new backup file {}. " - + "Make sure the file exists and has permissions to write. " - + "KSQL must be restarted afterwards to complete the backup process. Error = {}", - replayFile.getPath(), e.getMessage()); - } - } + @Override + public boolean backupExists() { + return latestReplayFile().isPresent(); } @VisibleForTesting diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java index efaafdc46629..b7465a16bcfe 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java @@ -32,4 +32,14 @@ public void writeRecord(final ConsumerRecord record) { public void close() { // no-op } + + @Override + public boolean commandTopicCorruption() { + return false; + } + + @Override + public boolean backupExists() { + return false; + } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index 2ca44b72ee3c..f8cdc9927139 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -188,6 +188,7 @@ public final class KsqlRestApplication implements Executable { private final CompletableFuture terminatedFuture = new CompletableFuture<>(); private final QueryMonitor queryMonitor; private final DenyListPropertyValidator denyListPropertyValidator; + private final CommandTopicBackup commandTopicBackup; // The startup thread that can be interrupted if necessary during shutdown. This should only // happen if startup hangs. @@ -224,7 +225,8 @@ public static SourceName getCommandsStreamName() { final Optional lagReportingAgent, final Vertx vertx, final QueryMonitor ksqlQueryMonitor, - final DenyListPropertyValidator denyListPropertyValidator + final DenyListPropertyValidator denyListPropertyValidator, + final CommandTopicBackup commandTopicBackup ) { log.debug("Creating instance of ksqlDB API server"); this.serviceContext = requireNonNull(serviceContext, "serviceContext"); @@ -253,6 +255,8 @@ public static SourceName getCommandsStreamName() { this.vertx = requireNonNull(vertx, "vertx"); this.denyListPropertyValidator = requireNonNull(denyListPropertyValidator, "denyListPropertyValidator"); + this.commandTopicBackup = + requireNonNull(commandTopicBackup, "commandTopicBackup"); this.serverInfoResource = new ServerInfoResource(serviceContext, ksqlConfigNoPort, commandRunner); @@ -656,6 +660,20 @@ static KsqlRestApplication buildApplication( final String commandTopicName = ReservedInternalTopics.commandTopic(ksqlConfig); + CommandTopicBackup commandTopicBackup = new CommandTopicBackupNoOp(); + if (ksqlConfig.getBoolean(KsqlConfig.KSQL_ENABLE_METASTORE_BACKUP)) { + if (ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION).isEmpty()) { + throw new KsqlException(String.format("Metastore backups is enabled, but location " + + "is empty. Please specify the location with the property '%s'", + KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION)); + } + + commandTopicBackup = new CommandTopicBackupImpl( + ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), + commandTopicName) + ; + } + final CommandStore commandStore = CommandStore.Factory.create( ksqlConfig, commandTopicName, @@ -663,7 +681,8 @@ static KsqlRestApplication buildApplication( ksqlConfig.addConfluentMetricsContextConfigsKafka( restConfig.getCommandConsumerProperties()), ksqlConfig.addConfluentMetricsContextConfigsKafka( - restConfig.getCommandProducerProperties()) + restConfig.getCommandProducerProperties()), + commandTopicBackup ); final InteractiveStatementExecutor statementExecutor = @@ -791,7 +810,8 @@ static KsqlRestApplication buildApplication( lagReportingAgent, vertx, queryMonitor, - denyListPropertyValidator + denyListPropertyValidator, + commandTopicBackup ); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java index e97e7795e27f..46d09dd77660 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java @@ -99,6 +99,11 @@ void ensureConsumedPast(long seqNum, Duration timeout) */ boolean isEmpty(); + /** + * @return whether or not data corruption is detected in the enqueued comamnds. + */ + boolean isCorrupted(); + /** * Cause any blocked {@link #getNewCommands(Duration)} calls to return early. * diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index c0fc3580b682..4a95c05f2cd3 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -82,7 +82,8 @@ public class CommandRunner implements Closeable { public enum CommandRunnerStatus { RUNNING, ERROR, - DEGRADED + DEGRADED, + CORRUPTED } public CommandRunner( @@ -311,6 +312,10 @@ private void terminateCluster(final Command command) { } public CommandRunnerStatus checkCommandRunnerStatus() { + if (commandStore.isCorrupted()) { + return CommandRunnerStatus.CORRUPTED; + } + if (incompatibleCommandDetected) { return CommandRunnerStatus.DEGRADED; } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 44b0d3b38bda..786172f5d819 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -76,6 +76,7 @@ public class CommandStore implements CommandQueue, Closeable { private final Serializer commandIdSerializer; private final Serializer commandSerializer; private final Deserializer commandIdDeserializer; + private final CommandTopicBackup commandTopicBackup; public static final class Factory { @@ -88,7 +89,8 @@ public static CommandStore create( final String commandTopicName, final Duration commandQueueCatchupTimeout, final Map kafkaConsumerProperties, - final Map kafkaProducerProperties + final Map kafkaProducerProperties, + final CommandTopicBackup commandTopicBackup ) { kafkaConsumerProperties.put( ConsumerConfig.ISOLATION_LEVEL_CONFIG, @@ -107,20 +109,6 @@ public static CommandStore create( "all" ); - CommandTopicBackup commandTopicBackup = new CommandTopicBackupNoOp(); - if (ksqlConfig.getBoolean(KsqlConfig.KSQL_ENABLE_METASTORE_BACKUP)) { - if (ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION).isEmpty()) { - throw new KsqlException(String.format("Metastore backups is enabled, but location " - + "is empty. Please specify the location with the property '%s'", - KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION)); - } - - commandTopicBackup = new CommandTopicBackupImpl( - ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), - commandTopicName) - ; - } - return new CommandStore( commandTopicName, new CommandTopic( @@ -134,7 +122,8 @@ public static CommandStore create( commandQueueCatchupTimeout, InternalTopicSerdes.serializer(), InternalTopicSerdes.serializer(), - InternalTopicSerdes.deserializer(CommandId.class) + InternalTopicSerdes.deserializer(CommandId.class), + commandTopicBackup ); } } @@ -148,7 +137,8 @@ public static CommandStore create( final Duration commandQueueCatchupTimeout, final Serializer commandIdSerializer, final Serializer commandSerializer, - final Deserializer commandIdDeserializer + final Deserializer commandIdDeserializer, + final CommandTopicBackup commandTopicBackup ) { this.commandTopic = Objects.requireNonNull(commandTopic, "commandTopic"); this.commandStatusMap = Maps.newConcurrentMap(); @@ -167,6 +157,7 @@ public static CommandStore create( Objects.requireNonNull(commandSerializer, "commandSerializer"); this.commandIdDeserializer = Objects.requireNonNull(commandIdDeserializer, "commandIdDeserializer"); + this.commandTopicBackup = Objects.requireNonNull(commandTopicBackup, "commandTopicBackup"); } @Override @@ -338,6 +329,11 @@ private long getCommandTopicOffset() { } } + @Override + public boolean isCorrupted() { + return commandTopicBackup.commandTopicCorruption(); + } + @Override public boolean isEmpty() { return commandTopic.getEndOffset() == 0; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java index 128602811a21..d4a946b3d21e 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java @@ -60,7 +60,7 @@ public class DistributingExecutor { private final CommandIdAssigner commandIdAssigner; private final ReservedInternalTopics internalTopics; private final Errors errorHandler; - private final Supplier commandRunnerDegraded; + private final Supplier commandRunnerWarning; public DistributingExecutor( final KsqlConfig ksqlConfig, @@ -70,7 +70,7 @@ public DistributingExecutor( final Optional authorizationValidator, final ValidatedCommandFactory validatedCommandFactory, final Errors errorHandler, - final Supplier commandRunnerDegraded + final Supplier commandRunnerWarning ) { this.commandQueue = commandQueue; this.distributedCmdResponseTimeout = @@ -86,8 +86,8 @@ public DistributingExecutor( this.internalTopics = new ReservedInternalTopics(Objects.requireNonNull(ksqlConfig, "ksqlConfig")); this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); - this.commandRunnerDegraded = - Objects.requireNonNull(commandRunnerDegraded, "commandRunnerDegraded"); + this.commandRunnerWarning = + Objects.requireNonNull(commandRunnerWarning, "commandRunnerWarning"); } /** @@ -105,10 +105,11 @@ public Optional execute( final KsqlExecutionContext executionContext, final KsqlSecurityContext securityContext ) { - if (commandRunnerDegraded.get()) { + final String commandRunnerWarningString = commandRunnerWarning.get(); + if (!commandRunnerWarningString.equals("")) { throw new KsqlServerException("Failed to handle Ksql Statement." + System.lineSeparator() - + errorHandler.commandRunnerDegradedErrorMessage()); + + commandRunnerWarningString); } final ConfiguredStatement injected = injectorFactory .apply(executionContext, securityContext.getServiceContext()) diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 421240cd1439..2b2c1d79f9a2 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -103,6 +103,7 @@ public class KsqlResource implements KsqlConfigurable { private final BiFunction injectorFactory; private final Optional authorizationValidator; private final DenyListPropertyValidator denyListPropertyValidator; + private final Supplier commandRunnerWarning; private RequestValidator validator; private RequestHandler handler; private final Errors errorHandler; @@ -126,7 +127,18 @@ public KsqlResource( Injectors.DEFAULT, authorizationValidator, errorHandler, - denyListPropertyValidator + denyListPropertyValidator, + () -> { + if (commandRunner.checkCommandRunnerStatus() + == CommandRunner.CommandRunnerStatus.DEGRADED) { + return errorHandler.commandRunnerDegradedErrorMessage(); + } + if (commandRunner.checkCommandRunnerStatus() + == CommandRunner.CommandRunnerStatus.CORRUPTED) { + return errorHandler.commandRunnerCorruptedErrorMessage(); + } + return ""; + } ); } @@ -138,7 +150,8 @@ public KsqlResource( final BiFunction injectorFactory, final Optional authorizationValidator, final Errors errorHandler, - final DenyListPropertyValidator denyListPropertyValidator + final DenyListPropertyValidator denyListPropertyValidator, + final Supplier commandRunnerWarning ) { this.ksqlEngine = Objects.requireNonNull(ksqlEngine, "ksqlEngine"); this.commandRunner = Objects.requireNonNull(commandRunner, "commandRunner"); @@ -152,6 +165,8 @@ public KsqlResource( this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); this.denyListPropertyValidator = Objects.requireNonNull(denyListPropertyValidator, "denyListPropertyValidator"); + this.commandRunnerWarning = + Objects.requireNonNull(commandRunnerWarning, "commandRunnerWarning"); } @Override @@ -190,8 +205,7 @@ public void configure(final KsqlConfig config) { authorizationValidator, new ValidatedCommandFactory(), errorHandler, - () -> commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.DEGRADED + commandRunnerWarning ), ksqlEngine, config, @@ -280,11 +294,10 @@ public EndpointResponse handleKsqlStatements( ); LOG.info("Processed successfully: " + request); - addCommandRunnerDegradedWarning( + addCommandRunnerWarning( entities, errorHandler, - () -> commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.DEGRADED); + commandRunnerWarning); return EndpointResponse.ok(entities); } catch (final KsqlRestException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); @@ -325,15 +338,16 @@ private static void ensureValidPatterns(final List deleteTopicList) { }); } - private static void addCommandRunnerDegradedWarning( + private static void addCommandRunnerWarning( final KsqlEntityList entityList, final Errors errorHandler, - final Supplier commandRunnerDegraded + final Supplier commandRunnerIssue ) { - if (commandRunnerDegraded.get()) { + final String commandRunnerIssueString = commandRunnerIssue.get(); + if (!commandRunnerIssueString.equals("")) { for (final KsqlEntity entity: entityList) { entity.updateWarnings(Collections.singletonList( - new KsqlWarning(errorHandler.commandRunnerDegradedErrorMessage()))); + new KsqlWarning(commandRunnerIssueString))); } } } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 2f534db14572..ee5b6b05efb7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -35,6 +35,7 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.server.CommandTopic; +import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.util.KsqlException; import java.time.Duration; import java.util.ArrayList; @@ -90,6 +91,8 @@ public class CommandStoreTest { private Serializer commandSerializer; @Mock private Deserializer commandIdDeserializer; + @Mock + private CommandTopicBackup commandTopicBackup; private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); @@ -150,7 +153,8 @@ public void setUp() { TIMEOUT, commandIdSerializer, commandSerializer, - commandIdDeserializer + commandIdDeserializer, + commandTopicBackup ); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java index 1b7b13fff433..dc8ecff07fff 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -46,6 +47,7 @@ import io.confluent.ksql.parser.tree.Statement; import io.confluent.ksql.parser.tree.TableElements; import io.confluent.ksql.properties.with.CommonCreateConfigs; +import io.confluent.ksql.rest.DefaultErrorMessages; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandId.Action; @@ -133,7 +135,7 @@ CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("json") @Mock private Errors errorHandler; @Mock - private Supplier commandRunnerDegraded; + private Supplier commandRunnerWarning; private DistributingExecutor distributor; private AtomicLong scnCounter; @@ -150,7 +152,7 @@ public void setUp() throws InterruptedException { when(status.getCommandSequenceNumber()).thenAnswer(inv -> scnCounter.incrementAndGet()); when(executionContext.getMetaStore()).thenReturn(metaStore); when(executionContext.createSandbox(any())).thenReturn(sandboxContext); - when(commandRunnerDegraded.get()).thenReturn(false); + when(commandRunnerWarning.get()).thenReturn(""); serviceContext = SandboxedServiceContext.create(TestServiceContext.create()); when(executionContext.getServiceContext()).thenReturn(serviceContext); when(validatedCommandFactory.create(any(), any())).thenReturn(command); @@ -166,7 +168,7 @@ public void setUp() throws InterruptedException { Optional.of(authorizationValidator), validatedCommandFactory, errorHandler, - commandRunnerDegraded + commandRunnerWarning ); } @@ -232,15 +234,16 @@ public void shouldReturnCommandStatus() { } @Test - public void shouldNotInitTransactionWhenCommandRunnerDegraded() { + public void shouldNotInitTransactionWhenCommandRunnerWarningPresent() { // When: - when(commandRunnerDegraded.get()).thenReturn(true); + when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE); // Then: assertThrows( KsqlServerException.class, () -> distributor.execute(CONFIGURED_STATEMENT, executionContext, securityContext) ); + verify(transactionalProducer, never()).initTransactions(); } @Test diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index d5f3bf22246a..4f1351bc42a7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -176,6 +176,11 @@ public boolean isEmpty() { return commandLog.isEmpty(); } + @Override + public boolean isCorrupted() { + return false; + } + @Override public void wakeup() { } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 8447478a2bb8..05396dabc21f 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -179,6 +179,7 @@ import java.util.Properties; import java.util.concurrent.TimeoutException; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.avro.Schema.Type; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -301,6 +302,8 @@ public class KsqlResourceTest { private Errors errorsHandler; @Mock private DenyListPropertyValidator denyListPropertyValidator; + @Mock + private Supplier commandRunnerWarning; private KsqlResource ksqlResource; private SchemaRegistryClient schemaRegistryClient; @@ -412,7 +415,8 @@ public void shouldThrowOnHandleStatementIfNotConfigured() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); // When: @@ -443,7 +447,8 @@ public void shouldThrowOnHandleTerminateIfNotConfigured() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); // When: @@ -2221,7 +2226,8 @@ private void setUpKsqlResource() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); ksqlResource.configure(ksqlConfig); @@ -2265,7 +2271,8 @@ public void shouldThrowOnDenyListValidatorWhenHandleKsqlStatement() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); final Map props = new HashMap<>(ksqlRestConfig.getKsqlConfigProperties()); props.put(KsqlConfig.KSQL_PROPERTIES_OVERRIDES_DENYLIST, diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index c4e4211c3dec..90119495bdc8 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -25,6 +25,20 @@ public class DefaultErrorMessages implements ErrorMessages { + System.lineSeparator() + "This is most likely due to the service being rolled back to an earlier version."; + public static final String COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE = + "The server has detected that the command topic may be corrupted. The backup of the " + + "command topic does not match the current contents of command topic." + + System.lineSeparator() + + "DDL statements will not be processed until either:" + + System.lineSeparator() + + "1. The current command topic is deleted and the backup file is used to restore the command topic." + + System.lineSeparator() + + "2. The current backup file is deleted." + + System.lineSeparator() + + "The server must be restarted after performing either operation in order to resume " + + "normal functionality"; + + @Override public String kafkaAuthorizationErrorMessage(final Exception e) { return ErrorMessageUtil.buildErrorMessage(e); @@ -53,4 +67,9 @@ public String schemaRegistryUnconfiguredErrorMessage(final Exception e) { public String commandRunnerDegradedErrorMessage() { return COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE; } + + @Override + public String metaStoreCorruptedErrorMessage() { + return COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE; + } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index 80aba990d596..98573f36b67c 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -24,4 +24,6 @@ public interface ErrorMessages { String schemaRegistryUnconfiguredErrorMessage(Exception e); String commandRunnerDegradedErrorMessage(); + + String metaStoreCorruptedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 920259f975e1..f3d62ec35fc3 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -230,6 +230,10 @@ public String commandRunnerDegradedErrorMessage() { return errorMessages.commandRunnerDegradedErrorMessage(); } + public String commandRunnerCorruptedErrorMessage() { + return errorMessages.metaStoreCorruptedErrorMessage(); + } + public EndpointResponse generateResponse( final Exception e, final EndpointResponse defaultResponse From 53a47a9a48ad3aa544a908ebb4b26ced31b8385b Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Wed, 9 Sep 2020 15:54:26 -0700 Subject: [PATCH 2/9] refactor --- .../io/confluent/ksql/util/KsqlConfig.java | 7 - .../ksql/rest/server/CommandTopic.java | 15 +- .../ksql/rest/server/CommandTopicBackup.java | 2 - .../rest/server/CommandTopicBackupImpl.java | 21 +-- .../rest/server/CommandTopicBackupNoOp.java | 5 - .../ksql/rest/server/KsqlRestApplication.java | 20 +-- .../rest/server/computation/CommandQueue.java | 5 - .../server/computation/CommandRunner.java | 73 ++++++-- ...sMetric.java => CommandRunnerMetrics.java} | 26 ++- .../rest/server/computation/CommandStore.java | 15 +- .../rest/server/resources/KsqlResource.java | 22 +-- .../server/CommandTopicBackupImplTest.java | 34 ++-- .../ksql/rest/server/CommandTopicTest.java | 49 +++++- .../computation/CommandRunnerMetricsTest.java | 159 ++++++++++++++++++ .../CommandRunnerStatusMetricTest.java | 117 ------------- .../server/computation/CommandRunnerTest.java | 53 +++++- .../server/computation/CommandStoreTest.java | 8 +- .../computation/DistributingExecutorTest.java | 2 +- .../rest/server/computation/RecoveryTest.java | 20 ++- .../server/resources/KsqlResourceTest.java | 12 +- .../ksql/rest/DefaultErrorMessages.java | 15 +- .../io/confluent/ksql/rest/ErrorMessages.java | 4 +- .../java/io/confluent/ksql/rest/Errors.java | 8 +- 23 files changed, 412 insertions(+), 280 deletions(-) rename ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/{CommandRunnerStatusMetric.java => CommandRunnerMetrics.java} (68%) create mode 100644 ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java delete mode 100644 ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index cde3d62f1a9b..39f2110732bb 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -303,13 +303,6 @@ public class KsqlConfig extends AbstractConfig { public static final String KSQL_METASTORE_BACKUP_LOCATION_DOC = "Specify the directory where " + "KSQL metastore backup files are located."; - public static final String KSQL_METASTORE_BACKUP_HARD_FAIL = "ksql.metastore.backup.hard.fail"; - public static final Boolean KSQL_METASTORE_BACKUP_HARD_FAIL_DEFAULT = true; - public static final String KSQL_METASTORE_BACKUP_HARD_FAIL_DOC = - "Enable hard failing when the metastore backup and command topic are inconsistent. " - + "The server will enter a degraded state if the command topic isn't present but the metastore " - + "back up is present, or the command topic and the metastore have conflicting commands"; - public static final String KSQL_SUPPRESS_ENABLED = "ksql.suppress.enabled"; public static final Boolean KSQL_SUPPRESS_ENABLED_DEFAULT = false; public static final String KSQL_SUPPRESS_ENABLED_DOC = diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java index 32348a60fc4a..8b7004d2a845 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java @@ -18,6 +18,7 @@ import com.google.common.collect.Lists; import io.confluent.ksql.rest.server.computation.QueuedCommand; import java.time.Duration; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -79,12 +80,18 @@ public void start() { public Iterable> getNewCommands(final Duration timeout) { final Iterable> iterable = commandConsumer.poll(timeout); + final List> records = new ArrayList<>(); if (iterable != null) { - iterable.forEach(this::backupRecord); + iterable.forEach(record -> { + backupRecord(record); + if (!commandTopicBackup.commandTopicCorruption()) { + records.add(record); + } + }); } - return iterable; + return records; } public List getRestoreCommands(final Duration duration) { @@ -100,6 +107,10 @@ public List getRestoreCommands(final Duration duration) { log.debug("Received {} records from poll", records.count()); for (final ConsumerRecord record : records) { backupRecord(record); + + if (commandTopicBackup.commandTopicCorruption()) { + continue; + } if (record.value() == null) { continue; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java index 87c3e3ecbc6f..06331f5046b6 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java @@ -24,7 +24,5 @@ public interface CommandTopicBackup { boolean commandTopicCorruption(); - boolean backupExists(); - void close(); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java index 6147777ccd6d..6cc6cceb3cf6 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java @@ -61,7 +61,9 @@ public long read() { private int latestReplayIdx; private boolean corruptionDetected; - public CommandTopicBackupImpl(final String location, final String topicName) { + public CommandTopicBackupImpl( + final String location, + final String topicName) { this(location, topicName, CURRENT_MILLIS_TICKER); } @@ -149,9 +151,11 @@ public void writeRecord(final ConsumerRecord record) { } void writeCommandToBackup(final ConsumerRecord record) { -// if (corruptionDetected) { -// LOG.warn("Failure to write command topic data to backup. Corruption detected in command topic."); -// } + if (corruptionDetected) { + LOG.warn("Failure to write command topic data to backup. " + + "Corruption detected in command topic."); + return; + } if (isRestoring()) { if (isRecordInLatestReplay(record)) { @@ -160,9 +164,7 @@ void writeCommandToBackup(final ConsumerRecord record) { } else { LOG.info("Previous command topic backup does not match the new command topic data."); corruptionDetected = true; -// createNewBackupFile(); -// latestReplay.clear(); -// LOG.info("New backup file created: {}", replayFile.getPath()); + return; } } else if (latestReplay.size() > 0) { // clear latest replay from memory @@ -184,11 +186,6 @@ public boolean commandTopicCorruption() { return corruptionDetected; } - @Override - public boolean backupExists() { - return latestReplayFile().isPresent(); - } - @VisibleForTesting BackupReplayFile openOrCreateReplayFile() { final Optional latestFile = latestReplayFile(); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java index b7465a16bcfe..1aca709cb400 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java @@ -37,9 +37,4 @@ public void close() { public boolean commandTopicCorruption() { return false; } - - @Override - public boolean backupExists() { - return false; - } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index f8cdc9927139..e1e5fc6c6b64 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -188,7 +188,6 @@ public final class KsqlRestApplication implements Executable { private final CompletableFuture terminatedFuture = new CompletableFuture<>(); private final QueryMonitor queryMonitor; private final DenyListPropertyValidator denyListPropertyValidator; - private final CommandTopicBackup commandTopicBackup; // The startup thread that can be interrupted if necessary during shutdown. This should only // happen if startup hangs. @@ -225,8 +224,7 @@ public static SourceName getCommandsStreamName() { final Optional lagReportingAgent, final Vertx vertx, final QueryMonitor ksqlQueryMonitor, - final DenyListPropertyValidator denyListPropertyValidator, - final CommandTopicBackup commandTopicBackup + final DenyListPropertyValidator denyListPropertyValidator ) { log.debug("Creating instance of ksqlDB API server"); this.serviceContext = requireNonNull(serviceContext, "serviceContext"); @@ -255,8 +253,6 @@ public static SourceName getCommandsStreamName() { this.vertx = requireNonNull(vertx, "vertx"); this.denyListPropertyValidator = requireNonNull(denyListPropertyValidator, "denyListPropertyValidator"); - this.commandTopicBackup = - requireNonNull(commandTopicBackup, "commandTopicBackup"); this.serverInfoResource = new ServerInfoResource(serviceContext, ksqlConfigNoPort, commandRunner); @@ -670,9 +666,10 @@ static KsqlRestApplication buildApplication( commandTopicBackup = new CommandTopicBackupImpl( ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), - commandTopicName) - ; + commandTopicName + ); } + final CommandTopicBackup finalCommandTopicBackup = commandTopicBackup; final CommandStore commandStore = CommandStore.Factory.create( ksqlConfig, @@ -682,7 +679,7 @@ static KsqlRestApplication buildApplication( restConfig.getCommandConsumerProperties()), ksqlConfig.addConfluentMetricsContextConfigsKafka( restConfig.getCommandProducerProperties()), - commandTopicBackup + finalCommandTopicBackup ); final InteractiveStatementExecutor statementExecutor = @@ -757,7 +754,9 @@ static KsqlRestApplication buildApplication( Duration.ofMillis(restConfig.getLong( KsqlRestConfig.KSQL_COMMAND_RUNNER_BLOCKED_THRESHHOLD_ERROR_MS)), metricsPrefix, - InternalTopicSerdes.deserializer(Command.class) + InternalTopicSerdes.deserializer(Command.class), + finalCommandTopicBackup, + errorHandler ); final KsqlResource ksqlResource = new KsqlResource( @@ -810,8 +809,7 @@ static KsqlRestApplication buildApplication( lagReportingAgent, vertx, queryMonitor, - denyListPropertyValidator, - commandTopicBackup + denyListPropertyValidator ); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java index 46d09dd77660..e97e7795e27f 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java @@ -99,11 +99,6 @@ void ensureConsumedPast(long seqNum, Duration timeout) */ boolean isEmpty(); - /** - * @return whether or not data corruption is detected in the enqueued comamnds. - */ - boolean isCorrupted(); - /** * Cause any blocked {@link #getNewCommands(Duration)} calls to return early. * diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 4a95c05f2cd3..24b7edd11172 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -16,7 +16,9 @@ package io.confluent.ksql.rest.server.computation; import com.google.common.annotations.VisibleForTesting; +import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.ClusterTerminateRequest; +import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -39,6 +41,8 @@ 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.common.errors.SerializationException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.serialization.Deserializer; @@ -69,7 +73,7 @@ public class CommandRunner implements Closeable { private final ClusterTerminator clusterTerminator; private final ServerState serverState; - private final CommandRunnerStatusMetric commandRunnerStatusMetric; + private final CommandRunnerMetrics commandRunnerMetric; private final AtomicReference> currentCommandRef; private final AtomicReference lastPollTime; private final Duration commandRunnerHealthTimeout; @@ -77,15 +81,23 @@ public class CommandRunner implements Closeable { private final Deserializer commandDeserializer; private final Consumer incompatibleCommandChecker; + private final Supplier backupCorrupted; + private final Errors errorHandler; private boolean incompatibleCommandDetected; public enum CommandRunnerStatus { RUNNING, ERROR, - DEGRADED, + DEGRADED + } + + public enum CommandRunnerDegradedReason { + NONE, + INCOMPATIBLE_COMMAND, CORRUPTED } + // CHECKSTYLE_RULES.OFF: ParameterNumberCheck public CommandRunner( final InteractiveStatementExecutor statementExecutor, final CommandQueue commandStore, @@ -95,7 +107,9 @@ public CommandRunner( final String ksqlServiceId, final Duration commandRunnerHealthTimeout, final String metricsGroupPrefix, - final Deserializer commandDeserializer + final Deserializer commandDeserializer, + final CommandTopicBackup commandTopicBackup, + final Errors errorHandler ) { this( statementExecutor, @@ -113,7 +127,9 @@ public CommandRunner( queuedCommand.getAndDeserializeCommandId(); queuedCommand.getAndDeserializeCommand(commandDeserializer); }, - commandDeserializer + commandDeserializer, + commandTopicBackup::commandTopicCorruption, + errorHandler ); } @@ -132,7 +148,9 @@ public CommandRunner( final Clock clock, final Function, List> compactor, final Consumer incompatibleCommandChecker, - final Deserializer commandDeserializer + final Deserializer commandDeserializer, + final Supplier backupCorrupted, + final Errors errorHandler ) { // CHECKSTYLE_RULES.ON: ParameterNumberCheck this.statementExecutor = Objects.requireNonNull(statementExecutor, "statementExecutor"); @@ -145,14 +163,18 @@ public CommandRunner( Objects.requireNonNull(commandRunnerHealthTimeout, "commandRunnerHealthTimeout"); this.currentCommandRef = new AtomicReference<>(null); this.lastPollTime = new AtomicReference<>(null); - this.commandRunnerStatusMetric = - new CommandRunnerStatusMetric(ksqlServiceId, this, metricsGroupPrefix); + this.commandRunnerMetric = + new CommandRunnerMetrics(ksqlServiceId, this, metricsGroupPrefix); this.clock = Objects.requireNonNull(clock, "clock"); this.compactor = Objects.requireNonNull(compactor, "compactor"); this.incompatibleCommandChecker = Objects.requireNonNull(incompatibleCommandChecker, "incompatibleCommandChecker"); this.commandDeserializer = Objects.requireNonNull(commandDeserializer, "commandDeserializer"); + this.backupCorrupted = + Objects.requireNonNull(backupCorrupted, "backupCorrupted"); + this.errorHandler = + Objects.requireNonNull(errorHandler, "errorHandler"); this.incompatibleCommandDetected = false; } @@ -173,7 +195,7 @@ public void close() { if (!closed) { closeEarly(); } - commandRunnerStatusMetric.close(); + commandRunnerMetric.close(); } /** @@ -312,11 +334,7 @@ private void terminateCluster(final Command command) { } public CommandRunnerStatus checkCommandRunnerStatus() { - if (commandStore.isCorrupted()) { - return CommandRunnerStatus.CORRUPTED; - } - - if (incompatibleCommandDetected) { + if (incompatibleCommandDetected || backupCorrupted.get()) { return CommandRunnerStatus.DEGRADED; } @@ -333,6 +351,30 @@ public CommandRunnerStatus checkCommandRunnerStatus() { ? CommandRunnerStatus.RUNNING : CommandRunnerStatus.ERROR; } + public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { + if (backupCorrupted.get()) { + return CommandRunnerDegradedReason.CORRUPTED; + } + + if (incompatibleCommandDetected) { + return CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND; + } + + return CommandRunnerDegradedReason.NONE; + } + + public String getCommandRunnerDegradedWarning() { + if (backupCorrupted.get()) { + return errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage(); + } + + if (incompatibleCommandDetected) { + return errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage(); + } + + return ""; + } + private List checkForIncompatibleCommands(final List commands) { final List compatibleCommands = new ArrayList<>(); try { @@ -357,8 +399,9 @@ private class Runner implements Runnable { public void run() { try { while (!closed) { - if (incompatibleCommandDetected) { - LOG.warn("CommandRunner entering degraded state after failing to deserialize command"); + if (incompatibleCommandDetected || backupCorrupted.get()) { + LOG.warn("CommandRunner entering degraded state due to: {}", + getCommandRunnerDegradedReason()); closeEarly(); } else { LOG.trace("Polling for new writes to command topic"); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetric.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java similarity index 68% rename from ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetric.java rename to ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java index 4f605300cc19..4dc1e62b5096 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetric.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java @@ -26,17 +26,18 @@ import org.apache.kafka.common.metrics.Metrics; /** - * Emits a JMX metric that indicates the health of the CommandRunner thread. + * Emits JMX metrics that for the CommandRunner thread. */ -public class CommandRunnerStatusMetric implements Closeable { +public class CommandRunnerMetrics implements Closeable { private static final String DEFAULT_METRIC_GROUP_PREFIX = "ksql-rest-app"; private static final String METRIC_GROUP_POST_FIX = "-command-runner"; private final Metrics metrics; - private final MetricName metricName; + private final MetricName commandRunnerStatusMetricName; + private final MetricName commandRunnerDegradedReasonMetricName; - CommandRunnerStatusMetric( + CommandRunnerMetrics( final String ksqlServiceId, final CommandRunner commandRunner, final String metricGroupPrefix @@ -50,7 +51,7 @@ public class CommandRunnerStatusMetric implements Closeable { } @VisibleForTesting - CommandRunnerStatusMetric( + CommandRunnerMetrics( final Metrics metrics, final CommandRunner commandRunner, final String ksqlServiceId, @@ -58,15 +59,23 @@ public class CommandRunnerStatusMetric implements Closeable { ) { this.metrics = Objects.requireNonNull(metrics, "metrics"); final String metricGroupName = metricsGroupPrefix + METRIC_GROUP_POST_FIX; - this.metricName = metrics.metricName( + this.commandRunnerStatusMetricName = metrics.metricName( "status", ReservedInternalTopics.KSQL_INTERNAL_TOPIC_PREFIX + ksqlServiceId + metricGroupName, "The status of the commandRunner thread as it processes the command topic.", Collections.emptyMap() ); + this.commandRunnerDegradedReasonMetricName = metrics.metricName( + "degraded-reason", + ReservedInternalTopics.KSQL_INTERNAL_TOPIC_PREFIX + ksqlServiceId + metricGroupName, + "The reason for why the commandRunner thread is in a DEGRADED state.", + Collections.emptyMap() + ); - this.metrics.addMetric(metricName, (Gauge) + this.metrics.addMetric(commandRunnerStatusMetricName, (Gauge) (config, now) -> commandRunner.checkCommandRunnerStatus().name()); + this.metrics.addMetric(commandRunnerDegradedReasonMetricName, (Gauge) + (config, now) -> commandRunner.getCommandRunnerDegradedReason().name()); } /** @@ -74,6 +83,7 @@ public class CommandRunnerStatusMetric implements Closeable { */ @Override public void close() { - metrics.removeMetric(metricName); + metrics.removeMetric(commandRunnerStatusMetricName); + metrics.removeMetric(commandRunnerDegradedReasonMetricName); } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 786172f5d819..06741dee3695 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -20,8 +20,6 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.server.CommandTopic; import io.confluent.ksql.rest.server.CommandTopicBackup; -import io.confluent.ksql.rest.server.CommandTopicBackupImpl; -import io.confluent.ksql.rest.server.CommandTopicBackupNoOp; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.KsqlServerException; @@ -76,7 +74,6 @@ public class CommandStore implements CommandQueue, Closeable { private final Serializer commandIdSerializer; private final Serializer commandSerializer; private final Deserializer commandIdDeserializer; - private final CommandTopicBackup commandTopicBackup; public static final class Factory { @@ -122,8 +119,7 @@ public static CommandStore create( commandQueueCatchupTimeout, InternalTopicSerdes.serializer(), InternalTopicSerdes.serializer(), - InternalTopicSerdes.deserializer(CommandId.class), - commandTopicBackup + InternalTopicSerdes.deserializer(CommandId.class) ); } } @@ -137,8 +133,7 @@ public static CommandStore create( final Duration commandQueueCatchupTimeout, final Serializer commandIdSerializer, final Serializer commandSerializer, - final Deserializer commandIdDeserializer, - final CommandTopicBackup commandTopicBackup + final Deserializer commandIdDeserializer ) { this.commandTopic = Objects.requireNonNull(commandTopic, "commandTopic"); this.commandStatusMap = Maps.newConcurrentMap(); @@ -157,7 +152,6 @@ public static CommandStore create( Objects.requireNonNull(commandSerializer, "commandSerializer"); this.commandIdDeserializer = Objects.requireNonNull(commandIdDeserializer, "commandIdDeserializer"); - this.commandTopicBackup = Objects.requireNonNull(commandTopicBackup, "commandTopicBackup"); } @Override @@ -329,11 +323,6 @@ private long getCommandTopicOffset() { } } - @Override - public boolean isCorrupted() { - return commandTopicBackup.commandTopicCorruption(); - } - @Override public boolean isEmpty() { return commandTopic.getEndOffset() == 0; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 2b2c1d79f9a2..4013bd54b96b 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -128,17 +128,7 @@ public KsqlResource( authorizationValidator, errorHandler, denyListPropertyValidator, - () -> { - if (commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.DEGRADED) { - return errorHandler.commandRunnerDegradedErrorMessage(); - } - if (commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.CORRUPTED) { - return errorHandler.commandRunnerCorruptedErrorMessage(); - } - return ""; - } + commandRunner::getCommandRunnerDegradedWarning ); } @@ -296,20 +286,23 @@ public EndpointResponse handleKsqlStatements( LOG.info("Processed successfully: " + request); addCommandRunnerWarning( entities, - errorHandler, commandRunnerWarning); return EndpointResponse.ok(entities); } catch (final KsqlRestException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); throw e; } catch (final KsqlStatementException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); return Errors.badStatement(e.getRawMessage(), e.getSqlStatement()); } catch (final KsqlException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); return errorHandler.generateResponse(e, Errors.badRequest(e)); } catch (final Exception e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); return errorHandler.generateResponse( e, Errors.serverErrorForStatement(e, request.getKsql())); } @@ -340,10 +333,9 @@ private static void ensureValidPatterns(final List deleteTopicList) { private static void addCommandRunnerWarning( final KsqlEntityList entityList, - final Errors errorHandler, - final Supplier commandRunnerIssue + final Supplier commandRunnerWarning ) { - final String commandRunnerIssueString = commandRunnerIssue.get(); + final String commandRunnerIssueString = commandRunnerWarning.get(); if (!commandRunnerIssueString.equals("")) { for (final KsqlEntity entity: entityList) { entity.updateWarnings(Collections.singletonList( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java index e88d9f49f412..6e7cb8c24fd3 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThrows; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; import com.google.common.base.Ticker; import io.confluent.ksql.rest.entity.CommandId; @@ -187,8 +188,10 @@ public void shouldIgnoreRecordPreviouslyReplayed() throws IOException { } @Test - public void shouldCreateNewReplayFileIfNewRecordsDoNotMatchPreviousBackups() throws IOException { + public void shouldNotCreateNewReplayFileIfNewRecordsDoNotMatchPreviousBackups() throws IOException { // Given + commandTopicBackup = new CommandTopicBackupImpl( + backupLocation.getRoot().getAbsolutePath(), COMMAND_TOPIC_NAME, ticker); final ConsumerRecord record1 = newConsumerRecord(command1); commandTopicBackup.initialize(); commandTopicBackup.writeCommandToBackup(record1); @@ -198,22 +201,13 @@ public void shouldCreateNewReplayFileIfNewRecordsDoNotMatchPreviousBackups() thr // A 2nd initialize call will open the latest backup and read the previous replayed commands commandTopicBackup.initialize(); final ConsumerRecord record2 = newConsumerRecord(command2); - // Need to increase the ticker so the new file has a new timestamp - when(ticker.read()).thenReturn(2L); - // The write command will create a new replay file with the new command + // The write command will conflicts with what's already in the backup file commandTopicBackup.writeCommandToBackup(record2); final BackupReplayFile currentReplayFile = commandTopicBackup.getReplayFile(); // Then - List> commands = previousReplayFile.readRecords(); - assertThat(commands.size(), is(1)); - assertThat(commands.get(0).left, is(command1.left)); - assertThat(commands.get(0).right, is(command1.right)); - commands = currentReplayFile.readRecords(); - assertThat(commands.size(), is(1)); - assertThat(commands.get(0).left, is(command2.left)); - assertThat(commands.get(0).right, is(command2.right)); - assertThat(currentReplayFile.getPath(), not(previousReplayFile.getPath())); + assertThat(currentReplayFile.getPath(), is(previousReplayFile.getPath())); + assertThat(commandTopicBackup.commandTopicCorruption(), is(true)); } @Test @@ -229,12 +223,9 @@ public void shouldWritePreviousReplayedRecordsAlreadyChecked() throws IOExceptio // When // A 2nd initialize call will open the latest backup and read the previous replayed commands commandTopicBackup.initialize(); - // Need to increase the ticker so the new file has a new timestamp - when(ticker.read()).thenReturn(2L); // command1 is ignored because it was previously replayed commandTopicBackup.writeCommandToBackup(record1); - // The write command will create a new replay file with the new command, and command1 will - // be written to have a complete backup + // The write command will conflicts with what's already in the backup file final ConsumerRecord record3 = newConsumerRecord(command3); commandTopicBackup.writeCommandToBackup(record3); final BackupReplayFile currentReplayFile = commandTopicBackup.getReplayFile(); @@ -246,13 +237,16 @@ public void shouldWritePreviousReplayedRecordsAlreadyChecked() throws IOExceptio assertThat(commands.get(0).right, is(command1.right)); assertThat(commands.get(1).left, is(command2.left)); assertThat(commands.get(1).right, is(command2.right)); + + // the backup file should be the same and the contents shouldn't have been modified commands = currentReplayFile.readRecords(); assertThat(commands.size(), is(2)); assertThat(commands.get(0).left, is(command1.left)); assertThat(commands.get(0).right, is(command1.right)); - assertThat(commands.get(1).left, is(command3.left)); - assertThat(commands.get(1).right, is(command3.right)); - assertThat(currentReplayFile.getPath(), not(previousReplayFile.getPath())); + assertThat(commands.get(1).left, is(command2.left)); + assertThat(commands.get(1).right, is(command2.right)); + assertThat(currentReplayFile.getPath(), is(previousReplayFile.getPath())); + assertThat(commandTopicBackup.commandTopicCorruption(), is(true)); } @Test diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java index a88cf2a38c3d..840f73b42471 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java @@ -16,6 +16,8 @@ package io.confluent.ksql.rest.server; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.sameInstance; import static org.hamcrest.core.IsEqual.equalTo; import static org.mockito.ArgumentMatchers.eq; @@ -32,7 +34,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import io.confluent.ksql.rest.server.computation.QueuedCommand; @@ -40,7 +44,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; -import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -62,6 +65,8 @@ public class CommandTopicTest { @Mock private CommandTopicBackup commandTopicBackup; + @Mock + private TopicPartition topicPartition; private final byte[] commandId1 = "commandId1".getBytes(Charset.defaultCharset()); private final byte[] command1 = "command1".getBytes(Charset.defaultCharset()); @@ -73,9 +78,7 @@ public class CommandTopicTest { private ConsumerRecord record1; private ConsumerRecord record2; private ConsumerRecord record3; - - @Mock private ConsumerRecords consumerRecords; @Captor private ArgumentCaptor> topicPartitionsCaptor; @@ -88,7 +91,10 @@ public void setup() { record1 = new ConsumerRecord<>("topic", 0, 0, commandId1, command1); record2 = new ConsumerRecord<>("topic", 0, 1, commandId2, command2); record3 = new ConsumerRecord<>("topic", 0, 2, commandId3, command3); + consumerRecords = + new ConsumerRecords<>(Collections.singletonMap(topicPartition, ImmutableList.of(record1, record2, record3))); commandTopic = new CommandTopic(COMMAND_TOPIC_NAME, commandConsumer, commandTopicBackup); + when(commandTopicBackup.commandTopicCorruption()).thenReturn(false); } @Test @@ -102,16 +108,49 @@ public void shouldAssignCorrectPartitionToConsumer() { } @Test - public void shouldGetNewCommandsIteratorCorrectly() { + public void shouldGetCommandsThatDoNotCorruptBackup() { // Given: when(commandConsumer.poll(any(Duration.class))).thenReturn(consumerRecords); + when(commandTopicBackup.commandTopicCorruption()) + .thenReturn(false) + .thenReturn(false) + .thenReturn(true); // When: final Iterable> newCommands = commandTopic .getNewCommands(Duration.ofHours(1)); + final List> newCommandsList = ImmutableList.copyOf(newCommands); + + // Then: + assertThat(newCommandsList.size(), is(2)); + assertThat(newCommandsList, equalTo(ImmutableList.of(record1, record2))); + } + + @Test + public void shouldGetCommandsThatDoNotCorruptBackupInRestore() { + // Given: + when(commandConsumer.poll(any(Duration.class))) + .thenReturn(someConsumerRecords( + record1, + record2)) + .thenReturn(someConsumerRecords( + record3)) + .thenReturn(new ConsumerRecords<>(Collections.emptyMap())); + when(commandTopicBackup.commandTopicCorruption()) + .thenReturn(false) + .thenReturn(true) + .thenReturn(true); + + // When: + final List queuedCommandList = commandTopic + .getRestoreCommands(Duration.ofMillis(1)); // Then: - assertThat(newCommands, sameInstance(consumerRecords)); + verify(commandConsumer).seekToBeginning(topicPartitionsCaptor.capture()); + assertThat(topicPartitionsCaptor.getValue(), + equalTo(Collections.singletonList(new TopicPartition(COMMAND_TOPIC_NAME, 0)))); + assertThat(queuedCommandList, equalTo(ImmutableList.of( + new QueuedCommand(commandId1, command1, Optional.empty(), 0L)))); } @Test diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java new file mode 100644 index 000000000000..136fef78b4fe --- /dev/null +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java @@ -0,0 +1,159 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.computation; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isA; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.common.metrics.Metrics; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; + +@RunWith(MockitoJUnitRunner.class) +public class CommandRunnerMetricsTest { + + private static final MetricName METRIC_NAME_1 = + new MetricName("bob", "g1", "d1", ImmutableMap.of()); + private static final MetricName METRIC_NAME_2 = + new MetricName("jill", "g1", "d2", ImmutableMap.of()); + private static final String KSQL_SERVICE_ID = "kcql-1-"; + + @Mock + private Metrics metrics; + @Mock + private CommandRunner commandRunner; + @Captor + private ArgumentCaptor> gaugeCaptor; + + private CommandRunnerMetrics commandRunnerMetrics; + + @Before + public void setUp() { + when(metrics.metricName(any(), any(), any(), anyMap())) + .thenReturn(METRIC_NAME_1) + .thenReturn(METRIC_NAME_2); + when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.RUNNING); + when(commandRunner.getCommandRunnerDegradedReason()).thenReturn(CommandRunner.CommandRunnerDegradedReason.NONE); + + commandRunnerMetrics = new CommandRunnerMetrics(metrics, commandRunner, KSQL_SERVICE_ID, "rest"); + } + + @Test + public void shouldAddMetricOnCreation() { + // When: + // Listener created in setup + + // Then: + verify(metrics).metricName("status", "_confluent-ksql-kcql-1-rest-command-runner", + "The status of the commandRunner thread as it processes the command topic.", + Collections.emptyMap()); + verify(metrics).metricName("degraded-reason", "_confluent-ksql-kcql-1-rest-command-runner", + "The reason for why the commandRunner thread is in a DEGRADED state.", + Collections.emptyMap()); + + verify(metrics).addMetric(eq(METRIC_NAME_1), isA(Gauge.class)); + verify(metrics).addMetric(eq(METRIC_NAME_2), isA(Gauge.class)); + } + + @Test + public void shouldInitiallyBeCommandRunnerStatusRunningState() { + // When: + // CommandRunnerStatusMetric created in setup + + // Then: + assertThat(commandRunnerStatusGaugeValue(), is(CommandRunner.CommandRunnerStatus.RUNNING.name())); + } + + @Test + public void shouldUpdateToCommandRunnerStatusErrorState() { + // When: + when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.ERROR); + + // Then: + assertThat(commandRunnerStatusGaugeValue(), is(CommandRunner.CommandRunnerStatus.ERROR.name())); + } + + @Test + public void shouldUpdateToCommandRunnerStatusDegradedState() { + // When: + when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.DEGRADED); + + // Then: + assertThat(commandRunnerStatusGaugeValue(), is(CommandRunner.CommandRunnerStatus.DEGRADED.name())); + } + + @Test + public void shouldInitiallyNoneCommandRunnerDegradedReason() { + // When: + // CommandRunnerStatusMetric created in setup + + // Then: + assertThat(commandRunnerDegradedReasonGaugeValue(), is(CommandRunner.CommandRunnerDegradedReason.NONE.name())); + } + + @Test + public void shouldUpdateToCorruptedCommandRunnerDegradedReason() { + // When: + when(commandRunner.getCommandRunnerDegradedReason()).thenReturn(CommandRunner.CommandRunnerDegradedReason.CORRUPTED); + + // Then: + assertThat(commandRunnerDegradedReasonGaugeValue(), is(CommandRunner.CommandRunnerDegradedReason.CORRUPTED.name())); + } + + @Test + public void shouldUpdateToIncompatibleCommandsCommandRunnerDegradedReason() { + // When: + when(commandRunner.getCommandRunnerDegradedReason()).thenReturn(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND); + + // Then: + assertThat(commandRunnerDegradedReasonGaugeValue(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND.name())); + } + + @Test + public void shouldRemoveNoneCommandRunnerDegradedReason() { + // When: + commandRunnerMetrics.close(); + + // Then: + verify(metrics).removeMetric(METRIC_NAME_1); + verify(metrics).removeMetric(METRIC_NAME_2); + } + + private String commandRunnerStatusGaugeValue() { + verify(metrics).addMetric(eq(METRIC_NAME_1), gaugeCaptor.capture()); + return gaugeCaptor.getValue().value(null, 0L); + } + + private String commandRunnerDegradedReasonGaugeValue() { + verify(metrics).addMetric(eq(METRIC_NAME_2), gaugeCaptor.capture()); + return gaugeCaptor.getValue().value(null, 0L); + } +} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java deleted file mode 100644 index e4ed583c6499..000000000000 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Copyright 2019 Confluent Inc. - * - * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.ksql.rest.server.computation; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyMap; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.isA; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import com.google.common.collect.ImmutableMap; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.metrics.Gauge; -import org.apache.kafka.common.metrics.Metrics; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -import java.util.Collections; - -@RunWith(MockitoJUnitRunner.class) -public class CommandRunnerStatusMetricTest { - - private static final MetricName METRIC_NAME = - new MetricName("bob", "g1", "d1", ImmutableMap.of()); - private static final String KSQL_SERVICE_ID = "kcql-1-"; - - @Mock - private Metrics metrics; - @Mock - private CommandRunner commandRunner; - @Captor - private ArgumentCaptor> gaugeCaptor; - - private CommandRunnerStatusMetric commandRunnerStatusMetric; - - @Before - public void setUp() { - when(metrics.metricName(any(), any(), any(), anyMap())).thenReturn(METRIC_NAME); - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.RUNNING); - - commandRunnerStatusMetric = new CommandRunnerStatusMetric(metrics, commandRunner, KSQL_SERVICE_ID, "rest"); - } - - @Test - public void shouldAddMetricOnCreation() { - // When: - // Listener created in setup - - // Then: - verify(metrics).metricName("status", "_confluent-ksql-kcql-1-rest-command-runner", - "The status of the commandRunner thread as it processes the command topic.", - Collections.emptyMap()); - - verify(metrics).addMetric(eq(METRIC_NAME), isA(Gauge.class)); - } - - @Test - public void shouldInitiallyBeRunningState() { - // When: - // CommandRunnerStatusMetric created in setup - - // Then: - assertThat(currentGaugeValue(), is(CommandRunner.CommandRunnerStatus.RUNNING.name())); - } - - @Test - public void shouldUpdateToErrorState() { - // When: - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.ERROR); - - // Then: - assertThat(currentGaugeValue(), is(CommandRunner.CommandRunnerStatus.ERROR.name())); - } - - @Test - public void shouldUpdateToDegradedState() { - // When: - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.DEGRADED); - - // Then: - assertThat(currentGaugeValue(), is(CommandRunner.CommandRunnerStatus.DEGRADED.name())); - } - - @Test - public void shouldRemoveMetricOnClose() { - // When: - commandRunnerStatusMetric.close(); - - // Then: - verify(metrics).removeMetric(METRIC_NAME); - } - - private String currentGaugeValue() { - verify(metrics).addMetric(any(), gaugeCaptor.capture()); - return gaugeCaptor.getValue().value(null, 0L); - } -} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index 921d6f41a28c..1d5e4012e963 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -37,6 +37,7 @@ import com.google.common.collect.ImmutableList; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.metrics.MetricCollectors; +import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -47,16 +48,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Function; -import java.util.stream.Collectors; +import java.util.function.Supplier; -import io.confluent.ksql.util.Pair; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.junit.Before; @@ -72,6 +70,8 @@ @RunWith(MockitoJUnitRunner.class) public class CommandRunnerTest { private static final long COMMAND_RUNNER_HEALTH_TIMEOUT = 1000; + private static final String BACKUP_CORRUPTED_ERROR_MESSAGE = "corrupted"; + private static final String INCOMPATIBLE_COMMANDS_ERROR_MESSAGE = "incompatible"; @Mock private InteractiveStatementExecutor statementExecutor; @@ -103,6 +103,10 @@ public class CommandRunnerTest { private Consumer incompatibleCommandChecker; @Mock private Deserializer commandDeserializer; + @Mock + private Supplier backupCorrupted; + @Mock + private Errors errorHandler; @Captor private ArgumentCaptor threadTaskCaptor; private CommandRunner commandRunner; @@ -123,7 +127,10 @@ public void setup() { doNothing().when(incompatibleCommandChecker).accept(queuedCommand2); doNothing().when(incompatibleCommandChecker).accept(queuedCommand3); + when(backupCorrupted.get()).thenReturn(false); when(compactor.apply(any())).thenAnswer(inv -> inv.getArgument(0)); + when(errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage()).thenReturn(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); + when(errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage()).thenReturn(BACKUP_CORRUPTED_ERROR_MESSAGE); givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); @@ -140,7 +147,9 @@ public void setup() { clock, compactor, incompatibleCommandChecker, - commandDeserializer + commandDeserializer, + backupCorrupted, + errorHandler ); } @@ -233,8 +242,9 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInRestor inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); verify(statementExecutor, never()).handleRestore(queuedCommand3); - } @Test @@ -251,10 +261,12 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInFetch( verify(statementExecutor, never()).handleRestore(queuedCommand2); verify(statementExecutor, never()).handleRestore(queuedCommand3); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); } @Test - public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInRestore() { + public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInRestore() { // Given: givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); doThrow(new IncomaptibleKsqlCommandVersionException("")).when(incompatibleCommandChecker).accept(queuedCommand3); @@ -268,11 +280,13 @@ public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInRestore() { inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); verify(statementExecutor, never()).handleRestore(queuedCommand3); } @Test - public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInFetch() { + public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInFetch() { // Given: givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); doThrow(new IncomaptibleKsqlCommandVersionException("")).when(incompatibleCommandChecker).accept(queuedCommand3); @@ -286,9 +300,32 @@ public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInFetch() { inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); verify(statementExecutor, never()).handleRestore(queuedCommand3); } + @Test + public void shouldNotProcessCommandTopicIfBackupCorrupted() throws InterruptedException { + // Given: + when(backupCorrupted.get()).thenReturn(true); + + // When: + commandRunner.start(); + verify(commandStore, never()).close(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); + + // Then: + final InOrder inOrder = inOrder(executor, commandStore); + inOrder.verify(commandStore).wakeup(); + inOrder.verify(executor).awaitTermination(anyLong(), any()); + inOrder.verify(commandStore).close(); + assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(BACKUP_CORRUPTED_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.CORRUPTED)); + } + @Test public void shouldPullAndRunStatements() { // Given: diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index ee5b6b05efb7..b61873123b83 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -35,20 +35,17 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.server.CommandTopic; -import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.util.KsqlException; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import io.confluent.ksql.util.Pair; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.producer.Producer; @@ -91,8 +88,6 @@ public class CommandStoreTest { private Serializer commandSerializer; @Mock private Deserializer commandIdDeserializer; - @Mock - private CommandTopicBackup commandTopicBackup; private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); @@ -153,8 +148,7 @@ public void setUp() { TIMEOUT, commandIdSerializer, commandSerializer, - commandIdDeserializer, - commandTopicBackup + commandIdDeserializer ); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java index dc8ecff07fff..f409e745fab8 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java @@ -236,7 +236,7 @@ public void shouldReturnCommandStatus() { @Test public void shouldNotInitTransactionWhenCommandRunnerWarningPresent() { // When: - when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE); + when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); // Then: assertThrows( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index 4f1351bc42a7..db9a8e6e716b 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.not; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -44,6 +45,8 @@ import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.KsqlRequest; +import io.confluent.ksql.rest.server.CommandTopicBackup; +import io.confluent.ksql.rest.server.CommandTopicBackupNoOp; import io.confluent.ksql.rest.server.resources.KsqlResource; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -96,6 +99,11 @@ public class RecoveryTest { @Mock private DenyListPropertyValidator denyListPropertyValidator = mock(DenyListPropertyValidator.class); + + @Mock + private CommandTopicBackup commandTopicBackup = mock(CommandTopicBackup.class); + @Mock + private Errors errorHandler = mock(Errors.class); private final KsqlServer server1 = new KsqlServer(commands); private final KsqlServer server2 = new KsqlServer(commands); @@ -104,6 +112,7 @@ public class RecoveryTest { @Before public void setup() { securityContext = new KsqlSecurityContext(Optional.empty(), serviceContext); + when(commandTopicBackup.commandTopicCorruption()).thenReturn(false); } @After @@ -176,11 +185,6 @@ public boolean isEmpty() { return commandLog.isEmpty(); } - @Override - public boolean isCorrupted() { - return false; - } - @Override public void wakeup() { } @@ -225,7 +229,9 @@ private class KsqlServer { "ksql-service-id", Duration.ofMillis(2000), "", - InternalTopicSerdes.deserializer(Command.class) + InternalTopicSerdes.deserializer(Command.class), + commandTopicBackup, + errorHandler ); this.ksqlResource = new KsqlResource( @@ -234,7 +240,7 @@ private class KsqlServer { Duration.ofMillis(0), ()->{}, Optional.of((sc, metastore, statement) -> { }), - mock(Errors.class), + errorHandler, denyListPropertyValidator ); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 05396dabc21f..36c24a147be8 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -346,6 +346,7 @@ public void setUp() throws IOException, RestClientException { securityContext = new KsqlSecurityContext(Optional.empty(), serviceContext); when(commandRunner.getCommandQueue()).thenReturn(commandStore); + when(commandRunnerWarning.get()).thenReturn(""); when(commandStore.createTransactionalProducer()) .thenReturn(transactionalProducer); @@ -591,18 +592,15 @@ public void shouldHaveKsqlWarningIfCommandRunnerDegraded() { schema); // When: - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.DEGRADED); - when(errorsHandler.commandRunnerDegradedErrorMessage()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE); - final SourceDescriptionList descriptionList1 = makeSingleRequest( "SHOW STREAMS EXTENDED;", SourceDescriptionList.class); - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.RUNNING); + when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); final SourceDescriptionList descriptionList2 = makeSingleRequest( "SHOW STREAMS EXTENDED;", SourceDescriptionList.class); - assertThat(descriptionList1.getWarnings().size(), is(1)); - assertThat(descriptionList1.getWarnings().get(0).getMessage(), is(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE)); - assertThat(descriptionList2.getWarnings().size(), is(0)); + assertThat(descriptionList1.getWarnings().size(), is(0)); + assertThat(descriptionList2.getWarnings().size(), is(1)); + assertThat(descriptionList2.getWarnings().get(0).getMessage(), is(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); } @Test diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index 90119495bdc8..e91b24805f8d 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -19,19 +19,20 @@ public class DefaultErrorMessages implements ErrorMessages { - public static final String COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE = + public static final String COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE = "The server has encountered an incompatible entry in its log " + "and cannot process further DDL statements." + System.lineSeparator() + "This is most likely due to the service being rolled back to an earlier version."; - public static final String COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE = + public static final String COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE = "The server has detected that the command topic may be corrupted. The backup of the " + "command topic does not match the current contents of command topic." + System.lineSeparator() + "DDL statements will not be processed until either:" + System.lineSeparator() - + "1. The current command topic is deleted and the backup file is used to restore the command topic." + + "1. The current command topic is deleted and the backup file is used " + + "to restore the command topic." + System.lineSeparator() + "2. The current backup file is deleted." + System.lineSeparator() @@ -64,12 +65,12 @@ public String schemaRegistryUnconfiguredErrorMessage(final Exception e) { } @Override - public String commandRunnerDegradedErrorMessage() { - return COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE; + public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { + return COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE; } @Override - public String metaStoreCorruptedErrorMessage() { - return COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE; + public String commandRunnerDegradedBackupCorruptedErrorMessage() { + return COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE; } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index 98573f36b67c..56d912e45b33 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -23,7 +23,7 @@ public interface ErrorMessages { String schemaRegistryUnconfiguredErrorMessage(Exception e); - String commandRunnerDegradedErrorMessage(); + String commandRunnerDegradedIncompatibleCommandsErrorMessage(); - String metaStoreCorruptedErrorMessage(); + String commandRunnerDegradedBackupCorruptedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index f3d62ec35fc3..85f5407adff2 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -226,12 +226,12 @@ public String transactionInitTimeoutErrorMessage(final Exception e) { return errorMessages.transactionInitTimeoutErrorMessage(e); } - public String commandRunnerDegradedErrorMessage() { - return errorMessages.commandRunnerDegradedErrorMessage(); + public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { + return errorMessages.commandRunnerDegradedIncompatibleCommandsErrorMessage(); } - public String commandRunnerCorruptedErrorMessage() { - return errorMessages.metaStoreCorruptedErrorMessage(); + public String commandRunnerDegradedBackupCorruptedErrorMessage() { + return errorMessages.commandRunnerDegradedBackupCorruptedErrorMessage(); } public EndpointResponse generateResponse( From 085317d3dff1f386e8f1260716d0a2e6abd343b2 Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Sun, 6 Sep 2020 19:56:16 -0700 Subject: [PATCH 3/9] test commit --- .../io/confluent/ksql/util/KsqlConfig.java | 7 ++++ .../ksql/rest/server/CommandTopicBackup.java | 4 ++ .../rest/server/CommandTopicBackupImpl.java | 42 ++++++++----------- .../rest/server/CommandTopicBackupNoOp.java | 10 +++++ .../ksql/rest/server/KsqlRestApplication.java | 26 ++++++++++-- .../rest/server/computation/CommandQueue.java | 5 +++ .../server/computation/CommandRunner.java | 7 +++- .../rest/server/computation/CommandStore.java | 30 ++++++------- .../computation/DistributingExecutor.java | 13 +++--- .../rest/server/resources/KsqlResource.java | 36 +++++++++++----- .../server/computation/CommandStoreTest.java | 6 ++- .../computation/DistributingExecutorTest.java | 13 +++--- .../rest/server/computation/RecoveryTest.java | 5 +++ .../server/resources/KsqlResourceTest.java | 15 +++++-- .../ksql/rest/DefaultErrorMessages.java | 19 +++++++++ .../io/confluent/ksql/rest/ErrorMessages.java | 2 + .../java/io/confluent/ksql/rest/Errors.java | 4 ++ 17 files changed, 172 insertions(+), 72 deletions(-) diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 39f2110732bb..cde3d62f1a9b 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -303,6 +303,13 @@ public class KsqlConfig extends AbstractConfig { public static final String KSQL_METASTORE_BACKUP_LOCATION_DOC = "Specify the directory where " + "KSQL metastore backup files are located."; + public static final String KSQL_METASTORE_BACKUP_HARD_FAIL = "ksql.metastore.backup.hard.fail"; + public static final Boolean KSQL_METASTORE_BACKUP_HARD_FAIL_DEFAULT = true; + public static final String KSQL_METASTORE_BACKUP_HARD_FAIL_DOC = + "Enable hard failing when the metastore backup and command topic are inconsistent. " + + "The server will enter a degraded state if the command topic isn't present but the metastore " + + "back up is present, or the command topic and the metastore have conflicting commands"; + public static final String KSQL_SUPPRESS_ENABLED = "ksql.suppress.enabled"; public static final Boolean KSQL_SUPPRESS_ENABLED_DEFAULT = false; public static final String KSQL_SUPPRESS_ENABLED_DOC = diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java index d12b23bb5d1b..87c3e3ecbc6f 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java @@ -22,5 +22,9 @@ public interface CommandTopicBackup { void writeRecord(ConsumerRecord record); + boolean commandTopicCorruption(); + + boolean backupExists(); + void close(); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java index b0fe42838c6c..6147777ccd6d 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java @@ -59,6 +59,7 @@ public long read() { private BackupReplayFile replayFile; private List> latestReplay; private int latestReplayIdx; + private boolean corruptionDetected; public CommandTopicBackupImpl(final String location, final String topicName) { this(location, topicName, CURRENT_MILLIS_TICKER); @@ -92,6 +93,7 @@ public void initialize() { } latestReplayIdx = 0; + corruptionDetected = false; LOG.info("Command topic will be backup on file: {}", replayFile.getPath()); } @@ -147,16 +149,20 @@ public void writeRecord(final ConsumerRecord record) { } void writeCommandToBackup(final ConsumerRecord record) { +// if (corruptionDetected) { +// LOG.warn("Failure to write command topic data to backup. Corruption detected in command topic."); +// } + if (isRestoring()) { if (isRecordInLatestReplay(record)) { // Ignore backup because record was already replayed return; } else { - LOG.info("Previous command topic backup does not match the new command topic data. " - + "A new backup file will be created."); - createNewBackupFile(); - latestReplay.clear(); - LOG.info("New backup file created: {}", replayFile.getPath()); + LOG.info("Previous command topic backup does not match the new command topic data."); + corruptionDetected = true; +// createNewBackupFile(); +// latestReplay.clear(); +// LOG.info("New backup file created: {}", replayFile.getPath()); } } else if (latestReplay.size() > 0) { // clear latest replay from memory @@ -173,26 +179,14 @@ void writeCommandToBackup(final ConsumerRecord record) { } } - private void createNewBackupFile() { - try { - replayFile.close(); - } catch (IOException e) { - LOG.warn("Couldn't close the current backup file {}. Error = {}", - replayFile.getPath(), e.getMessage()); - } - - replayFile = newReplayFile(); + @Override + public boolean commandTopicCorruption() { + return corruptionDetected; + } - if (latestReplay.size() > 0 && latestReplayIdx > 0) { - try { - replayFile.write(latestReplay.subList(0, latestReplayIdx)); - } catch (final IOException e) { - LOG.warn("Couldn't write the latest replayed commands to the new backup file {}. " - + "Make sure the file exists and has permissions to write. " - + "KSQL must be restarted afterwards to complete the backup process. Error = {}", - replayFile.getPath(), e.getMessage()); - } - } + @Override + public boolean backupExists() { + return latestReplayFile().isPresent(); } @VisibleForTesting diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java index efaafdc46629..b7465a16bcfe 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java @@ -32,4 +32,14 @@ public void writeRecord(final ConsumerRecord record) { public void close() { // no-op } + + @Override + public boolean commandTopicCorruption() { + return false; + } + + @Override + public boolean backupExists() { + return false; + } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index 2ca44b72ee3c..f8cdc9927139 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -188,6 +188,7 @@ public final class KsqlRestApplication implements Executable { private final CompletableFuture terminatedFuture = new CompletableFuture<>(); private final QueryMonitor queryMonitor; private final DenyListPropertyValidator denyListPropertyValidator; + private final CommandTopicBackup commandTopicBackup; // The startup thread that can be interrupted if necessary during shutdown. This should only // happen if startup hangs. @@ -224,7 +225,8 @@ public static SourceName getCommandsStreamName() { final Optional lagReportingAgent, final Vertx vertx, final QueryMonitor ksqlQueryMonitor, - final DenyListPropertyValidator denyListPropertyValidator + final DenyListPropertyValidator denyListPropertyValidator, + final CommandTopicBackup commandTopicBackup ) { log.debug("Creating instance of ksqlDB API server"); this.serviceContext = requireNonNull(serviceContext, "serviceContext"); @@ -253,6 +255,8 @@ public static SourceName getCommandsStreamName() { this.vertx = requireNonNull(vertx, "vertx"); this.denyListPropertyValidator = requireNonNull(denyListPropertyValidator, "denyListPropertyValidator"); + this.commandTopicBackup = + requireNonNull(commandTopicBackup, "commandTopicBackup"); this.serverInfoResource = new ServerInfoResource(serviceContext, ksqlConfigNoPort, commandRunner); @@ -656,6 +660,20 @@ static KsqlRestApplication buildApplication( final String commandTopicName = ReservedInternalTopics.commandTopic(ksqlConfig); + CommandTopicBackup commandTopicBackup = new CommandTopicBackupNoOp(); + if (ksqlConfig.getBoolean(KsqlConfig.KSQL_ENABLE_METASTORE_BACKUP)) { + if (ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION).isEmpty()) { + throw new KsqlException(String.format("Metastore backups is enabled, but location " + + "is empty. Please specify the location with the property '%s'", + KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION)); + } + + commandTopicBackup = new CommandTopicBackupImpl( + ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), + commandTopicName) + ; + } + final CommandStore commandStore = CommandStore.Factory.create( ksqlConfig, commandTopicName, @@ -663,7 +681,8 @@ static KsqlRestApplication buildApplication( ksqlConfig.addConfluentMetricsContextConfigsKafka( restConfig.getCommandConsumerProperties()), ksqlConfig.addConfluentMetricsContextConfigsKafka( - restConfig.getCommandProducerProperties()) + restConfig.getCommandProducerProperties()), + commandTopicBackup ); final InteractiveStatementExecutor statementExecutor = @@ -791,7 +810,8 @@ static KsqlRestApplication buildApplication( lagReportingAgent, vertx, queryMonitor, - denyListPropertyValidator + denyListPropertyValidator, + commandTopicBackup ); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java index e97e7795e27f..46d09dd77660 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java @@ -99,6 +99,11 @@ void ensureConsumedPast(long seqNum, Duration timeout) */ boolean isEmpty(); + /** + * @return whether or not data corruption is detected in the enqueued comamnds. + */ + boolean isCorrupted(); + /** * Cause any blocked {@link #getNewCommands(Duration)} calls to return early. * diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index c0fc3580b682..4a95c05f2cd3 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -82,7 +82,8 @@ public class CommandRunner implements Closeable { public enum CommandRunnerStatus { RUNNING, ERROR, - DEGRADED + DEGRADED, + CORRUPTED } public CommandRunner( @@ -311,6 +312,10 @@ private void terminateCluster(final Command command) { } public CommandRunnerStatus checkCommandRunnerStatus() { + if (commandStore.isCorrupted()) { + return CommandRunnerStatus.CORRUPTED; + } + if (incompatibleCommandDetected) { return CommandRunnerStatus.DEGRADED; } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 44b0d3b38bda..786172f5d819 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -76,6 +76,7 @@ public class CommandStore implements CommandQueue, Closeable { private final Serializer commandIdSerializer; private final Serializer commandSerializer; private final Deserializer commandIdDeserializer; + private final CommandTopicBackup commandTopicBackup; public static final class Factory { @@ -88,7 +89,8 @@ public static CommandStore create( final String commandTopicName, final Duration commandQueueCatchupTimeout, final Map kafkaConsumerProperties, - final Map kafkaProducerProperties + final Map kafkaProducerProperties, + final CommandTopicBackup commandTopicBackup ) { kafkaConsumerProperties.put( ConsumerConfig.ISOLATION_LEVEL_CONFIG, @@ -107,20 +109,6 @@ public static CommandStore create( "all" ); - CommandTopicBackup commandTopicBackup = new CommandTopicBackupNoOp(); - if (ksqlConfig.getBoolean(KsqlConfig.KSQL_ENABLE_METASTORE_BACKUP)) { - if (ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION).isEmpty()) { - throw new KsqlException(String.format("Metastore backups is enabled, but location " - + "is empty. Please specify the location with the property '%s'", - KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION)); - } - - commandTopicBackup = new CommandTopicBackupImpl( - ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), - commandTopicName) - ; - } - return new CommandStore( commandTopicName, new CommandTopic( @@ -134,7 +122,8 @@ public static CommandStore create( commandQueueCatchupTimeout, InternalTopicSerdes.serializer(), InternalTopicSerdes.serializer(), - InternalTopicSerdes.deserializer(CommandId.class) + InternalTopicSerdes.deserializer(CommandId.class), + commandTopicBackup ); } } @@ -148,7 +137,8 @@ public static CommandStore create( final Duration commandQueueCatchupTimeout, final Serializer commandIdSerializer, final Serializer commandSerializer, - final Deserializer commandIdDeserializer + final Deserializer commandIdDeserializer, + final CommandTopicBackup commandTopicBackup ) { this.commandTopic = Objects.requireNonNull(commandTopic, "commandTopic"); this.commandStatusMap = Maps.newConcurrentMap(); @@ -167,6 +157,7 @@ public static CommandStore create( Objects.requireNonNull(commandSerializer, "commandSerializer"); this.commandIdDeserializer = Objects.requireNonNull(commandIdDeserializer, "commandIdDeserializer"); + this.commandTopicBackup = Objects.requireNonNull(commandTopicBackup, "commandTopicBackup"); } @Override @@ -338,6 +329,11 @@ private long getCommandTopicOffset() { } } + @Override + public boolean isCorrupted() { + return commandTopicBackup.commandTopicCorruption(); + } + @Override public boolean isEmpty() { return commandTopic.getEndOffset() == 0; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java index 128602811a21..d4a946b3d21e 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/DistributingExecutor.java @@ -60,7 +60,7 @@ public class DistributingExecutor { private final CommandIdAssigner commandIdAssigner; private final ReservedInternalTopics internalTopics; private final Errors errorHandler; - private final Supplier commandRunnerDegraded; + private final Supplier commandRunnerWarning; public DistributingExecutor( final KsqlConfig ksqlConfig, @@ -70,7 +70,7 @@ public DistributingExecutor( final Optional authorizationValidator, final ValidatedCommandFactory validatedCommandFactory, final Errors errorHandler, - final Supplier commandRunnerDegraded + final Supplier commandRunnerWarning ) { this.commandQueue = commandQueue; this.distributedCmdResponseTimeout = @@ -86,8 +86,8 @@ public DistributingExecutor( this.internalTopics = new ReservedInternalTopics(Objects.requireNonNull(ksqlConfig, "ksqlConfig")); this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); - this.commandRunnerDegraded = - Objects.requireNonNull(commandRunnerDegraded, "commandRunnerDegraded"); + this.commandRunnerWarning = + Objects.requireNonNull(commandRunnerWarning, "commandRunnerWarning"); } /** @@ -105,10 +105,11 @@ public Optional execute( final KsqlExecutionContext executionContext, final KsqlSecurityContext securityContext ) { - if (commandRunnerDegraded.get()) { + final String commandRunnerWarningString = commandRunnerWarning.get(); + if (!commandRunnerWarningString.equals("")) { throw new KsqlServerException("Failed to handle Ksql Statement." + System.lineSeparator() - + errorHandler.commandRunnerDegradedErrorMessage()); + + commandRunnerWarningString); } final ConfiguredStatement injected = injectorFactory .apply(executionContext, securityContext.getServiceContext()) diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 421240cd1439..2b2c1d79f9a2 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -103,6 +103,7 @@ public class KsqlResource implements KsqlConfigurable { private final BiFunction injectorFactory; private final Optional authorizationValidator; private final DenyListPropertyValidator denyListPropertyValidator; + private final Supplier commandRunnerWarning; private RequestValidator validator; private RequestHandler handler; private final Errors errorHandler; @@ -126,7 +127,18 @@ public KsqlResource( Injectors.DEFAULT, authorizationValidator, errorHandler, - denyListPropertyValidator + denyListPropertyValidator, + () -> { + if (commandRunner.checkCommandRunnerStatus() + == CommandRunner.CommandRunnerStatus.DEGRADED) { + return errorHandler.commandRunnerDegradedErrorMessage(); + } + if (commandRunner.checkCommandRunnerStatus() + == CommandRunner.CommandRunnerStatus.CORRUPTED) { + return errorHandler.commandRunnerCorruptedErrorMessage(); + } + return ""; + } ); } @@ -138,7 +150,8 @@ public KsqlResource( final BiFunction injectorFactory, final Optional authorizationValidator, final Errors errorHandler, - final DenyListPropertyValidator denyListPropertyValidator + final DenyListPropertyValidator denyListPropertyValidator, + final Supplier commandRunnerWarning ) { this.ksqlEngine = Objects.requireNonNull(ksqlEngine, "ksqlEngine"); this.commandRunner = Objects.requireNonNull(commandRunner, "commandRunner"); @@ -152,6 +165,8 @@ public KsqlResource( this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); this.denyListPropertyValidator = Objects.requireNonNull(denyListPropertyValidator, "denyListPropertyValidator"); + this.commandRunnerWarning = + Objects.requireNonNull(commandRunnerWarning, "commandRunnerWarning"); } @Override @@ -190,8 +205,7 @@ public void configure(final KsqlConfig config) { authorizationValidator, new ValidatedCommandFactory(), errorHandler, - () -> commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.DEGRADED + commandRunnerWarning ), ksqlEngine, config, @@ -280,11 +294,10 @@ public EndpointResponse handleKsqlStatements( ); LOG.info("Processed successfully: " + request); - addCommandRunnerDegradedWarning( + addCommandRunnerWarning( entities, errorHandler, - () -> commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.DEGRADED); + commandRunnerWarning); return EndpointResponse.ok(entities); } catch (final KsqlRestException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); @@ -325,15 +338,16 @@ private static void ensureValidPatterns(final List deleteTopicList) { }); } - private static void addCommandRunnerDegradedWarning( + private static void addCommandRunnerWarning( final KsqlEntityList entityList, final Errors errorHandler, - final Supplier commandRunnerDegraded + final Supplier commandRunnerIssue ) { - if (commandRunnerDegraded.get()) { + final String commandRunnerIssueString = commandRunnerIssue.get(); + if (!commandRunnerIssueString.equals("")) { for (final KsqlEntity entity: entityList) { entity.updateWarnings(Collections.singletonList( - new KsqlWarning(errorHandler.commandRunnerDegradedErrorMessage()))); + new KsqlWarning(commandRunnerIssueString))); } } } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index 2f534db14572..ee5b6b05efb7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -35,6 +35,7 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.server.CommandTopic; +import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.util.KsqlException; import java.time.Duration; import java.util.ArrayList; @@ -90,6 +91,8 @@ public class CommandStoreTest { private Serializer commandSerializer; @Mock private Deserializer commandIdDeserializer; + @Mock + private CommandTopicBackup commandTopicBackup; private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); @@ -150,7 +153,8 @@ public void setUp() { TIMEOUT, commandIdSerializer, commandSerializer, - commandIdDeserializer + commandIdDeserializer, + commandTopicBackup ); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java index 1b7b13fff433..dc8ecff07fff 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -46,6 +47,7 @@ import io.confluent.ksql.parser.tree.Statement; import io.confluent.ksql.parser.tree.TableElements; import io.confluent.ksql.properties.with.CommonCreateConfigs; +import io.confluent.ksql.rest.DefaultErrorMessages; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandId.Action; @@ -133,7 +135,7 @@ CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("json") @Mock private Errors errorHandler; @Mock - private Supplier commandRunnerDegraded; + private Supplier commandRunnerWarning; private DistributingExecutor distributor; private AtomicLong scnCounter; @@ -150,7 +152,7 @@ public void setUp() throws InterruptedException { when(status.getCommandSequenceNumber()).thenAnswer(inv -> scnCounter.incrementAndGet()); when(executionContext.getMetaStore()).thenReturn(metaStore); when(executionContext.createSandbox(any())).thenReturn(sandboxContext); - when(commandRunnerDegraded.get()).thenReturn(false); + when(commandRunnerWarning.get()).thenReturn(""); serviceContext = SandboxedServiceContext.create(TestServiceContext.create()); when(executionContext.getServiceContext()).thenReturn(serviceContext); when(validatedCommandFactory.create(any(), any())).thenReturn(command); @@ -166,7 +168,7 @@ public void setUp() throws InterruptedException { Optional.of(authorizationValidator), validatedCommandFactory, errorHandler, - commandRunnerDegraded + commandRunnerWarning ); } @@ -232,15 +234,16 @@ public void shouldReturnCommandStatus() { } @Test - public void shouldNotInitTransactionWhenCommandRunnerDegraded() { + public void shouldNotInitTransactionWhenCommandRunnerWarningPresent() { // When: - when(commandRunnerDegraded.get()).thenReturn(true); + when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE); // Then: assertThrows( KsqlServerException.class, () -> distributor.execute(CONFIGURED_STATEMENT, executionContext, securityContext) ); + verify(transactionalProducer, never()).initTransactions(); } @Test diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index d5f3bf22246a..4f1351bc42a7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -176,6 +176,11 @@ public boolean isEmpty() { return commandLog.isEmpty(); } + @Override + public boolean isCorrupted() { + return false; + } + @Override public void wakeup() { } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 8447478a2bb8..05396dabc21f 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -179,6 +179,7 @@ import java.util.Properties; import java.util.concurrent.TimeoutException; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.avro.Schema.Type; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -301,6 +302,8 @@ public class KsqlResourceTest { private Errors errorsHandler; @Mock private DenyListPropertyValidator denyListPropertyValidator; + @Mock + private Supplier commandRunnerWarning; private KsqlResource ksqlResource; private SchemaRegistryClient schemaRegistryClient; @@ -412,7 +415,8 @@ public void shouldThrowOnHandleStatementIfNotConfigured() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); // When: @@ -443,7 +447,8 @@ public void shouldThrowOnHandleTerminateIfNotConfigured() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); // When: @@ -2221,7 +2226,8 @@ private void setUpKsqlResource() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); ksqlResource.configure(ksqlConfig); @@ -2265,7 +2271,8 @@ public void shouldThrowOnDenyListValidatorWhenHandleKsqlStatement() { new TopicDeleteInjector(ec, sc)), Optional.of(authorizationValidator), errorsHandler, - denyListPropertyValidator + denyListPropertyValidator, + commandRunnerWarning ); final Map props = new HashMap<>(ksqlRestConfig.getKsqlConfigProperties()); props.put(KsqlConfig.KSQL_PROPERTIES_OVERRIDES_DENYLIST, diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index c4e4211c3dec..90119495bdc8 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -25,6 +25,20 @@ public class DefaultErrorMessages implements ErrorMessages { + System.lineSeparator() + "This is most likely due to the service being rolled back to an earlier version."; + public static final String COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE = + "The server has detected that the command topic may be corrupted. The backup of the " + + "command topic does not match the current contents of command topic." + + System.lineSeparator() + + "DDL statements will not be processed until either:" + + System.lineSeparator() + + "1. The current command topic is deleted and the backup file is used to restore the command topic." + + System.lineSeparator() + + "2. The current backup file is deleted." + + System.lineSeparator() + + "The server must be restarted after performing either operation in order to resume " + + "normal functionality"; + + @Override public String kafkaAuthorizationErrorMessage(final Exception e) { return ErrorMessageUtil.buildErrorMessage(e); @@ -53,4 +67,9 @@ public String schemaRegistryUnconfiguredErrorMessage(final Exception e) { public String commandRunnerDegradedErrorMessage() { return COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE; } + + @Override + public String metaStoreCorruptedErrorMessage() { + return COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE; + } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index 80aba990d596..98573f36b67c 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -24,4 +24,6 @@ public interface ErrorMessages { String schemaRegistryUnconfiguredErrorMessage(Exception e); String commandRunnerDegradedErrorMessage(); + + String metaStoreCorruptedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 920259f975e1..f3d62ec35fc3 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -230,6 +230,10 @@ public String commandRunnerDegradedErrorMessage() { return errorMessages.commandRunnerDegradedErrorMessage(); } + public String commandRunnerCorruptedErrorMessage() { + return errorMessages.metaStoreCorruptedErrorMessage(); + } + public EndpointResponse generateResponse( final Exception e, final EndpointResponse defaultResponse From deb88e15c93f03161dccbd45bc424d17fbe05f0f Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Wed, 9 Sep 2020 15:54:26 -0700 Subject: [PATCH 4/9] refactor --- .../io/confluent/ksql/util/KsqlConfig.java | 7 - .../ksql/rest/server/CommandTopic.java | 15 +- .../ksql/rest/server/CommandTopicBackup.java | 2 - .../rest/server/CommandTopicBackupImpl.java | 21 +-- .../rest/server/CommandTopicBackupNoOp.java | 5 - .../ksql/rest/server/KsqlRestApplication.java | 20 +-- .../rest/server/computation/CommandQueue.java | 5 - .../server/computation/CommandRunner.java | 73 ++++++-- ...sMetric.java => CommandRunnerMetrics.java} | 26 ++- .../rest/server/computation/CommandStore.java | 15 +- .../rest/server/resources/KsqlResource.java | 22 +-- .../server/CommandTopicBackupImplTest.java | 34 ++-- .../ksql/rest/server/CommandTopicTest.java | 49 +++++- .../computation/CommandRunnerMetricsTest.java | 159 ++++++++++++++++++ .../CommandRunnerStatusMetricTest.java | 117 ------------- .../server/computation/CommandRunnerTest.java | 53 +++++- .../server/computation/CommandStoreTest.java | 8 +- .../computation/DistributingExecutorTest.java | 2 +- .../rest/server/computation/RecoveryTest.java | 20 ++- .../server/resources/KsqlResourceTest.java | 12 +- .../ksql/rest/DefaultErrorMessages.java | 15 +- .../io/confluent/ksql/rest/ErrorMessages.java | 4 +- .../java/io/confluent/ksql/rest/Errors.java | 8 +- 23 files changed, 412 insertions(+), 280 deletions(-) rename ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/{CommandRunnerStatusMetric.java => CommandRunnerMetrics.java} (68%) create mode 100644 ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java delete mode 100644 ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index cde3d62f1a9b..39f2110732bb 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -303,13 +303,6 @@ public class KsqlConfig extends AbstractConfig { public static final String KSQL_METASTORE_BACKUP_LOCATION_DOC = "Specify the directory where " + "KSQL metastore backup files are located."; - public static final String KSQL_METASTORE_BACKUP_HARD_FAIL = "ksql.metastore.backup.hard.fail"; - public static final Boolean KSQL_METASTORE_BACKUP_HARD_FAIL_DEFAULT = true; - public static final String KSQL_METASTORE_BACKUP_HARD_FAIL_DOC = - "Enable hard failing when the metastore backup and command topic are inconsistent. " - + "The server will enter a degraded state if the command topic isn't present but the metastore " - + "back up is present, or the command topic and the metastore have conflicting commands"; - public static final String KSQL_SUPPRESS_ENABLED = "ksql.suppress.enabled"; public static final Boolean KSQL_SUPPRESS_ENABLED_DEFAULT = false; public static final String KSQL_SUPPRESS_ENABLED_DOC = diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java index 32348a60fc4a..8b7004d2a845 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java @@ -18,6 +18,7 @@ import com.google.common.collect.Lists; import io.confluent.ksql.rest.server.computation.QueuedCommand; import java.time.Duration; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -79,12 +80,18 @@ public void start() { public Iterable> getNewCommands(final Duration timeout) { final Iterable> iterable = commandConsumer.poll(timeout); + final List> records = new ArrayList<>(); if (iterable != null) { - iterable.forEach(this::backupRecord); + iterable.forEach(record -> { + backupRecord(record); + if (!commandTopicBackup.commandTopicCorruption()) { + records.add(record); + } + }); } - return iterable; + return records; } public List getRestoreCommands(final Duration duration) { @@ -100,6 +107,10 @@ public List getRestoreCommands(final Duration duration) { log.debug("Received {} records from poll", records.count()); for (final ConsumerRecord record : records) { backupRecord(record); + + if (commandTopicBackup.commandTopicCorruption()) { + continue; + } if (record.value() == null) { continue; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java index 87c3e3ecbc6f..06331f5046b6 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackup.java @@ -24,7 +24,5 @@ public interface CommandTopicBackup { boolean commandTopicCorruption(); - boolean backupExists(); - void close(); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java index 6147777ccd6d..6cc6cceb3cf6 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupImpl.java @@ -61,7 +61,9 @@ public long read() { private int latestReplayIdx; private boolean corruptionDetected; - public CommandTopicBackupImpl(final String location, final String topicName) { + public CommandTopicBackupImpl( + final String location, + final String topicName) { this(location, topicName, CURRENT_MILLIS_TICKER); } @@ -149,9 +151,11 @@ public void writeRecord(final ConsumerRecord record) { } void writeCommandToBackup(final ConsumerRecord record) { -// if (corruptionDetected) { -// LOG.warn("Failure to write command topic data to backup. Corruption detected in command topic."); -// } + if (corruptionDetected) { + LOG.warn("Failure to write command topic data to backup. " + + "Corruption detected in command topic."); + return; + } if (isRestoring()) { if (isRecordInLatestReplay(record)) { @@ -160,9 +164,7 @@ void writeCommandToBackup(final ConsumerRecord record) { } else { LOG.info("Previous command topic backup does not match the new command topic data."); corruptionDetected = true; -// createNewBackupFile(); -// latestReplay.clear(); -// LOG.info("New backup file created: {}", replayFile.getPath()); + return; } } else if (latestReplay.size() > 0) { // clear latest replay from memory @@ -184,11 +186,6 @@ public boolean commandTopicCorruption() { return corruptionDetected; } - @Override - public boolean backupExists() { - return latestReplayFile().isPresent(); - } - @VisibleForTesting BackupReplayFile openOrCreateReplayFile() { final Optional latestFile = latestReplayFile(); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java index b7465a16bcfe..1aca709cb400 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopicBackupNoOp.java @@ -37,9 +37,4 @@ public void close() { public boolean commandTopicCorruption() { return false; } - - @Override - public boolean backupExists() { - return false; - } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index f8cdc9927139..e1e5fc6c6b64 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -188,7 +188,6 @@ public final class KsqlRestApplication implements Executable { private final CompletableFuture terminatedFuture = new CompletableFuture<>(); private final QueryMonitor queryMonitor; private final DenyListPropertyValidator denyListPropertyValidator; - private final CommandTopicBackup commandTopicBackup; // The startup thread that can be interrupted if necessary during shutdown. This should only // happen if startup hangs. @@ -225,8 +224,7 @@ public static SourceName getCommandsStreamName() { final Optional lagReportingAgent, final Vertx vertx, final QueryMonitor ksqlQueryMonitor, - final DenyListPropertyValidator denyListPropertyValidator, - final CommandTopicBackup commandTopicBackup + final DenyListPropertyValidator denyListPropertyValidator ) { log.debug("Creating instance of ksqlDB API server"); this.serviceContext = requireNonNull(serviceContext, "serviceContext"); @@ -255,8 +253,6 @@ public static SourceName getCommandsStreamName() { this.vertx = requireNonNull(vertx, "vertx"); this.denyListPropertyValidator = requireNonNull(denyListPropertyValidator, "denyListPropertyValidator"); - this.commandTopicBackup = - requireNonNull(commandTopicBackup, "commandTopicBackup"); this.serverInfoResource = new ServerInfoResource(serviceContext, ksqlConfigNoPort, commandRunner); @@ -670,9 +666,10 @@ static KsqlRestApplication buildApplication( commandTopicBackup = new CommandTopicBackupImpl( ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), - commandTopicName) - ; + commandTopicName + ); } + final CommandTopicBackup finalCommandTopicBackup = commandTopicBackup; final CommandStore commandStore = CommandStore.Factory.create( ksqlConfig, @@ -682,7 +679,7 @@ static KsqlRestApplication buildApplication( restConfig.getCommandConsumerProperties()), ksqlConfig.addConfluentMetricsContextConfigsKafka( restConfig.getCommandProducerProperties()), - commandTopicBackup + finalCommandTopicBackup ); final InteractiveStatementExecutor statementExecutor = @@ -757,7 +754,9 @@ static KsqlRestApplication buildApplication( Duration.ofMillis(restConfig.getLong( KsqlRestConfig.KSQL_COMMAND_RUNNER_BLOCKED_THRESHHOLD_ERROR_MS)), metricsPrefix, - InternalTopicSerdes.deserializer(Command.class) + InternalTopicSerdes.deserializer(Command.class), + finalCommandTopicBackup, + errorHandler ); final KsqlResource ksqlResource = new KsqlResource( @@ -810,8 +809,7 @@ static KsqlRestApplication buildApplication( lagReportingAgent, vertx, queryMonitor, - denyListPropertyValidator, - commandTopicBackup + denyListPropertyValidator ); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java index 46d09dd77660..e97e7795e27f 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java @@ -99,11 +99,6 @@ void ensureConsumedPast(long seqNum, Duration timeout) */ boolean isEmpty(); - /** - * @return whether or not data corruption is detected in the enqueued comamnds. - */ - boolean isCorrupted(); - /** * Cause any blocked {@link #getNewCommands(Duration)} calls to return early. * diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 4a95c05f2cd3..24b7edd11172 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -16,7 +16,9 @@ package io.confluent.ksql.rest.server.computation; import com.google.common.annotations.VisibleForTesting; +import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.ClusterTerminateRequest; +import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -39,6 +41,8 @@ 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.common.errors.SerializationException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.serialization.Deserializer; @@ -69,7 +73,7 @@ public class CommandRunner implements Closeable { private final ClusterTerminator clusterTerminator; private final ServerState serverState; - private final CommandRunnerStatusMetric commandRunnerStatusMetric; + private final CommandRunnerMetrics commandRunnerMetric; private final AtomicReference> currentCommandRef; private final AtomicReference lastPollTime; private final Duration commandRunnerHealthTimeout; @@ -77,15 +81,23 @@ public class CommandRunner implements Closeable { private final Deserializer commandDeserializer; private final Consumer incompatibleCommandChecker; + private final Supplier backupCorrupted; + private final Errors errorHandler; private boolean incompatibleCommandDetected; public enum CommandRunnerStatus { RUNNING, ERROR, - DEGRADED, + DEGRADED + } + + public enum CommandRunnerDegradedReason { + NONE, + INCOMPATIBLE_COMMAND, CORRUPTED } + // CHECKSTYLE_RULES.OFF: ParameterNumberCheck public CommandRunner( final InteractiveStatementExecutor statementExecutor, final CommandQueue commandStore, @@ -95,7 +107,9 @@ public CommandRunner( final String ksqlServiceId, final Duration commandRunnerHealthTimeout, final String metricsGroupPrefix, - final Deserializer commandDeserializer + final Deserializer commandDeserializer, + final CommandTopicBackup commandTopicBackup, + final Errors errorHandler ) { this( statementExecutor, @@ -113,7 +127,9 @@ public CommandRunner( queuedCommand.getAndDeserializeCommandId(); queuedCommand.getAndDeserializeCommand(commandDeserializer); }, - commandDeserializer + commandDeserializer, + commandTopicBackup::commandTopicCorruption, + errorHandler ); } @@ -132,7 +148,9 @@ public CommandRunner( final Clock clock, final Function, List> compactor, final Consumer incompatibleCommandChecker, - final Deserializer commandDeserializer + final Deserializer commandDeserializer, + final Supplier backupCorrupted, + final Errors errorHandler ) { // CHECKSTYLE_RULES.ON: ParameterNumberCheck this.statementExecutor = Objects.requireNonNull(statementExecutor, "statementExecutor"); @@ -145,14 +163,18 @@ public CommandRunner( Objects.requireNonNull(commandRunnerHealthTimeout, "commandRunnerHealthTimeout"); this.currentCommandRef = new AtomicReference<>(null); this.lastPollTime = new AtomicReference<>(null); - this.commandRunnerStatusMetric = - new CommandRunnerStatusMetric(ksqlServiceId, this, metricsGroupPrefix); + this.commandRunnerMetric = + new CommandRunnerMetrics(ksqlServiceId, this, metricsGroupPrefix); this.clock = Objects.requireNonNull(clock, "clock"); this.compactor = Objects.requireNonNull(compactor, "compactor"); this.incompatibleCommandChecker = Objects.requireNonNull(incompatibleCommandChecker, "incompatibleCommandChecker"); this.commandDeserializer = Objects.requireNonNull(commandDeserializer, "commandDeserializer"); + this.backupCorrupted = + Objects.requireNonNull(backupCorrupted, "backupCorrupted"); + this.errorHandler = + Objects.requireNonNull(errorHandler, "errorHandler"); this.incompatibleCommandDetected = false; } @@ -173,7 +195,7 @@ public void close() { if (!closed) { closeEarly(); } - commandRunnerStatusMetric.close(); + commandRunnerMetric.close(); } /** @@ -312,11 +334,7 @@ private void terminateCluster(final Command command) { } public CommandRunnerStatus checkCommandRunnerStatus() { - if (commandStore.isCorrupted()) { - return CommandRunnerStatus.CORRUPTED; - } - - if (incompatibleCommandDetected) { + if (incompatibleCommandDetected || backupCorrupted.get()) { return CommandRunnerStatus.DEGRADED; } @@ -333,6 +351,30 @@ public CommandRunnerStatus checkCommandRunnerStatus() { ? CommandRunnerStatus.RUNNING : CommandRunnerStatus.ERROR; } + public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { + if (backupCorrupted.get()) { + return CommandRunnerDegradedReason.CORRUPTED; + } + + if (incompatibleCommandDetected) { + return CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND; + } + + return CommandRunnerDegradedReason.NONE; + } + + public String getCommandRunnerDegradedWarning() { + if (backupCorrupted.get()) { + return errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage(); + } + + if (incompatibleCommandDetected) { + return errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage(); + } + + return ""; + } + private List checkForIncompatibleCommands(final List commands) { final List compatibleCommands = new ArrayList<>(); try { @@ -357,8 +399,9 @@ private class Runner implements Runnable { public void run() { try { while (!closed) { - if (incompatibleCommandDetected) { - LOG.warn("CommandRunner entering degraded state after failing to deserialize command"); + if (incompatibleCommandDetected || backupCorrupted.get()) { + LOG.warn("CommandRunner entering degraded state due to: {}", + getCommandRunnerDegradedReason()); closeEarly(); } else { LOG.trace("Polling for new writes to command topic"); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetric.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java similarity index 68% rename from ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetric.java rename to ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java index 4f605300cc19..4dc1e62b5096 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetric.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java @@ -26,17 +26,18 @@ import org.apache.kafka.common.metrics.Metrics; /** - * Emits a JMX metric that indicates the health of the CommandRunner thread. + * Emits JMX metrics that for the CommandRunner thread. */ -public class CommandRunnerStatusMetric implements Closeable { +public class CommandRunnerMetrics implements Closeable { private static final String DEFAULT_METRIC_GROUP_PREFIX = "ksql-rest-app"; private static final String METRIC_GROUP_POST_FIX = "-command-runner"; private final Metrics metrics; - private final MetricName metricName; + private final MetricName commandRunnerStatusMetricName; + private final MetricName commandRunnerDegradedReasonMetricName; - CommandRunnerStatusMetric( + CommandRunnerMetrics( final String ksqlServiceId, final CommandRunner commandRunner, final String metricGroupPrefix @@ -50,7 +51,7 @@ public class CommandRunnerStatusMetric implements Closeable { } @VisibleForTesting - CommandRunnerStatusMetric( + CommandRunnerMetrics( final Metrics metrics, final CommandRunner commandRunner, final String ksqlServiceId, @@ -58,15 +59,23 @@ public class CommandRunnerStatusMetric implements Closeable { ) { this.metrics = Objects.requireNonNull(metrics, "metrics"); final String metricGroupName = metricsGroupPrefix + METRIC_GROUP_POST_FIX; - this.metricName = metrics.metricName( + this.commandRunnerStatusMetricName = metrics.metricName( "status", ReservedInternalTopics.KSQL_INTERNAL_TOPIC_PREFIX + ksqlServiceId + metricGroupName, "The status of the commandRunner thread as it processes the command topic.", Collections.emptyMap() ); + this.commandRunnerDegradedReasonMetricName = metrics.metricName( + "degraded-reason", + ReservedInternalTopics.KSQL_INTERNAL_TOPIC_PREFIX + ksqlServiceId + metricGroupName, + "The reason for why the commandRunner thread is in a DEGRADED state.", + Collections.emptyMap() + ); - this.metrics.addMetric(metricName, (Gauge) + this.metrics.addMetric(commandRunnerStatusMetricName, (Gauge) (config, now) -> commandRunner.checkCommandRunnerStatus().name()); + this.metrics.addMetric(commandRunnerDegradedReasonMetricName, (Gauge) + (config, now) -> commandRunner.getCommandRunnerDegradedReason().name()); } /** @@ -74,6 +83,7 @@ public class CommandRunnerStatusMetric implements Closeable { */ @Override public void close() { - metrics.removeMetric(metricName); + metrics.removeMetric(commandRunnerStatusMetricName); + metrics.removeMetric(commandRunnerDegradedReasonMetricName); } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 786172f5d819..06741dee3695 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -20,8 +20,6 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.server.CommandTopic; import io.confluent.ksql.rest.server.CommandTopicBackup; -import io.confluent.ksql.rest.server.CommandTopicBackupImpl; -import io.confluent.ksql.rest.server.CommandTopicBackupNoOp; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.KsqlServerException; @@ -76,7 +74,6 @@ public class CommandStore implements CommandQueue, Closeable { private final Serializer commandIdSerializer; private final Serializer commandSerializer; private final Deserializer commandIdDeserializer; - private final CommandTopicBackup commandTopicBackup; public static final class Factory { @@ -122,8 +119,7 @@ public static CommandStore create( commandQueueCatchupTimeout, InternalTopicSerdes.serializer(), InternalTopicSerdes.serializer(), - InternalTopicSerdes.deserializer(CommandId.class), - commandTopicBackup + InternalTopicSerdes.deserializer(CommandId.class) ); } } @@ -137,8 +133,7 @@ public static CommandStore create( final Duration commandQueueCatchupTimeout, final Serializer commandIdSerializer, final Serializer commandSerializer, - final Deserializer commandIdDeserializer, - final CommandTopicBackup commandTopicBackup + final Deserializer commandIdDeserializer ) { this.commandTopic = Objects.requireNonNull(commandTopic, "commandTopic"); this.commandStatusMap = Maps.newConcurrentMap(); @@ -157,7 +152,6 @@ public static CommandStore create( Objects.requireNonNull(commandSerializer, "commandSerializer"); this.commandIdDeserializer = Objects.requireNonNull(commandIdDeserializer, "commandIdDeserializer"); - this.commandTopicBackup = Objects.requireNonNull(commandTopicBackup, "commandTopicBackup"); } @Override @@ -329,11 +323,6 @@ private long getCommandTopicOffset() { } } - @Override - public boolean isCorrupted() { - return commandTopicBackup.commandTopicCorruption(); - } - @Override public boolean isEmpty() { return commandTopic.getEndOffset() == 0; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 2b2c1d79f9a2..4013bd54b96b 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -128,17 +128,7 @@ public KsqlResource( authorizationValidator, errorHandler, denyListPropertyValidator, - () -> { - if (commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.DEGRADED) { - return errorHandler.commandRunnerDegradedErrorMessage(); - } - if (commandRunner.checkCommandRunnerStatus() - == CommandRunner.CommandRunnerStatus.CORRUPTED) { - return errorHandler.commandRunnerCorruptedErrorMessage(); - } - return ""; - } + commandRunner::getCommandRunnerDegradedWarning ); } @@ -296,20 +286,23 @@ public EndpointResponse handleKsqlStatements( LOG.info("Processed successfully: " + request); addCommandRunnerWarning( entities, - errorHandler, commandRunnerWarning); return EndpointResponse.ok(entities); } catch (final KsqlRestException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); throw e; } catch (final KsqlStatementException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); return Errors.badStatement(e.getRawMessage(), e.getSqlStatement()); } catch (final KsqlException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); return errorHandler.generateResponse(e, Errors.badRequest(e)); } catch (final Exception e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); + e.printStackTrace(); return errorHandler.generateResponse( e, Errors.serverErrorForStatement(e, request.getKsql())); } @@ -340,10 +333,9 @@ private static void ensureValidPatterns(final List deleteTopicList) { private static void addCommandRunnerWarning( final KsqlEntityList entityList, - final Errors errorHandler, - final Supplier commandRunnerIssue + final Supplier commandRunnerWarning ) { - final String commandRunnerIssueString = commandRunnerIssue.get(); + final String commandRunnerIssueString = commandRunnerWarning.get(); if (!commandRunnerIssueString.equals("")) { for (final KsqlEntity entity: entityList) { entity.updateWarnings(Collections.singletonList( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java index e88d9f49f412..6e7cb8c24fd3 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThrows; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; import com.google.common.base.Ticker; import io.confluent.ksql.rest.entity.CommandId; @@ -187,8 +188,10 @@ public void shouldIgnoreRecordPreviouslyReplayed() throws IOException { } @Test - public void shouldCreateNewReplayFileIfNewRecordsDoNotMatchPreviousBackups() throws IOException { + public void shouldNotCreateNewReplayFileIfNewRecordsDoNotMatchPreviousBackups() throws IOException { // Given + commandTopicBackup = new CommandTopicBackupImpl( + backupLocation.getRoot().getAbsolutePath(), COMMAND_TOPIC_NAME, ticker); final ConsumerRecord record1 = newConsumerRecord(command1); commandTopicBackup.initialize(); commandTopicBackup.writeCommandToBackup(record1); @@ -198,22 +201,13 @@ public void shouldCreateNewReplayFileIfNewRecordsDoNotMatchPreviousBackups() thr // A 2nd initialize call will open the latest backup and read the previous replayed commands commandTopicBackup.initialize(); final ConsumerRecord record2 = newConsumerRecord(command2); - // Need to increase the ticker so the new file has a new timestamp - when(ticker.read()).thenReturn(2L); - // The write command will create a new replay file with the new command + // The write command will conflicts with what's already in the backup file commandTopicBackup.writeCommandToBackup(record2); final BackupReplayFile currentReplayFile = commandTopicBackup.getReplayFile(); // Then - List> commands = previousReplayFile.readRecords(); - assertThat(commands.size(), is(1)); - assertThat(commands.get(0).left, is(command1.left)); - assertThat(commands.get(0).right, is(command1.right)); - commands = currentReplayFile.readRecords(); - assertThat(commands.size(), is(1)); - assertThat(commands.get(0).left, is(command2.left)); - assertThat(commands.get(0).right, is(command2.right)); - assertThat(currentReplayFile.getPath(), not(previousReplayFile.getPath())); + assertThat(currentReplayFile.getPath(), is(previousReplayFile.getPath())); + assertThat(commandTopicBackup.commandTopicCorruption(), is(true)); } @Test @@ -229,12 +223,9 @@ public void shouldWritePreviousReplayedRecordsAlreadyChecked() throws IOExceptio // When // A 2nd initialize call will open the latest backup and read the previous replayed commands commandTopicBackup.initialize(); - // Need to increase the ticker so the new file has a new timestamp - when(ticker.read()).thenReturn(2L); // command1 is ignored because it was previously replayed commandTopicBackup.writeCommandToBackup(record1); - // The write command will create a new replay file with the new command, and command1 will - // be written to have a complete backup + // The write command will conflicts with what's already in the backup file final ConsumerRecord record3 = newConsumerRecord(command3); commandTopicBackup.writeCommandToBackup(record3); final BackupReplayFile currentReplayFile = commandTopicBackup.getReplayFile(); @@ -246,13 +237,16 @@ public void shouldWritePreviousReplayedRecordsAlreadyChecked() throws IOExceptio assertThat(commands.get(0).right, is(command1.right)); assertThat(commands.get(1).left, is(command2.left)); assertThat(commands.get(1).right, is(command2.right)); + + // the backup file should be the same and the contents shouldn't have been modified commands = currentReplayFile.readRecords(); assertThat(commands.size(), is(2)); assertThat(commands.get(0).left, is(command1.left)); assertThat(commands.get(0).right, is(command1.right)); - assertThat(commands.get(1).left, is(command3.left)); - assertThat(commands.get(1).right, is(command3.right)); - assertThat(currentReplayFile.getPath(), not(previousReplayFile.getPath())); + assertThat(commands.get(1).left, is(command2.left)); + assertThat(commands.get(1).right, is(command2.right)); + assertThat(currentReplayFile.getPath(), is(previousReplayFile.getPath())); + assertThat(commandTopicBackup.commandTopicCorruption(), is(true)); } @Test diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java index a88cf2a38c3d..840f73b42471 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java @@ -16,6 +16,8 @@ package io.confluent.ksql.rest.server; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.sameInstance; import static org.hamcrest.core.IsEqual.equalTo; import static org.mockito.ArgumentMatchers.eq; @@ -32,7 +34,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import io.confluent.ksql.rest.server.computation.QueuedCommand; @@ -40,7 +44,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; -import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -62,6 +65,8 @@ public class CommandTopicTest { @Mock private CommandTopicBackup commandTopicBackup; + @Mock + private TopicPartition topicPartition; private final byte[] commandId1 = "commandId1".getBytes(Charset.defaultCharset()); private final byte[] command1 = "command1".getBytes(Charset.defaultCharset()); @@ -73,9 +78,7 @@ public class CommandTopicTest { private ConsumerRecord record1; private ConsumerRecord record2; private ConsumerRecord record3; - - @Mock private ConsumerRecords consumerRecords; @Captor private ArgumentCaptor> topicPartitionsCaptor; @@ -88,7 +91,10 @@ public void setup() { record1 = new ConsumerRecord<>("topic", 0, 0, commandId1, command1); record2 = new ConsumerRecord<>("topic", 0, 1, commandId2, command2); record3 = new ConsumerRecord<>("topic", 0, 2, commandId3, command3); + consumerRecords = + new ConsumerRecords<>(Collections.singletonMap(topicPartition, ImmutableList.of(record1, record2, record3))); commandTopic = new CommandTopic(COMMAND_TOPIC_NAME, commandConsumer, commandTopicBackup); + when(commandTopicBackup.commandTopicCorruption()).thenReturn(false); } @Test @@ -102,16 +108,49 @@ public void shouldAssignCorrectPartitionToConsumer() { } @Test - public void shouldGetNewCommandsIteratorCorrectly() { + public void shouldGetCommandsThatDoNotCorruptBackup() { // Given: when(commandConsumer.poll(any(Duration.class))).thenReturn(consumerRecords); + when(commandTopicBackup.commandTopicCorruption()) + .thenReturn(false) + .thenReturn(false) + .thenReturn(true); // When: final Iterable> newCommands = commandTopic .getNewCommands(Duration.ofHours(1)); + final List> newCommandsList = ImmutableList.copyOf(newCommands); + + // Then: + assertThat(newCommandsList.size(), is(2)); + assertThat(newCommandsList, equalTo(ImmutableList.of(record1, record2))); + } + + @Test + public void shouldGetCommandsThatDoNotCorruptBackupInRestore() { + // Given: + when(commandConsumer.poll(any(Duration.class))) + .thenReturn(someConsumerRecords( + record1, + record2)) + .thenReturn(someConsumerRecords( + record3)) + .thenReturn(new ConsumerRecords<>(Collections.emptyMap())); + when(commandTopicBackup.commandTopicCorruption()) + .thenReturn(false) + .thenReturn(true) + .thenReturn(true); + + // When: + final List queuedCommandList = commandTopic + .getRestoreCommands(Duration.ofMillis(1)); // Then: - assertThat(newCommands, sameInstance(consumerRecords)); + verify(commandConsumer).seekToBeginning(topicPartitionsCaptor.capture()); + assertThat(topicPartitionsCaptor.getValue(), + equalTo(Collections.singletonList(new TopicPartition(COMMAND_TOPIC_NAME, 0)))); + assertThat(queuedCommandList, equalTo(ImmutableList.of( + new QueuedCommand(commandId1, command1, Optional.empty(), 0L)))); } @Test diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java new file mode 100644 index 000000000000..136fef78b4fe --- /dev/null +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetricsTest.java @@ -0,0 +1,159 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.computation; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isA; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.common.metrics.Metrics; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; + +@RunWith(MockitoJUnitRunner.class) +public class CommandRunnerMetricsTest { + + private static final MetricName METRIC_NAME_1 = + new MetricName("bob", "g1", "d1", ImmutableMap.of()); + private static final MetricName METRIC_NAME_2 = + new MetricName("jill", "g1", "d2", ImmutableMap.of()); + private static final String KSQL_SERVICE_ID = "kcql-1-"; + + @Mock + private Metrics metrics; + @Mock + private CommandRunner commandRunner; + @Captor + private ArgumentCaptor> gaugeCaptor; + + private CommandRunnerMetrics commandRunnerMetrics; + + @Before + public void setUp() { + when(metrics.metricName(any(), any(), any(), anyMap())) + .thenReturn(METRIC_NAME_1) + .thenReturn(METRIC_NAME_2); + when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.RUNNING); + when(commandRunner.getCommandRunnerDegradedReason()).thenReturn(CommandRunner.CommandRunnerDegradedReason.NONE); + + commandRunnerMetrics = new CommandRunnerMetrics(metrics, commandRunner, KSQL_SERVICE_ID, "rest"); + } + + @Test + public void shouldAddMetricOnCreation() { + // When: + // Listener created in setup + + // Then: + verify(metrics).metricName("status", "_confluent-ksql-kcql-1-rest-command-runner", + "The status of the commandRunner thread as it processes the command topic.", + Collections.emptyMap()); + verify(metrics).metricName("degraded-reason", "_confluent-ksql-kcql-1-rest-command-runner", + "The reason for why the commandRunner thread is in a DEGRADED state.", + Collections.emptyMap()); + + verify(metrics).addMetric(eq(METRIC_NAME_1), isA(Gauge.class)); + verify(metrics).addMetric(eq(METRIC_NAME_2), isA(Gauge.class)); + } + + @Test + public void shouldInitiallyBeCommandRunnerStatusRunningState() { + // When: + // CommandRunnerStatusMetric created in setup + + // Then: + assertThat(commandRunnerStatusGaugeValue(), is(CommandRunner.CommandRunnerStatus.RUNNING.name())); + } + + @Test + public void shouldUpdateToCommandRunnerStatusErrorState() { + // When: + when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.ERROR); + + // Then: + assertThat(commandRunnerStatusGaugeValue(), is(CommandRunner.CommandRunnerStatus.ERROR.name())); + } + + @Test + public void shouldUpdateToCommandRunnerStatusDegradedState() { + // When: + when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.DEGRADED); + + // Then: + assertThat(commandRunnerStatusGaugeValue(), is(CommandRunner.CommandRunnerStatus.DEGRADED.name())); + } + + @Test + public void shouldInitiallyNoneCommandRunnerDegradedReason() { + // When: + // CommandRunnerStatusMetric created in setup + + // Then: + assertThat(commandRunnerDegradedReasonGaugeValue(), is(CommandRunner.CommandRunnerDegradedReason.NONE.name())); + } + + @Test + public void shouldUpdateToCorruptedCommandRunnerDegradedReason() { + // When: + when(commandRunner.getCommandRunnerDegradedReason()).thenReturn(CommandRunner.CommandRunnerDegradedReason.CORRUPTED); + + // Then: + assertThat(commandRunnerDegradedReasonGaugeValue(), is(CommandRunner.CommandRunnerDegradedReason.CORRUPTED.name())); + } + + @Test + public void shouldUpdateToIncompatibleCommandsCommandRunnerDegradedReason() { + // When: + when(commandRunner.getCommandRunnerDegradedReason()).thenReturn(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND); + + // Then: + assertThat(commandRunnerDegradedReasonGaugeValue(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND.name())); + } + + @Test + public void shouldRemoveNoneCommandRunnerDegradedReason() { + // When: + commandRunnerMetrics.close(); + + // Then: + verify(metrics).removeMetric(METRIC_NAME_1); + verify(metrics).removeMetric(METRIC_NAME_2); + } + + private String commandRunnerStatusGaugeValue() { + verify(metrics).addMetric(eq(METRIC_NAME_1), gaugeCaptor.capture()); + return gaugeCaptor.getValue().value(null, 0L); + } + + private String commandRunnerDegradedReasonGaugeValue() { + verify(metrics).addMetric(eq(METRIC_NAME_2), gaugeCaptor.capture()); + return gaugeCaptor.getValue().value(null, 0L); + } +} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java deleted file mode 100644 index e4ed583c6499..000000000000 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerStatusMetricTest.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Copyright 2019 Confluent Inc. - * - * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.ksql.rest.server.computation; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyMap; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.isA; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import com.google.common.collect.ImmutableMap; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.metrics.Gauge; -import org.apache.kafka.common.metrics.Metrics; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -import java.util.Collections; - -@RunWith(MockitoJUnitRunner.class) -public class CommandRunnerStatusMetricTest { - - private static final MetricName METRIC_NAME = - new MetricName("bob", "g1", "d1", ImmutableMap.of()); - private static final String KSQL_SERVICE_ID = "kcql-1-"; - - @Mock - private Metrics metrics; - @Mock - private CommandRunner commandRunner; - @Captor - private ArgumentCaptor> gaugeCaptor; - - private CommandRunnerStatusMetric commandRunnerStatusMetric; - - @Before - public void setUp() { - when(metrics.metricName(any(), any(), any(), anyMap())).thenReturn(METRIC_NAME); - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.RUNNING); - - commandRunnerStatusMetric = new CommandRunnerStatusMetric(metrics, commandRunner, KSQL_SERVICE_ID, "rest"); - } - - @Test - public void shouldAddMetricOnCreation() { - // When: - // Listener created in setup - - // Then: - verify(metrics).metricName("status", "_confluent-ksql-kcql-1-rest-command-runner", - "The status of the commandRunner thread as it processes the command topic.", - Collections.emptyMap()); - - verify(metrics).addMetric(eq(METRIC_NAME), isA(Gauge.class)); - } - - @Test - public void shouldInitiallyBeRunningState() { - // When: - // CommandRunnerStatusMetric created in setup - - // Then: - assertThat(currentGaugeValue(), is(CommandRunner.CommandRunnerStatus.RUNNING.name())); - } - - @Test - public void shouldUpdateToErrorState() { - // When: - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.ERROR); - - // Then: - assertThat(currentGaugeValue(), is(CommandRunner.CommandRunnerStatus.ERROR.name())); - } - - @Test - public void shouldUpdateToDegradedState() { - // When: - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.DEGRADED); - - // Then: - assertThat(currentGaugeValue(), is(CommandRunner.CommandRunnerStatus.DEGRADED.name())); - } - - @Test - public void shouldRemoveMetricOnClose() { - // When: - commandRunnerStatusMetric.close(); - - // Then: - verify(metrics).removeMetric(METRIC_NAME); - } - - private String currentGaugeValue() { - verify(metrics).addMetric(any(), gaugeCaptor.capture()); - return gaugeCaptor.getValue().value(null, 0L); - } -} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index 921d6f41a28c..1d5e4012e963 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -37,6 +37,7 @@ import com.google.common.collect.ImmutableList; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.metrics.MetricCollectors; +import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -47,16 +48,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Function; -import java.util.stream.Collectors; +import java.util.function.Supplier; -import io.confluent.ksql.util.Pair; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.junit.Before; @@ -72,6 +70,8 @@ @RunWith(MockitoJUnitRunner.class) public class CommandRunnerTest { private static final long COMMAND_RUNNER_HEALTH_TIMEOUT = 1000; + private static final String BACKUP_CORRUPTED_ERROR_MESSAGE = "corrupted"; + private static final String INCOMPATIBLE_COMMANDS_ERROR_MESSAGE = "incompatible"; @Mock private InteractiveStatementExecutor statementExecutor; @@ -103,6 +103,10 @@ public class CommandRunnerTest { private Consumer incompatibleCommandChecker; @Mock private Deserializer commandDeserializer; + @Mock + private Supplier backupCorrupted; + @Mock + private Errors errorHandler; @Captor private ArgumentCaptor threadTaskCaptor; private CommandRunner commandRunner; @@ -123,7 +127,10 @@ public void setup() { doNothing().when(incompatibleCommandChecker).accept(queuedCommand2); doNothing().when(incompatibleCommandChecker).accept(queuedCommand3); + when(backupCorrupted.get()).thenReturn(false); when(compactor.apply(any())).thenAnswer(inv -> inv.getArgument(0)); + when(errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage()).thenReturn(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); + when(errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage()).thenReturn(BACKUP_CORRUPTED_ERROR_MESSAGE); givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); @@ -140,7 +147,9 @@ public void setup() { clock, compactor, incompatibleCommandChecker, - commandDeserializer + commandDeserializer, + backupCorrupted, + errorHandler ); } @@ -233,8 +242,9 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInRestor inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); verify(statementExecutor, never()).handleRestore(queuedCommand3); - } @Test @@ -251,10 +261,12 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInFetch( verify(statementExecutor, never()).handleRestore(queuedCommand2); verify(statementExecutor, never()).handleRestore(queuedCommand3); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); } @Test - public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInRestore() { + public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInRestore() { // Given: givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); doThrow(new IncomaptibleKsqlCommandVersionException("")).when(incompatibleCommandChecker).accept(queuedCommand3); @@ -268,11 +280,13 @@ public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInRestore() { inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); verify(statementExecutor, never()).handleRestore(queuedCommand3); } @Test - public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInFetch() { + public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInFetch() { // Given: givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); doThrow(new IncomaptibleKsqlCommandVersionException("")).when(incompatibleCommandChecker).accept(queuedCommand3); @@ -286,9 +300,32 @@ public void shouldProcessPartialListOfCommandsOnIncomaptibleCommandInFetch() { inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); verify(statementExecutor, never()).handleRestore(queuedCommand3); } + @Test + public void shouldNotProcessCommandTopicIfBackupCorrupted() throws InterruptedException { + // Given: + when(backupCorrupted.get()).thenReturn(true); + + // When: + commandRunner.start(); + verify(commandStore, never()).close(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); + + // Then: + final InOrder inOrder = inOrder(executor, commandStore); + inOrder.verify(commandStore).wakeup(); + inOrder.verify(executor).awaitTermination(anyLong(), any()); + inOrder.verify(commandStore).close(); + assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(BACKUP_CORRUPTED_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.CORRUPTED)); + } + @Test public void shouldPullAndRunStatements() { // Given: diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index ee5b6b05efb7..b61873123b83 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -35,20 +35,17 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.server.CommandTopic; -import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.util.KsqlException; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import io.confluent.ksql.util.Pair; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.producer.Producer; @@ -91,8 +88,6 @@ public class CommandStoreTest { private Serializer commandSerializer; @Mock private Deserializer commandIdDeserializer; - @Mock - private CommandTopicBackup commandTopicBackup; private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); @@ -153,8 +148,7 @@ public void setUp() { TIMEOUT, commandIdSerializer, commandSerializer, - commandIdDeserializer, - commandTopicBackup + commandIdDeserializer ); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java index dc8ecff07fff..f409e745fab8 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java @@ -236,7 +236,7 @@ public void shouldReturnCommandStatus() { @Test public void shouldNotInitTransactionWhenCommandRunnerWarningPresent() { // When: - when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE); + when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); // Then: assertThrows( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index 4f1351bc42a7..db9a8e6e716b 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.not; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -44,6 +45,8 @@ import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.KsqlRequest; +import io.confluent.ksql.rest.server.CommandTopicBackup; +import io.confluent.ksql.rest.server.CommandTopicBackupNoOp; import io.confluent.ksql.rest.server.resources.KsqlResource; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -96,6 +99,11 @@ public class RecoveryTest { @Mock private DenyListPropertyValidator denyListPropertyValidator = mock(DenyListPropertyValidator.class); + + @Mock + private CommandTopicBackup commandTopicBackup = mock(CommandTopicBackup.class); + @Mock + private Errors errorHandler = mock(Errors.class); private final KsqlServer server1 = new KsqlServer(commands); private final KsqlServer server2 = new KsqlServer(commands); @@ -104,6 +112,7 @@ public class RecoveryTest { @Before public void setup() { securityContext = new KsqlSecurityContext(Optional.empty(), serviceContext); + when(commandTopicBackup.commandTopicCorruption()).thenReturn(false); } @After @@ -176,11 +185,6 @@ public boolean isEmpty() { return commandLog.isEmpty(); } - @Override - public boolean isCorrupted() { - return false; - } - @Override public void wakeup() { } @@ -225,7 +229,9 @@ private class KsqlServer { "ksql-service-id", Duration.ofMillis(2000), "", - InternalTopicSerdes.deserializer(Command.class) + InternalTopicSerdes.deserializer(Command.class), + commandTopicBackup, + errorHandler ); this.ksqlResource = new KsqlResource( @@ -234,7 +240,7 @@ private class KsqlServer { Duration.ofMillis(0), ()->{}, Optional.of((sc, metastore, statement) -> { }), - mock(Errors.class), + errorHandler, denyListPropertyValidator ); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 05396dabc21f..36c24a147be8 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -346,6 +346,7 @@ public void setUp() throws IOException, RestClientException { securityContext = new KsqlSecurityContext(Optional.empty(), serviceContext); when(commandRunner.getCommandQueue()).thenReturn(commandStore); + when(commandRunnerWarning.get()).thenReturn(""); when(commandStore.createTransactionalProducer()) .thenReturn(transactionalProducer); @@ -591,18 +592,15 @@ public void shouldHaveKsqlWarningIfCommandRunnerDegraded() { schema); // When: - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.DEGRADED); - when(errorsHandler.commandRunnerDegradedErrorMessage()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE); - final SourceDescriptionList descriptionList1 = makeSingleRequest( "SHOW STREAMS EXTENDED;", SourceDescriptionList.class); - when(commandRunner.checkCommandRunnerStatus()).thenReturn(CommandRunner.CommandRunnerStatus.RUNNING); + when(commandRunnerWarning.get()).thenReturn(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); final SourceDescriptionList descriptionList2 = makeSingleRequest( "SHOW STREAMS EXTENDED;", SourceDescriptionList.class); - assertThat(descriptionList1.getWarnings().size(), is(1)); - assertThat(descriptionList1.getWarnings().get(0).getMessage(), is(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE)); - assertThat(descriptionList2.getWarnings().size(), is(0)); + assertThat(descriptionList1.getWarnings().size(), is(0)); + assertThat(descriptionList2.getWarnings().size(), is(1)); + assertThat(descriptionList2.getWarnings().get(0).getMessage(), is(DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); } @Test diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index 90119495bdc8..e91b24805f8d 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -19,19 +19,20 @@ public class DefaultErrorMessages implements ErrorMessages { - public static final String COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE = + public static final String COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE = "The server has encountered an incompatible entry in its log " + "and cannot process further DDL statements." + System.lineSeparator() + "This is most likely due to the service being rolled back to an earlier version."; - public static final String COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE = + public static final String COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE = "The server has detected that the command topic may be corrupted. The backup of the " + "command topic does not match the current contents of command topic." + System.lineSeparator() + "DDL statements will not be processed until either:" + System.lineSeparator() - + "1. The current command topic is deleted and the backup file is used to restore the command topic." + + "1. The current command topic is deleted and the backup file is used " + + "to restore the command topic." + System.lineSeparator() + "2. The current backup file is deleted." + System.lineSeparator() @@ -64,12 +65,12 @@ public String schemaRegistryUnconfiguredErrorMessage(final Exception e) { } @Override - public String commandRunnerDegradedErrorMessage() { - return COMMAND_RUNNER_DEGRADED_ERROR_MESSAGE; + public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { + return COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE; } @Override - public String metaStoreCorruptedErrorMessage() { - return COMMAND_TOPIC_CORRUPTED_ERROR_MESSAGE; + public String commandRunnerDegradedBackupCorruptedErrorMessage() { + return COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE; } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index 98573f36b67c..56d912e45b33 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -23,7 +23,7 @@ public interface ErrorMessages { String schemaRegistryUnconfiguredErrorMessage(Exception e); - String commandRunnerDegradedErrorMessage(); + String commandRunnerDegradedIncompatibleCommandsErrorMessage(); - String metaStoreCorruptedErrorMessage(); + String commandRunnerDegradedBackupCorruptedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index f3d62ec35fc3..85f5407adff2 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -226,12 +226,12 @@ public String transactionInitTimeoutErrorMessage(final Exception e) { return errorMessages.transactionInitTimeoutErrorMessage(e); } - public String commandRunnerDegradedErrorMessage() { - return errorMessages.commandRunnerDegradedErrorMessage(); + public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { + return errorMessages.commandRunnerDegradedIncompatibleCommandsErrorMessage(); } - public String commandRunnerCorruptedErrorMessage() { - return errorMessages.metaStoreCorruptedErrorMessage(); + public String commandRunnerDegradedBackupCorruptedErrorMessage() { + return errorMessages.commandRunnerDegradedBackupCorruptedErrorMessage(); } public EndpointResponse generateResponse( From 04bf50f4fb7a371f61abf983aaadde99dcd918a2 Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Fri, 18 Sep 2020 14:19:47 -0700 Subject: [PATCH 5/9] feat: surface error to user when command topic deleted while server running --- .../ksql/rest/server/KsqlRestApplication.java | 4 +- .../server/computation/CommandRunner.java | 41 +++++++++++++++--- .../CommandTopicBackupImplFunctionalTest.java | 4 ++ .../server/computation/CommandRunnerTest.java | 43 ++++++++++++++++++- .../rest/server/computation/RecoveryTest.java | 7 ++- .../ksql/rest/DefaultErrorMessages.java | 11 +++++ .../io/confluent/ksql/rest/ErrorMessages.java | 2 + .../java/io/confluent/ksql/rest/Errors.java | 4 ++ 8 files changed, 105 insertions(+), 11 deletions(-) create mode 100644 ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/CommandTopicBackupImplFunctionalTest.java diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index e1e5fc6c6b64..535ae9ab401e 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -756,7 +756,9 @@ static KsqlRestApplication buildApplication( metricsPrefix, InternalTopicSerdes.deserializer(Command.class), finalCommandTopicBackup, - errorHandler + errorHandler, + serviceContext.getTopicClient(), + commandTopicName ); final KsqlResource ksqlResource = new KsqlResource( diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 24b7edd11172..29889aa96c6d 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -23,6 +23,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; @@ -43,6 +44,7 @@ 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,9 +83,11 @@ public class CommandRunner implements Closeable { private final Deserializer commandDeserializer; private final Consumer incompatibleCommandChecker; - private final Supplier backupCorrupted; private final Errors errorHandler; private boolean incompatibleCommandDetected; + private final Supplier backupCorrupted; + private final Supplier commandTopicExists; + private boolean commandTopicDeleted; public enum CommandRunnerStatus { RUNNING, @@ -94,7 +98,8 @@ public enum CommandRunnerStatus { public enum CommandRunnerDegradedReason { NONE, INCOMPATIBLE_COMMAND, - CORRUPTED + CORRUPTED, + COMMAND_TOPIC_DELETED } // CHECKSTYLE_RULES.OFF: ParameterNumberCheck @@ -109,7 +114,9 @@ public CommandRunner( final String metricsGroupPrefix, final Deserializer commandDeserializer, final CommandTopicBackup commandTopicBackup, - final Errors errorHandler + final Errors errorHandler, + final KafkaTopicClient kafkaTopicClient, + final String commandTopicName ) { this( statementExecutor, @@ -129,7 +136,8 @@ public CommandRunner( }, commandDeserializer, commandTopicBackup::commandTopicCorruption, - errorHandler + errorHandler, + () -> kafkaTopicClient.isTopicExists(commandTopicName) ); } @@ -150,7 +158,8 @@ public CommandRunner( final Consumer incompatibleCommandChecker, final Deserializer commandDeserializer, final Supplier backupCorrupted, - final Errors errorHandler + final Errors errorHandler, + final Supplier commandTopicExists ) { // CHECKSTYLE_RULES.ON: ParameterNumberCheck this.statementExecutor = Objects.requireNonNull(statementExecutor, "statementExecutor"); @@ -175,7 +184,10 @@ public CommandRunner( Objects.requireNonNull(backupCorrupted, "backupCorrupted"); this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); + this.commandTopicExists = + Objects.requireNonNull(commandTopicExists, "commandTopicExists"); this.incompatibleCommandDetected = false; + this.commandTopicDeleted = false; } /** @@ -265,6 +277,9 @@ void fetchAndRunCommands() { lastPollTime.set(clock.instant()); final List commands = commandStore.getNewCommands(NEW_CMDS_TIMEOUT); if (commands.isEmpty()) { + if (!commandTopicExists.get()) { + commandTopicDeleted = true; + } return; } @@ -334,7 +349,7 @@ private void terminateCluster(final Command command) { } public CommandRunnerStatus checkCommandRunnerStatus() { - if (incompatibleCommandDetected || backupCorrupted.get()) { + if (incompatibleCommandDetected || backupCorrupted.get() || commandTopicDeleted) { return CommandRunnerStatus.DEGRADED; } @@ -360,6 +375,10 @@ public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { return CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND; } + if (commandTopicDeleted) { + return CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED; + } + return CommandRunnerDegradedReason.NONE; } @@ -372,6 +391,10 @@ public String getCommandRunnerDegradedWarning() { return errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage(); } + if (commandTopicDeleted) { + return errorHandler.commandRunnerDegradedCommandTopicDeletedErrorMessage(); + } + return ""; } @@ -399,7 +422,7 @@ private class Runner implements Runnable { public void run() { try { while (!closed) { - if (incompatibleCommandDetected || backupCorrupted.get()) { + if (incompatibleCommandDetected || backupCorrupted.get() || commandTopicDeleted) { LOG.warn("CommandRunner entering degraded state due to: {}", getCommandRunnerDegradedReason()); closeEarly(); @@ -412,6 +435,10 @@ public void run() { if (!closed) { throw wue; } + } catch (final OffsetOutOfRangeException e) { + LOG.warn("The command topic offset was reset. CommandRunner thread exiting."); + commandTopicDeleted = true; + closeEarly(); } finally { commandStore.close(); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/CommandTopicBackupImplFunctionalTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/CommandTopicBackupImplFunctionalTest.java new file mode 100644 index 000000000000..90697f83617a --- /dev/null +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/CommandTopicBackupImplFunctionalTest.java @@ -0,0 +1,4 @@ +package io.confluent.ksql.rest.integration; + +public class CommandTopicBackupImplFunctionalTest { +} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index 1d5e4012e963..82698cc7e928 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -55,6 +55,8 @@ import java.util.function.Function; import java.util.function.Supplier; +import org.apache.kafka.clients.consumer.OffsetOutOfRangeException; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.junit.Before; @@ -72,6 +74,7 @@ public class CommandRunnerTest { private static final long COMMAND_RUNNER_HEALTH_TIMEOUT = 1000; private static final String BACKUP_CORRUPTED_ERROR_MESSAGE = "corrupted"; private static final String INCOMPATIBLE_COMMANDS_ERROR_MESSAGE = "incompatible"; + private static final String MISSING_COMMAND_TOPIC_ERROR_MESSAGE = "command topic missing"; @Mock private InteractiveStatementExecutor statementExecutor; @@ -106,6 +109,8 @@ public class CommandRunnerTest { @Mock private Supplier backupCorrupted; @Mock + private Supplier commandTopicExists; + @Mock private Errors errorHandler; @Captor private ArgumentCaptor threadTaskCaptor; @@ -128,9 +133,11 @@ public void setup() { doNothing().when(incompatibleCommandChecker).accept(queuedCommand3); when(backupCorrupted.get()).thenReturn(false); + when(commandTopicExists.get()).thenReturn(true); when(compactor.apply(any())).thenAnswer(inv -> inv.getArgument(0)); when(errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage()).thenReturn(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); when(errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage()).thenReturn(BACKUP_CORRUPTED_ERROR_MESSAGE); + when(errorHandler.commandRunnerDegradedCommandTopicDeletedErrorMessage()).thenReturn(MISSING_COMMAND_TOPIC_ERROR_MESSAGE); givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); @@ -149,7 +156,8 @@ public void setup() { incompatibleCommandChecker, commandDeserializer, backupCorrupted, - errorHandler + errorHandler, + commandTopicExists ); } @@ -305,6 +313,20 @@ public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInFetch() { verify(statementExecutor, never()).handleRestore(queuedCommand3); } + @Test + public void shouldEnterDegradedStateIfCommandTopicMissing() { + // Given: + givenQueuedCommands(); + when(commandTopicExists.get()).thenReturn(false); + + // When: + commandRunner.fetchAndRunCommands(); + + assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(MISSING_COMMAND_TOPIC_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED)); + } + @Test public void shouldNotProcessCommandTopicIfBackupCorrupted() throws InterruptedException { // Given: @@ -517,6 +539,25 @@ public void shouldCloseEarlyWhenSerializationExceptionInFetch() throws Exception inOrder.verify(commandStore).close(); } + public void shouldCloseEarlyWhenOffsetOutOfRangeException() throws Exception { + // Given: + when(commandStore.getNewCommands(any())) + .thenReturn(Collections.singletonList(queuedCommand1)) + .thenThrow(new OffsetOutOfRangeException(Collections.singletonMap(new TopicPartition("command_topic", 0), 0L))); + + // When: + commandRunner.start(); + verify(commandStore, never()).close(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); + + // Then: + final InOrder inOrder = inOrder(executor, commandStore); + inOrder.verify(commandStore).wakeup(); + inOrder.verify(executor).awaitTermination(anyLong(), any()); + inOrder.verify(commandStore).close(); + } + @Test public void shouldCloseTheCommandRunnerCorrectly() throws Exception { // Given: diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index db9a8e6e716b..c3cd5651ce6d 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -46,7 +46,6 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.server.CommandTopicBackup; -import io.confluent.ksql.rest.server.CommandTopicBackupNoOp; import io.confluent.ksql.rest.server.resources.KsqlResource; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -68,6 +67,7 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.streams.StreamsConfig; @@ -231,7 +231,9 @@ private class KsqlServer { "", InternalTopicSerdes.deserializer(Command.class), commandTopicBackup, - errorHandler + errorHandler, + topicClient, + "command_topic" ); this.ksqlResource = new KsqlResource( @@ -567,6 +569,7 @@ private void shouldRecover(final List commands) { @Before public void setUp() { topicClient.preconditionTopicExists("A"); + topicClient.preconditionTopicExists("command_topic"); } @Test diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index e91b24805f8d..aaa6e0ea0185 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -38,6 +38,12 @@ public class DefaultErrorMessages implements ErrorMessages { + System.lineSeparator() + "The server must be restarted after performing either operation in order to resume " + "normal functionality"; + + public static final String COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED = + "The server is in a degraded state due to deletion of the command topic. " + + "DDL statements will not be processed." + + System.lineSeparator() + + "Restart the server to restore server functionality."; @Override @@ -73,4 +79,9 @@ public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { public String commandRunnerDegradedBackupCorruptedErrorMessage() { return COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE; } + + @Override + public String commandRunnerDegradedCommandTopicDeletedErrorMessage() { + return COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED; + } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index 56d912e45b33..bee9c2c3ae8a 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -26,4 +26,6 @@ public interface ErrorMessages { String commandRunnerDegradedIncompatibleCommandsErrorMessage(); String commandRunnerDegradedBackupCorruptedErrorMessage(); + + String commandRunnerDegradedCommandTopicDeletedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 85f5407adff2..86d4228af26e 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -234,6 +234,10 @@ public String commandRunnerDegradedBackupCorruptedErrorMessage() { return errorMessages.commandRunnerDegradedBackupCorruptedErrorMessage(); } + public String commandRunnerDegradedCommandTopicDeletedErrorMessage() { + return errorMessages.commandRunnerDegradedCommandTopicDeletedErrorMessage(); + } + public EndpointResponse generateResponse( final Exception e, final EndpointResponse defaultResponse From 98e60fad32c9fee69d48c3033851d83f01d6b5f6 Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Mon, 21 Sep 2020 11:14:32 -0700 Subject: [PATCH 6/9] feedback --- .../io/confluent/ksql/util/KsqlConfig.java | 13 ---------- .../ksql/rest/server/CommandTopic.java | 11 +++++---- .../ksql/rest/server/KsqlRestApplication.java | 9 +------ .../rest/server/computation/CommandQueue.java | 5 ++++ .../server/computation/CommandRunner.java | 16 ++++--------- .../rest/server/computation/CommandStore.java | 14 +++++++++-- .../rest/server/resources/KsqlResource.java | 4 ---- .../server/resources/ServerInfoResource.java | 24 ++++++++++--------- .../ksql/rest/server/CommandTopicTest.java | 9 +++---- .../server/computation/CommandRunnerTest.java | 23 ++++++++---------- .../server/computation/CommandStoreTest.java | 6 ++++- .../rest/server/computation/RecoveryTest.java | 13 +++++----- 12 files changed, 68 insertions(+), 79 deletions(-) diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 39f2110732bb..6787f45bd3f6 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -292,12 +292,6 @@ public class KsqlConfig extends AbstractConfig { public static final String KSQL_CREATE_OR_REPLACE_ENABLED_DOC = "Feature flag for CREATE OR REPLACE"; - public static final String KSQL_ENABLE_METASTORE_BACKUP = "ksql.enable.metastore.backup"; - public static final Boolean KSQL_ENABLE_METASTORE_BACKUP_DEFAULT = false; - public static final String KSQL_ENABLE_METASTORE_BACKUP_DOC = "Enable the KSQL metastore " - + "backup service. The backup replays the KSQL command_topic to a file located in the " - + "same KSQL node."; - public static final String KSQL_METASTORE_BACKUP_LOCATION = "ksql.metastore.backup.location"; public static final String KSQL_METASTORE_BACKUP_LOCATION_DEFAULT = ""; public static final String KSQL_METASTORE_BACKUP_LOCATION_DOC = "Specify the directory where " @@ -739,13 +733,6 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) { Importance.LOW, KSQL_CREATE_OR_REPLACE_ENABLED_DOC ) - .define( - KSQL_ENABLE_METASTORE_BACKUP, - Type.BOOLEAN, - KSQL_ENABLE_METASTORE_BACKUP_DEFAULT, - Importance.LOW, - KSQL_ENABLE_METASTORE_BACKUP_DOC - ) .define( KSQL_METASTORE_BACKUP_LOCATION, Type.STRING, diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java index 8b7004d2a845..15bb5da125f2 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/CommandTopic.java @@ -83,12 +83,13 @@ public Iterable> getNewCommands(final Duration ti final List> records = new ArrayList<>(); if (iterable != null) { - iterable.forEach(record -> { + for (ConsumerRecord record : iterable) { backupRecord(record); - if (!commandTopicBackup.commandTopicCorruption()) { - records.add(record); + if (commandTopicBackup.commandTopicCorruption()) { + return records; } - }); + records.add(record); + } } return records; @@ -109,7 +110,7 @@ public List getRestoreCommands(final Duration duration) { backupRecord(record); if (commandTopicBackup.commandTopicCorruption()) { - continue; + return restoreCommands; } if (record.value() == null) { diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index e1e5fc6c6b64..d52510399b7c 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -657,13 +657,7 @@ static KsqlRestApplication buildApplication( final String commandTopicName = ReservedInternalTopics.commandTopic(ksqlConfig); CommandTopicBackup commandTopicBackup = new CommandTopicBackupNoOp(); - if (ksqlConfig.getBoolean(KsqlConfig.KSQL_ENABLE_METASTORE_BACKUP)) { - if (ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION).isEmpty()) { - throw new KsqlException(String.format("Metastore backups is enabled, but location " - + "is empty. Please specify the location with the property '%s'", - KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION)); - } - + if (!ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION).isEmpty()) { commandTopicBackup = new CommandTopicBackupImpl( ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION), commandTopicName @@ -755,7 +749,6 @@ static KsqlRestApplication buildApplication( KsqlRestConfig.KSQL_COMMAND_RUNNER_BLOCKED_THRESHHOLD_ERROR_MS)), metricsPrefix, InternalTopicSerdes.deserializer(Command.class), - finalCommandTopicBackup, errorHandler ); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java index e97e7795e27f..cffaebb41dd2 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandQueue.java @@ -99,6 +99,11 @@ void ensureConsumedPast(long seqNum, Duration timeout) */ boolean isEmpty(); + /** + * @return whether or not corruption is detected in the queue + */ + boolean corruptionDetected(); + /** * Cause any blocked {@link #getNewCommands(Duration)} calls to return early. * diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 24b7edd11172..24c58aabafd4 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -18,7 +18,6 @@ import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.ClusterTerminateRequest; -import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException; import io.confluent.ksql.rest.server.state.ServerState; import io.confluent.ksql.rest.util.ClusterTerminator; @@ -41,7 +40,6 @@ 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.common.errors.SerializationException; import org.apache.kafka.common.errors.WakeupException; @@ -81,7 +79,6 @@ public class CommandRunner implements Closeable { private final Deserializer commandDeserializer; private final Consumer incompatibleCommandChecker; - private final Supplier backupCorrupted; private final Errors errorHandler; private boolean incompatibleCommandDetected; @@ -108,7 +105,6 @@ public CommandRunner( final Duration commandRunnerHealthTimeout, final String metricsGroupPrefix, final Deserializer commandDeserializer, - final CommandTopicBackup commandTopicBackup, final Errors errorHandler ) { this( @@ -128,7 +124,6 @@ public CommandRunner( queuedCommand.getAndDeserializeCommand(commandDeserializer); }, commandDeserializer, - commandTopicBackup::commandTopicCorruption, errorHandler ); } @@ -149,7 +144,6 @@ public CommandRunner( final Function, List> compactor, final Consumer incompatibleCommandChecker, final Deserializer commandDeserializer, - final Supplier backupCorrupted, final Errors errorHandler ) { // CHECKSTYLE_RULES.ON: ParameterNumberCheck @@ -171,8 +165,6 @@ public CommandRunner( Objects.requireNonNull(incompatibleCommandChecker, "incompatibleCommandChecker"); this.commandDeserializer = Objects.requireNonNull(commandDeserializer, "commandDeserializer"); - this.backupCorrupted = - Objects.requireNonNull(backupCorrupted, "backupCorrupted"); this.errorHandler = Objects.requireNonNull(errorHandler, "errorHandler"); this.incompatibleCommandDetected = false; @@ -334,7 +326,7 @@ private void terminateCluster(final Command command) { } public CommandRunnerStatus checkCommandRunnerStatus() { - if (incompatibleCommandDetected || backupCorrupted.get()) { + if (incompatibleCommandDetected || commandStore.corruptionDetected()) { return CommandRunnerStatus.DEGRADED; } @@ -352,7 +344,7 @@ public CommandRunnerStatus checkCommandRunnerStatus() { } public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { - if (backupCorrupted.get()) { + if (commandStore.corruptionDetected()) { return CommandRunnerDegradedReason.CORRUPTED; } @@ -364,7 +356,7 @@ public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { } public String getCommandRunnerDegradedWarning() { - if (backupCorrupted.get()) { + if (commandStore.corruptionDetected()) { return errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage(); } @@ -399,7 +391,7 @@ private class Runner implements Runnable { public void run() { try { while (!closed) { - if (incompatibleCommandDetected || backupCorrupted.get()) { + if (incompatibleCommandDetected || commandStore.corruptionDetected()) { LOG.warn("CommandRunner entering degraded state due to: {}", getCommandRunnerDegradedReason()); closeEarly(); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java index 06741dee3695..09603670f3e3 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandStore.java @@ -74,6 +74,7 @@ public class CommandStore implements CommandQueue, Closeable { private final Serializer commandIdSerializer; private final Serializer commandSerializer; private final Deserializer commandIdDeserializer; + private final CommandTopicBackup commandTopicBackup; public static final class Factory { @@ -119,7 +120,8 @@ public static CommandStore create( commandQueueCatchupTimeout, InternalTopicSerdes.serializer(), InternalTopicSerdes.serializer(), - InternalTopicSerdes.deserializer(CommandId.class) + InternalTopicSerdes.deserializer(CommandId.class), + commandTopicBackup ); } } @@ -133,7 +135,8 @@ public static CommandStore create( final Duration commandQueueCatchupTimeout, final Serializer commandIdSerializer, final Serializer commandSerializer, - final Deserializer commandIdDeserializer + final Deserializer commandIdDeserializer, + final CommandTopicBackup commandTopicBackup ) { this.commandTopic = Objects.requireNonNull(commandTopic, "commandTopic"); this.commandStatusMap = Maps.newConcurrentMap(); @@ -152,6 +155,8 @@ public static CommandStore create( Objects.requireNonNull(commandSerializer, "commandSerializer"); this.commandIdDeserializer = Objects.requireNonNull(commandIdDeserializer, "commandIdDeserializer"); + this.commandTopicBackup = + Objects.requireNonNull(commandTopicBackup, "commandTopicBackup"); } @Override @@ -323,6 +328,11 @@ private long getCommandTopicOffset() { } } + @Override + public boolean corruptionDetected() { + return commandTopicBackup.commandTopicCorruption(); + } + @Override public boolean isEmpty() { return commandTopic.getEndOffset() == 0; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java index 4013bd54b96b..8cd7e4b3e31a 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/KsqlResource.java @@ -290,19 +290,15 @@ public EndpointResponse handleKsqlStatements( return EndpointResponse.ok(entities); } catch (final KsqlRestException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); - e.printStackTrace(); throw e; } catch (final KsqlStatementException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); - e.printStackTrace(); return Errors.badStatement(e.getRawMessage(), e.getSqlStatement()); } catch (final KsqlException e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); - e.printStackTrace(); return errorHandler.generateResponse(e, Errors.badRequest(e)); } catch (final Exception e) { LOG.info("Processed unsuccessfully: " + request + ", reason: " + e.getMessage()); - e.printStackTrace(); return errorHandler.generateResponse( e, Errors.serverErrorForStatement(e, request.getKsql())); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ServerInfoResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ServerInfoResource.java index ca343fb63b28..92feec5b270b 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ServerInfoResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ServerInfoResource.java @@ -15,7 +15,6 @@ package io.confluent.ksql.rest.server.resources; -import com.google.common.base.Suppliers; import io.confluent.ksql.rest.EndpointResponse; import io.confluent.ksql.rest.entity.ServerInfo; import io.confluent.ksql.rest.server.computation.CommandRunner; @@ -27,23 +26,26 @@ public class ServerInfoResource { - private final Supplier serverInfo; + private final String appVersion; + private final String kafkaClusterId; + private final String ksqlServiceId; + private final Supplier serverStatus; public ServerInfoResource( final ServiceContext serviceContext, final KsqlConfig ksqlConfig, final CommandRunner commandRunner) { - this.serverInfo = Suppliers.memoize( - () -> new ServerInfo( - AppInfo.getVersion(), - KafkaClusterUtil.getKafkaClusterId(serviceContext), - ksqlConfig.getString(KsqlConfig.KSQL_SERVICE_ID_CONFIG), - commandRunner.checkCommandRunnerStatus().toString() - ) - )::get; + appVersion = AppInfo.getVersion(); + kafkaClusterId = KafkaClusterUtil.getKafkaClusterId(serviceContext); + ksqlServiceId = ksqlConfig.getString(KsqlConfig.KSQL_SERVICE_ID_CONFIG); + serverStatus = commandRunner::checkCommandRunnerStatus; } public EndpointResponse get() { - return EndpointResponse.ok(serverInfo.get()); + return EndpointResponse.ok(new ServerInfo( + appVersion, + kafkaClusterId, + ksqlServiceId, + serverStatus.get().toString())); } } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java index 840f73b42471..480960ff872b 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java @@ -22,6 +22,7 @@ import static org.hamcrest.core.IsEqual.equalTo; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -112,7 +113,6 @@ public void shouldGetCommandsThatDoNotCorruptBackup() { // Given: when(commandConsumer.poll(any(Duration.class))).thenReturn(consumerRecords); when(commandTopicBackup.commandTopicCorruption()) - .thenReturn(false) .thenReturn(false) .thenReturn(true); @@ -122,8 +122,9 @@ public void shouldGetCommandsThatDoNotCorruptBackup() { final List> newCommandsList = ImmutableList.copyOf(newCommands); // Then: - assertThat(newCommandsList.size(), is(2)); - assertThat(newCommandsList, equalTo(ImmutableList.of(record1, record2))); + assertThat(newCommandsList.size(), is(1)); + assertThat(newCommandsList, equalTo(ImmutableList.of(record1))); + verify(commandTopicBackup, never()).writeRecord(record3); } @Test @@ -138,7 +139,6 @@ public void shouldGetCommandsThatDoNotCorruptBackupInRestore() { .thenReturn(new ConsumerRecords<>(Collections.emptyMap())); when(commandTopicBackup.commandTopicCorruption()) .thenReturn(false) - .thenReturn(true) .thenReturn(true); // When: @@ -147,6 +147,7 @@ public void shouldGetCommandsThatDoNotCorruptBackupInRestore() { // Then: verify(commandConsumer).seekToBeginning(topicPartitionsCaptor.capture()); + verify(commandConsumer, times(1)).poll(any()); assertThat(topicPartitionsCaptor.getValue(), equalTo(Collections.singletonList(new TopicPartition(COMMAND_TOPIC_NAME, 0)))); assertThat(queuedCommandList, equalTo(ImmutableList.of( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index 1d5e4012e963..b86c45917063 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -104,8 +104,6 @@ public class CommandRunnerTest { @Mock private Deserializer commandDeserializer; @Mock - private Supplier backupCorrupted; - @Mock private Errors errorHandler; @Captor private ArgumentCaptor threadTaskCaptor; @@ -127,7 +125,7 @@ public void setup() { doNothing().when(incompatibleCommandChecker).accept(queuedCommand2); doNothing().when(incompatibleCommandChecker).accept(queuedCommand3); - when(backupCorrupted.get()).thenReturn(false); + when(commandStore.corruptionDetected()).thenReturn(false); when(compactor.apply(any())).thenAnswer(inv -> inv.getArgument(0)); when(errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage()).thenReturn(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); when(errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage()).thenReturn(BACKUP_CORRUPTED_ERROR_MESSAGE); @@ -148,7 +146,6 @@ public void setup() { compactor, incompatibleCommandChecker, commandDeserializer, - backupCorrupted, errorHandler ); } @@ -254,12 +251,12 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInFetch( doThrow(new SerializationException()).when(incompatibleCommandChecker).accept(queuedCommand2); // When: - commandRunner.processPriorCommands(); + commandRunner.fetchAndRunCommands(); // Then: - verify(statementExecutor).handleRestore(eq(queuedCommand1)); - verify(statementExecutor, never()).handleRestore(queuedCommand2); - verify(statementExecutor, never()).handleRestore(queuedCommand3); + verify(statementExecutor).handleStatement(eq(queuedCommand1)); + verify(statementExecutor, never()).handleStatement(queuedCommand2); + verify(statementExecutor, never()).handleStatement(queuedCommand3); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); @@ -292,23 +289,23 @@ public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInFetch() { doThrow(new IncomaptibleKsqlCommandVersionException("")).when(incompatibleCommandChecker).accept(queuedCommand3); // When: - commandRunner.processPriorCommands(); + commandRunner.fetchAndRunCommands(); // Then: final InOrder inOrder = inOrder(statementExecutor); - inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand1)); - inOrder.verify(statementExecutor).handleRestore(eq(queuedCommand2)); + inOrder.verify(statementExecutor).handleStatement(eq(queuedCommand1)); + inOrder.verify(statementExecutor).handleStatement(eq(queuedCommand2)); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND)); - verify(statementExecutor, never()).handleRestore(queuedCommand3); + verify(statementExecutor, never()).handleStatement(queuedCommand3); } @Test public void shouldNotProcessCommandTopicIfBackupCorrupted() throws InterruptedException { // Given: - when(backupCorrupted.get()).thenReturn(true); + when(commandStore.corruptionDetected()).thenReturn(true); // When: commandRunner.start(); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java index b61873123b83..3fc9c39baf90 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandStoreTest.java @@ -35,6 +35,7 @@ import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.server.CommandTopic; +import io.confluent.ksql.rest.server.CommandTopicBackup; import io.confluent.ksql.util.KsqlException; import java.time.Duration; import java.util.ArrayList; @@ -88,6 +89,8 @@ public class CommandStoreTest { private Serializer commandSerializer; @Mock private Deserializer commandIdDeserializer; + @Mock + private CommandTopicBackup commandTopicBackup; private final CommandId commandId = new CommandId(CommandId.Type.STREAM, "foo", CommandId.Action.CREATE); @@ -148,7 +151,8 @@ public void setUp() { TIMEOUT, commandIdSerializer, commandSerializer, - commandIdDeserializer + commandIdDeserializer, + commandTopicBackup ); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java index db9a8e6e716b..526232762953 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/RecoveryTest.java @@ -99,9 +99,7 @@ public class RecoveryTest { @Mock private DenyListPropertyValidator denyListPropertyValidator = mock(DenyListPropertyValidator.class); - - @Mock - private CommandTopicBackup commandTopicBackup = mock(CommandTopicBackup.class); + @Mock private Errors errorHandler = mock(Errors.class); @@ -112,7 +110,6 @@ public class RecoveryTest { @Before public void setup() { securityContext = new KsqlSecurityContext(Optional.empty(), serviceContext); - when(commandTopicBackup.commandTopicCorruption()).thenReturn(false); } @After @@ -179,7 +176,12 @@ public void ensureConsumedPast(final long seqNum, final Duration timeout) { public Producer createTransactionalProducer() { return transactionalProducer; } - + + @Override + public boolean corruptionDetected() { + return false; + } + @Override public boolean isEmpty() { return commandLog.isEmpty(); @@ -230,7 +232,6 @@ private class KsqlServer { Duration.ofMillis(2000), "", InternalTopicSerdes.deserializer(Command.class), - commandTopicBackup, errorHandler ); From 6e8a7e23fb4593c05af86d1809b6389b37a660af Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Mon, 21 Sep 2020 15:27:05 -0700 Subject: [PATCH 7/9] refactor logic in CommandRunner --- .../server/computation/CommandRunner.java | 121 +++++++++++------- .../computation/CommandRunnerMetrics.java | 2 +- .../server/computation/CommandRunnerTest.java | 14 +- 3 files changed, 89 insertions(+), 48 deletions(-) diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 24c58aabafd4..8579d4473e47 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -81,6 +81,7 @@ public class CommandRunner implements Closeable { private final Consumer incompatibleCommandChecker; private final Errors errorHandler; private boolean incompatibleCommandDetected; + private Status state = new Status(CommandRunnerStatus.RUNNING, CommandRunnerDegradedReason.NONE); public enum CommandRunnerStatus { RUNNING, @@ -89,9 +90,19 @@ public enum CommandRunnerStatus { } public enum CommandRunnerDegradedReason { - NONE, - INCOMPATIBLE_COMMAND, - CORRUPTED + NONE(errors -> ""), + CORRUPTED(Errors:: commandRunnerDegradedBackupCorruptedErrorMessage), + INCOMPATIBLE_COMMAND(Errors:: commandRunnerDegradedIncompatibleCommandsErrorMessage); + + private Function msgFactory; + + public String getMsg(final Errors errors) { + return msgFactory.apply(errors); + } + + CommandRunnerDegradedReason(Function msgFactory) { + this.msgFactory = msgFactory; + } } // CHECKSTYLE_RULES.OFF: ParameterNumberCheck @@ -325,64 +336,73 @@ private void terminateCluster(final Command command) { LOG.info("The KSQL server was terminated."); } + private List checkForIncompatibleCommands(final List commands) { + final List compatibleCommands = new ArrayList<>(); + try { + for (final QueuedCommand command : commands) { + incompatibleCommandChecker.accept(command); + compatibleCommands.add(command); + } + } catch (final SerializationException | IncomaptibleKsqlCommandVersionException e) { + LOG.info("Incompatible command record detected when processing command topic", e); + incompatibleCommandDetected = true; + } + return compatibleCommands; + } + + public CommandQueue getCommandQueue() { + return commandStore; + } + public CommandRunnerStatus checkCommandRunnerStatus() { - if (incompatibleCommandDetected || commandStore.corruptionDetected()) { + if (state.getStatus() == CommandRunnerStatus.DEGRADED) { return CommandRunnerStatus.DEGRADED; } final Pair currentCommand = currentCommandRef.get(); if (currentCommand == null) { - return lastPollTime.get() == null + state = lastPollTime.get() == null || Duration.between(lastPollTime.get(), clock.instant()).toMillis() < NEW_CMDS_TIMEOUT.toMillis() * 3 - ? CommandRunnerStatus.RUNNING : CommandRunnerStatus.ERROR; - } - - return Duration.between(currentCommand.right, clock.instant()).toMillis() + ? new Status(CommandRunnerStatus.RUNNING, CommandRunnerDegradedReason.NONE) + : new Status(CommandRunnerStatus.ERROR, CommandRunnerDegradedReason.NONE); + + } else { + state = Duration.between(currentCommand.right, clock.instant()).toMillis() < commandRunnerHealthTimeout.toMillis() - ? CommandRunnerStatus.RUNNING : CommandRunnerStatus.ERROR; - } - - public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { - if (commandStore.corruptionDetected()) { - return CommandRunnerDegradedReason.CORRUPTED; + ? new Status(CommandRunnerStatus.RUNNING, CommandRunnerDegradedReason.NONE) + : new Status(CommandRunnerStatus.ERROR, CommandRunnerDegradedReason.NONE); } - if (incompatibleCommandDetected) { - return CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND; - } + return state.getStatus(); + } - return CommandRunnerDegradedReason.NONE; + public CommandRunnerDegradedReason getCommandRunnerDegradedReason() { + return state.getDegradedReason(); } public String getCommandRunnerDegradedWarning() { - if (commandStore.corruptionDetected()) { - return errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage(); - } - - if (incompatibleCommandDetected) { - return errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage(); - } - - return ""; + return getCommandRunnerDegradedReason().getMsg(errorHandler); } - private List checkForIncompatibleCommands(final List commands) { - final List compatibleCommands = new ArrayList<>(); - try { - for (final QueuedCommand command : commands) { - incompatibleCommandChecker.accept(command); - compatibleCommands.add(command); - } - } catch (final SerializationException | IncomaptibleKsqlCommandVersionException e) { - LOG.info("Incompatible command record detected when processing command topic", e); - incompatibleCommandDetected = true; + public static class Status { + private final CommandRunnerStatus status; + private final CommandRunnerDegradedReason degradedReason; + + public Status( + final CommandRunnerStatus status, + final CommandRunnerDegradedReason degradedReason + ) { + this.status = status; + this.degradedReason = degradedReason; } - return compatibleCommands; - } - public CommandQueue getCommandQueue() { - return commandStore; + public CommandRunnerStatus getStatus() { + return status; + } + public CommandRunnerDegradedReason getDegradedReason() { + return degradedReason; + } } private class Runner implements Runnable { @@ -391,9 +411,20 @@ private class Runner implements Runnable { public void run() { try { while (!closed) { - if (incompatibleCommandDetected || commandStore.corruptionDetected()) { - LOG.warn("CommandRunner entering degraded state due to: {}", - getCommandRunnerDegradedReason()); + if (incompatibleCommandDetected) { + LOG.warn("CommandRunner entering degraded state due to encountering an incompatible command"); + state = new Status( + CommandRunnerStatus.DEGRADED, + CommandRunnerDegradedReason.INCOMPATIBLE_COMMAND + ); + closeEarly(); + } else if (commandStore.corruptionDetected()) { + LOG.warn("CommandRunner entering degraded state due to encountering corruption " + + "between topic and backup"); + state = new Status( + CommandRunnerStatus.DEGRADED, + CommandRunnerDegradedReason.CORRUPTED + ); closeEarly(); } else { LOG.trace("Polling for new writes to command topic"); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java index 4dc1e62b5096..8ac0d926e0f8 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunnerMetrics.java @@ -26,7 +26,7 @@ import org.apache.kafka.common.metrics.Metrics; /** - * Emits JMX metrics that for the CommandRunner thread. + * Emits JMX metrics for the CommandRunner thread. */ public class CommandRunnerMetrics implements Closeable { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index b86c45917063..8ec5c7b6ab3e 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -232,6 +232,9 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInRestor // When: commandRunner.processPriorCommands(); + commandRunner.start(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); // Then: final InOrder inOrder = inOrder(statementExecutor); @@ -251,7 +254,9 @@ public void shouldProcessPartialListOfCommandsOnDeserializationExceptionInFetch( doThrow(new SerializationException()).when(incompatibleCommandChecker).accept(queuedCommand2); // When: - commandRunner.fetchAndRunCommands(); + commandRunner.start(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); // Then: verify(statementExecutor).handleStatement(eq(queuedCommand1)); @@ -270,6 +275,9 @@ public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInRestore() { // When: commandRunner.processPriorCommands(); + commandRunner.start(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); // Then: final InOrder inOrder = inOrder(statementExecutor); @@ -289,7 +297,9 @@ public void shouldProcessPartialListOfCommandsOnIncompatibleCommandInFetch() { doThrow(new IncomaptibleKsqlCommandVersionException("")).when(incompatibleCommandChecker).accept(queuedCommand3); // When: - commandRunner.fetchAndRunCommands(); + commandRunner.start(); + final Runnable threadTask = getThreadTask(); + threadTask.run(); // Then: final InOrder inOrder = inOrder(statementExecutor); From 1ec87c075a61ef906bd12b387217de12f45f2cef Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Tue, 22 Sep 2020 14:48:29 -0700 Subject: [PATCH 8/9] update error messages --- .../rest/server/computation/CommandRunner.java | 17 +++++++++-------- .../server/computation/CommandRunnerTest.java | 7 +++++-- .../ksql/rest/DefaultErrorMessages.java | 8 ++++---- .../io/confluent/ksql/rest/ErrorMessages.java | 2 +- .../java/io/confluent/ksql/rest/Errors.java | 4 ++-- 5 files changed, 21 insertions(+), 17 deletions(-) diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 9968b6599a36..55f546aa5b97 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -98,7 +98,8 @@ public enum CommandRunnerDegradedReason { NONE(errors -> ""), CORRUPTED(Errors::commandRunnerDegradedBackupCorruptedErrorMessage), INCOMPATIBLE_COMMAND(Errors::commandRunnerDegradedIncompatibleCommandsErrorMessage), - COMMAND_TOPIC_DELETED(Errors::commandRunnerDegradedCommandTopicDeletedErrorMessage); + COMMAND_TOPIC_DELETED_OR_MODIFIED( + Errors::commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage); private final Function msgFactory; @@ -445,12 +446,12 @@ public void run() { ); 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(); + LOG.warn("CommandRunner entering degraded state due to command topic deletion."); + state = new Status( + CommandRunnerStatus.DEGRADED, + CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED_OR_MODIFIED + ); + closeEarly(); } else { LOG.trace("Polling for new writes to command topic"); fetchAndRunCommands(); @@ -464,7 +465,7 @@ public void run() { LOG.warn("The command topic offset was reset. CommandRunner thread exiting."); state = new Status( CommandRunnerStatus.DEGRADED, - CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED + CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED_OR_MODIFIED ); closeEarly(); } finally { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index 8e0add77878e..ebc0cf1301e7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -135,7 +135,7 @@ public void setup() { when(compactor.apply(any())).thenAnswer(inv -> inv.getArgument(0)); when(errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage()).thenReturn(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); when(errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage()).thenReturn(BACKUP_CORRUPTED_ERROR_MESSAGE); - when(errorHandler.commandRunnerDegradedCommandTopicDeletedErrorMessage()).thenReturn(MISSING_COMMAND_TOPIC_ERROR_MESSAGE); + when(errorHandler.commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage()).thenReturn(MISSING_COMMAND_TOPIC_ERROR_MESSAGE); givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); @@ -355,7 +355,9 @@ public void shouldEnterDegradedStateIfCommandTopicMissing() { assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(MISSING_COMMAND_TOPIC_ERROR_MESSAGE)); - assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED)); + assertThat( + commandRunner.getCommandRunnerDegradedReason(), + is(CommandRunner.CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED_OR_MODIFIED)); } @Test @@ -549,6 +551,7 @@ public void shouldCloseEarlyWhenSerializationExceptionInFetch() throws Exception inOrder.verify(commandStore).close(); } + @Test public void shouldCloseEarlyWhenOffsetOutOfRangeException() throws Exception { // Given: when(commandStore.getNewCommands(any())) diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index aaa6e0ea0185..60889b049909 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -39,8 +39,8 @@ public class DefaultErrorMessages implements ErrorMessages { + "The server must be restarted after performing either operation in order to resume " + "normal functionality"; - public static final String COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED = - "The server is in a degraded state due to deletion of the command topic. " + public static final String COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED_MODIFIED = + "The server has detected that the command topic has been modified or deleted." + "DDL statements will not be processed." + System.lineSeparator() + "Restart the server to restore server functionality."; @@ -81,7 +81,7 @@ public String commandRunnerDegradedBackupCorruptedErrorMessage() { } @Override - public String commandRunnerDegradedCommandTopicDeletedErrorMessage() { - return COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED; + public String commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage() { + return COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED_MODIFIED; } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index bee9c2c3ae8a..0d610b6791e2 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -27,5 +27,5 @@ public interface ErrorMessages { String commandRunnerDegradedBackupCorruptedErrorMessage(); - String commandRunnerDegradedCommandTopicDeletedErrorMessage(); + String commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 86d4228af26e..604e2412533c 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -234,8 +234,8 @@ public String commandRunnerDegradedBackupCorruptedErrorMessage() { return errorMessages.commandRunnerDegradedBackupCorruptedErrorMessage(); } - public String commandRunnerDegradedCommandTopicDeletedErrorMessage() { - return errorMessages.commandRunnerDegradedCommandTopicDeletedErrorMessage(); + public String commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage() { + return errorMessages.commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage(); } public EndpointResponse generateResponse( From 0368e77d39ca15908756509d2cc4e5c0efa7fbd3 Mon Sep 17 00:00:00 2001 From: Steven Zhang Date: Tue, 22 Sep 2020 21:49:38 -0700 Subject: [PATCH 9/9] combine corrupted and command topic missing degraded reason --- .../server/computation/CommandRunner.java | 10 +++--- .../server/computation/CommandRunnerTest.java | 12 +++---- .../ksql/rest/DefaultErrorMessages.java | 32 ++++++------------- .../io/confluent/ksql/rest/ErrorMessages.java | 4 +-- .../java/io/confluent/ksql/rest/Errors.java | 8 ++--- 5 files changed, 21 insertions(+), 45 deletions(-) diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java index 55f546aa5b97..0bb68edced0a 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/CommandRunner.java @@ -96,10 +96,8 @@ public enum CommandRunnerStatus { public enum CommandRunnerDegradedReason { NONE(errors -> ""), - CORRUPTED(Errors::commandRunnerDegradedBackupCorruptedErrorMessage), - INCOMPATIBLE_COMMAND(Errors::commandRunnerDegradedIncompatibleCommandsErrorMessage), - COMMAND_TOPIC_DELETED_OR_MODIFIED( - Errors::commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage); + CORRUPTED(Errors::commandRunnerDegradedCorruptedErrorMessage), + INCOMPATIBLE_COMMAND(Errors::commandRunnerDegradedIncompatibleCommandsErrorMessage); private final Function msgFactory; @@ -449,7 +447,7 @@ public void run() { LOG.warn("CommandRunner entering degraded state due to command topic deletion."); state = new Status( CommandRunnerStatus.DEGRADED, - CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED_OR_MODIFIED + CommandRunnerDegradedReason.CORRUPTED ); closeEarly(); } else { @@ -465,7 +463,7 @@ public void run() { LOG.warn("The command topic offset was reset. CommandRunner thread exiting."); state = new Status( CommandRunnerStatus.DEGRADED, - CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED_OR_MODIFIED + CommandRunnerDegradedReason.CORRUPTED ); closeEarly(); } finally { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java index ebc0cf1301e7..7d3dc09f1851 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/CommandRunnerTest.java @@ -72,9 +72,8 @@ @RunWith(MockitoJUnitRunner.class) public class CommandRunnerTest { private static final long COMMAND_RUNNER_HEALTH_TIMEOUT = 1000; - private static final String BACKUP_CORRUPTED_ERROR_MESSAGE = "corrupted"; + private static final String CORRUPTED_ERROR_MESSAGE = "corrupted"; private static final String INCOMPATIBLE_COMMANDS_ERROR_MESSAGE = "incompatible"; - private static final String MISSING_COMMAND_TOPIC_ERROR_MESSAGE = "command topic missing"; @Mock private InteractiveStatementExecutor statementExecutor; @@ -134,8 +133,7 @@ public void setup() { when(commandTopicExists.get()).thenReturn(true); when(compactor.apply(any())).thenAnswer(inv -> inv.getArgument(0)); when(errorHandler.commandRunnerDegradedIncompatibleCommandsErrorMessage()).thenReturn(INCOMPATIBLE_COMMANDS_ERROR_MESSAGE); - when(errorHandler.commandRunnerDegradedBackupCorruptedErrorMessage()).thenReturn(BACKUP_CORRUPTED_ERROR_MESSAGE); - when(errorHandler.commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage()).thenReturn(MISSING_COMMAND_TOPIC_ERROR_MESSAGE); + when(errorHandler.commandRunnerDegradedCorruptedErrorMessage()).thenReturn(CORRUPTED_ERROR_MESSAGE); givenQueuedCommands(queuedCommand1, queuedCommand2, queuedCommand3); @@ -337,7 +335,7 @@ public void shouldNotProcessCommandTopicIfBackupCorrupted() throws InterruptedEx inOrder.verify(executor).awaitTermination(anyLong(), any()); inOrder.verify(commandStore).close(); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); - assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(BACKUP_CORRUPTED_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(CORRUPTED_ERROR_MESSAGE)); assertThat(commandRunner.getCommandRunnerDegradedReason(), is(CommandRunner.CommandRunnerDegradedReason.CORRUPTED)); } @@ -354,10 +352,10 @@ public void shouldEnterDegradedStateIfCommandTopicMissing() { threadTask.run(); assertThat(commandRunner.checkCommandRunnerStatus(), is(CommandRunner.CommandRunnerStatus.DEGRADED)); - assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(MISSING_COMMAND_TOPIC_ERROR_MESSAGE)); + assertThat(commandRunner.getCommandRunnerDegradedWarning(), is(CORRUPTED_ERROR_MESSAGE)); assertThat( commandRunner.getCommandRunnerDegradedReason(), - is(CommandRunner.CommandRunnerDegradedReason.COMMAND_TOPIC_DELETED_OR_MODIFIED)); + is(CommandRunner.CommandRunnerDegradedReason.CORRUPTED)); } @Test diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java index 60889b049909..6037391037cb 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/DefaultErrorMessages.java @@ -25,25 +25,16 @@ public class DefaultErrorMessages implements ErrorMessages { + System.lineSeparator() + "This is most likely due to the service being rolled back to an earlier version."; - public static final String COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE = - "The server has detected that the command topic may be corrupted. The backup of the " - + "command topic does not match the current contents of command topic." + public static final String COMMAND_RUNNER_DEGRADED_CORRUPTED_ERROR_MESSAGE = + "The server has detected corruption in the command topic due " + + "to modifications performed on it. " + System.lineSeparator() - + "DDL statements will not be processed until either:" + + "DDL statements will not be processed any further." + System.lineSeparator() - + "1. The current command topic is deleted and the backup file is used " - + "to restore the command topic." + + "If a backup of the command topic is available, " + + "restore the command topic using the backup file." + System.lineSeparator() - + "2. The current backup file is deleted." - + System.lineSeparator() - + "The server must be restarted after performing either operation in order to resume " - + "normal functionality"; - - public static final String COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED_MODIFIED = - "The server has detected that the command topic has been modified or deleted." - + "DDL statements will not be processed." - + System.lineSeparator() - + "Restart the server to restore server functionality."; + + "A server restart is required to restore full functionality."; @Override @@ -76,12 +67,7 @@ public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { } @Override - public String commandRunnerDegradedBackupCorruptedErrorMessage() { - return COMMAND_RUNNER_DEGRADED_BACKUP_CORRUPTED_ERROR_MESSAGE; - } - - @Override - public String commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage() { - return COMMAND_RUNNER_DEGRADED_COMMAND_TOPIC_DELETED_MODIFIED; + public String commandRunnerDegradedCorruptedErrorMessage() { + return COMMAND_RUNNER_DEGRADED_CORRUPTED_ERROR_MESSAGE; } } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java index 0d610b6791e2..269e0e9f1487 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/ErrorMessages.java @@ -25,7 +25,5 @@ public interface ErrorMessages { String commandRunnerDegradedIncompatibleCommandsErrorMessage(); - String commandRunnerDegradedBackupCorruptedErrorMessage(); - - String commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage(); + String commandRunnerDegradedCorruptedErrorMessage(); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 604e2412533c..e929982cc83e 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -230,12 +230,8 @@ public String commandRunnerDegradedIncompatibleCommandsErrorMessage() { return errorMessages.commandRunnerDegradedIncompatibleCommandsErrorMessage(); } - public String commandRunnerDegradedBackupCorruptedErrorMessage() { - return errorMessages.commandRunnerDegradedBackupCorruptedErrorMessage(); - } - - public String commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage() { - return errorMessages.commandRunnerDegradedCommandTopicDeletedModifiedErrorMessage(); + public String commandRunnerDegradedCorruptedErrorMessage() { + return errorMessages.commandRunnerDegradedCorruptedErrorMessage(); } public EndpointResponse generateResponse(