Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: update ksql restore command to skip incompatible commands if flag set #6524

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,15 @@

import com.google.common.collect.Lists;
import io.confluent.ksql.rest.server.computation.QueuedCommand;
import io.confluent.ksql.rest.server.resources.CommandTopicCorruptionException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;

import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
Expand Down Expand Up @@ -86,7 +88,7 @@ public Iterable<ConsumerRecord<byte[], byte[]>> getNewCommands(final Duration ti
for (ConsumerRecord<byte[], byte[]> record : iterable) {
try {
backupRecord(record);
} catch (final Exception e) {
} catch (final CommandTopicCorruptionException e) {
log.warn("Backup is out of sync with the current command topic. "
+ "Backups will not work until the previous command topic is "
+ "restored or all backup files are deleted.", e);
Expand All @@ -113,7 +115,7 @@ public List<QueuedCommand> getRestoreCommands(final Duration duration) {
for (final ConsumerRecord<byte[], byte[]> record : records) {
try {
backupRecord(record);
} catch (final Exception e) {
} catch (final CommandTopicCorruptionException e) {
log.warn("Backup is out of sync with the current command topic. "
+ "Backups will not work until the previous command topic is "
+ "restored or all backup files are deleted.", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,12 @@
package io.confluent.ksql.rest.server;

import com.google.common.annotations.VisibleForTesting;
import io.confluent.ksql.rest.entity.CommandId;
import io.confluent.ksql.rest.server.computation.Command;
import io.confluent.ksql.rest.server.computation.InternalTopicSerdes;
import io.confluent.ksql.rest.server.resources.CommandTopicCorruptionException;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.KsqlServerException;
import io.confluent.ksql.util.Pair;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.nio.file.attribute.PosixFilePermissions;
Expand Down Expand Up @@ -129,41 +125,20 @@ private boolean isRecordInLatestReplay(final ConsumerRecord<byte[], byte[]> reco
return false;
}

private static void throwIfInvalidRecord(final ConsumerRecord<byte[], byte[]> record) {
try {
InternalTopicSerdes.deserializer(CommandId.class).deserialize(record.topic(), record.key());
} catch (final Exception e) {
throw new KsqlException(String.format(
"Failed to backup record because it cannot deserialize key: %s",
new String(record.key(), StandardCharsets.UTF_8)), e);
}

try {
InternalTopicSerdes.deserializer(Command.class).deserialize(record.topic(), record.value());
} catch (final Exception e) {
throw new KsqlException(String.format(
"Failed to backup record because it cannot deserialize value: %s",
new String(record.value(), StandardCharsets.UTF_8)), e
);
}
}

@Override
public void writeRecord(final ConsumerRecord<byte[], byte[]> record) {
if (corruptionDetected) {
throw new KsqlServerException(
throw new CommandTopicCorruptionException(
"Failed to write record due to out of sync command topic and backup file: " + record);
}

throwIfInvalidRecord(record);

if (isRestoring()) {
if (isRecordInLatestReplay(record)) {
// Ignore backup because record was already replayed
return;
} else {
corruptionDetected = true;
throw new KsqlServerException(
throw new CommandTopicCorruptionException(
"Failed to write record due to out of sync command topic and backup file: " + record);
}
} else if (latestReplay.size() > 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
import io.confluent.ksql.config.SessionConfig;
import io.confluent.ksql.engine.KsqlPlan;
import io.confluent.ksql.planner.plan.ConfiguredKsqlPlan;
import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException;
import io.confluent.ksql.rest.server.resources.IncompatibleKsqlCommandVersionException;
import io.confluent.ksql.statement.ConfiguredStatement;
import java.util.Collections;
import java.util.Map;
Expand All @@ -36,7 +36,7 @@
public class Command {

@VisibleForTesting
static final int VERSION = 2;
public static final int VERSION = 2;

private final String statement;
private final Map<String, Object> overwriteProperties;
Expand Down Expand Up @@ -80,7 +80,7 @@ public Command(
}

@VisibleForTesting
Command(
public Command(
final String statement,
final Map<String, Object> overwriteProperties,
final Map<String, String> originalProperties,
Expand All @@ -97,7 +97,7 @@ public Command(
this.version = requireNonNull(version, "version");

if (expectedVersion < version.orElse(0)) {
throw new IncomaptibleKsqlCommandVersionException(
throw new IncompatibleKsqlCommandVersionException(
"Received a command from an incompatible command topic version. "
+ "Expected version less than or equal to " + expectedVersion
+ " but got " + version.orElse(0));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
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.resources.IncomaptibleKsqlCommandVersionException;
import io.confluent.ksql.rest.server.resources.IncompatibleKsqlCommandVersionException;
import io.confluent.ksql.rest.server.state.ServerState;
import io.confluent.ksql.rest.util.ClusterTerminator;
import io.confluent.ksql.rest.util.TerminateCluster;
Expand Down Expand Up @@ -378,7 +378,7 @@ private List<QueuedCommand> checkForIncompatibleCommands(final List<QueuedComman
incompatibleCommandChecker.accept(command);
compatibleCommands.add(command);
}
} catch (final SerializationException | IncomaptibleKsqlCommandVersionException e) {
} catch (final SerializationException | IncompatibleKsqlCommandVersionException e) {
LOG.info("Incompatible command record detected when processing command topic", e);
incompatibleCommandDetected = true;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Copyright 2020 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.resources;

import io.confluent.ksql.util.KsqlServerException;

public class CommandTopicCorruptionException extends KsqlServerException {

public CommandTopicCorruptionException(final String message) {
super(message);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,9 @@

import io.confluent.ksql.util.KsqlServerException;

public class IncomaptibleKsqlCommandVersionException extends KsqlServerException {
public class IncompatibleKsqlCommandVersionException extends KsqlServerException {

public IncomaptibleKsqlCommandVersionException(final String message) {
public IncompatibleKsqlCommandVersionException(final String message) {
super(message);
}

Expand Down
Loading