-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
fix: During ksql startup, avoid recreating deleted command topic when a valid backup exists #7753 #8257
Merged
mkandaswamy
merged 7 commits into
confluentinc:master
from
mkandaswamy:fix_auto_recreating_cmd_topic
Oct 20, 2021
Merged
fix: During ksql startup, avoid recreating deleted command topic when a valid backup exists #7753 #8257
Changes from 6 commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
77bd8ea
fix: Avoid recreating command topic when it has been deleted and a va…
mkandaswamy 0b54c09
fix Integration test avoidRecreatingCommandTopicWithActiveBackup
mkandaswamy cf2e4f4
Added a new unit test in CommandTopicBackupImplTest and fixed checkst…
mkandaswamy 164fb35
Fixed unit test failure due to a NullPointerException. Also, improved…
mkandaswamy 995a19f
Fix integration test failure. Also, refactor integration test code ch…
mkandaswamy 3c1a338
Addressed Steven's review comment. Also, got rid of an unnecessary im…
mkandaswamy b4ecd3b
Addressed Steven's review comment by swapping check for degraded stat…
mkandaswamy File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
55 changes: 55 additions & 0 deletions
55
ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/util/CommandTopicBackupUtil.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* Copyright 2021 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.util; | ||
|
||
import io.confluent.ksql.services.KafkaTopicClient; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
import java.io.File; | ||
import java.lang.reflect.Array; | ||
import java.util.Optional; | ||
|
||
public final class CommandTopicBackupUtil { | ||
|
||
private CommandTopicBackupUtil() { | ||
} | ||
|
||
public static String backupLocation(final KsqlConfig ksqlConfig) { | ||
return Optional.ofNullable( | ||
ksqlConfig.getString(KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION)) | ||
.orElse(""); | ||
} | ||
|
||
public static boolean commandTopicMissingWithValidBackup( | ||
final String commandTopic, | ||
final KafkaTopicClient kafkaTopicClient, | ||
final KsqlConfig ksqlConfig) { | ||
if (kafkaTopicClient.isTopicExists(commandTopic)) { | ||
return false; | ||
} | ||
|
||
final String backupLocation = CommandTopicBackupUtil.backupLocation(ksqlConfig); | ||
if (!backupLocation.isEmpty()) { | ||
final File backupDir = new File(backupLocation); | ||
if (backupDir.exists() && backupDir.isDirectory()) { | ||
final int fileCount = Optional.ofNullable(backupDir.listFiles()) | ||
.map(Array::getLength) | ||
.orElse(0); | ||
return fileCount > 0; | ||
} | ||
} | ||
return false; | ||
} | ||
} |
164 changes: 164 additions & 0 deletions
164
.../io/confluent/ksql/api/integration/QuickDegradeAndRestoreCommandTopicIntegrationTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,164 @@ | ||
package io.confluent.ksql.api.integration; | ||
|
||
import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; | ||
import static io.confluent.ksql.util.KsqlConfig.KSQL_METASTORE_BACKUP_LOCATION; | ||
import static io.confluent.ksql.util.KsqlConfig.KSQL_STREAMS_PREFIX; | ||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.is; | ||
|
||
import io.confluent.common.utils.IntegrationTest; | ||
import io.confluent.ksql.integration.IntegrationTestHarness; | ||
import io.confluent.ksql.integration.Retry; | ||
import io.confluent.ksql.rest.DefaultErrorMessages; | ||
import io.confluent.ksql.rest.entity.KsqlEntity; | ||
import io.confluent.ksql.rest.entity.KsqlWarning; | ||
import io.confluent.ksql.rest.entity.SourceInfo; | ||
import io.confluent.ksql.rest.entity.StreamsList; | ||
import io.confluent.ksql.rest.integration.RestIntegrationTestUtil; | ||
import io.confluent.ksql.rest.server.TestKsqlRestApp; | ||
import io.confluent.ksql.rest.server.restore.KsqlRestoreCommandTopic; | ||
import io.confluent.ksql.test.util.KsqlTestFolder; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
import io.confluent.ksql.util.ReservedInternalTopics; | ||
import java.io.File; | ||
import java.io.IOException; | ||
import java.nio.charset.StandardCharsets; | ||
import java.nio.file.Files; | ||
import java.nio.file.Path; | ||
import java.nio.file.StandardOpenOption; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.stream.Collectors; | ||
import kafka.zookeeper.ZooKeeperClientException; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
import org.junit.After; | ||
import org.junit.Before; | ||
import org.junit.BeforeClass; | ||
import org.junit.ClassRule; | ||
import org.junit.Test; | ||
import org.junit.experimental.categories.Category; | ||
import org.junit.rules.RuleChain; | ||
import org.junit.rules.TemporaryFolder; | ||
|
||
@Category({IntegrationTest.class}) | ||
public class QuickDegradeAndRestoreCommandTopicIntegrationTest { | ||
private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); | ||
|
||
@ClassRule | ||
public static final RuleChain CHAIN = RuleChain | ||
.outerRule(Retry.of(3, ZooKeeperClientException.class, 3, TimeUnit.SECONDS)) | ||
.around(TEST_HARNESS); | ||
|
||
@ClassRule | ||
public static final TemporaryFolder TMP_FOLDER = KsqlTestFolder.temporaryFolder(); | ||
|
||
private static File BACKUP_LOCATION; | ||
private static TestKsqlRestApp REST_APP; | ||
private String commandTopic; | ||
private Path backupFile; | ||
private Path propertiesFile; | ||
|
||
@BeforeClass | ||
public static void classSetUp() throws IOException { | ||
BACKUP_LOCATION = TMP_FOLDER.newFolder(); | ||
|
||
REST_APP = TestKsqlRestApp | ||
.builder(TEST_HARNESS::kafkaBootstrapServers) | ||
.withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) | ||
.withProperty(KSQL_METASTORE_BACKUP_LOCATION, BACKUP_LOCATION.getPath()) | ||
.build(); | ||
} | ||
|
||
@Before | ||
public void setup() throws IOException { | ||
REST_APP.start(); | ||
final KsqlConfig ksqlConfig = new KsqlConfig(REST_APP.getKsqlRestConfig().getKsqlConfigProperties()); | ||
commandTopic = ReservedInternalTopics.commandTopic(ksqlConfig); | ||
backupFile = Files.list(BACKUP_LOCATION.toPath()).findFirst().get(); | ||
propertiesFile = TMP_FOLDER.newFile().toPath(); | ||
writeServerProperties(propertiesFile); | ||
} | ||
|
||
@After | ||
public void teardown() { | ||
REST_APP.stop(); | ||
TEST_HARNESS.deleteTopics(Collections.singletonList(commandTopic)); | ||
} | ||
|
||
@After | ||
public void teardownClass() { | ||
TMP_FOLDER.delete(); | ||
} | ||
|
||
private static void writeServerProperties(final Path propertiesFile) throws IOException { | ||
final Map<String, Object> map = REST_APP.getKsqlRestConfig().getKsqlConfigProperties(); | ||
|
||
Files.write( | ||
propertiesFile, | ||
map.keySet().stream() | ||
.map((key -> key + "=" + map.get(key))) | ||
.collect(Collectors.joining("\n")) | ||
.getBytes(StandardCharsets.UTF_8), | ||
StandardOpenOption.CREATE | ||
); | ||
} | ||
|
||
@Test | ||
public void shouldBeInDegradeModeAfterCmdTopicDeleteAndRestart() throws Exception { | ||
// Given | ||
TEST_HARNESS.ensureTopics("topic5"); | ||
|
||
makeKsqlRequest("CREATE STREAM TOPIC5 (ID INT) " | ||
+ "WITH (KAFKA_TOPIC='topic5', VALUE_FORMAT='JSON');"); | ||
makeKsqlRequest("CREATE STREAM stream5 AS SELECT * FROM topic5;"); | ||
|
||
// When | ||
// Delete the command topic and restart | ||
TEST_HARNESS.deleteTopics(Collections.singletonList(commandTopic)); | ||
REST_APP.stop(); | ||
REST_APP.start(); | ||
|
||
// Then | ||
assertThat(TEST_HARNESS.topicExists(commandTopic), is(false)); | ||
assertThatEventually("Degraded State", this::isDegradedState, is(true)); | ||
|
||
REST_APP.stop(); | ||
KsqlRestoreCommandTopic.main( | ||
new String[]{ | ||
"--yes", | ||
"--config-file", propertiesFile.toString(), | ||
backupFile.toString() | ||
}); | ||
|
||
// Re-load the command topic | ||
REST_APP.start(); | ||
final List<String> streamsNames = showStreams(); | ||
assertThat("Should have TOPIC5", streamsNames.contains("TOPIC5"), is(true)); | ||
assertThat("Should have STREAM5", streamsNames.contains("STREAM5"), is(true)); | ||
assertThatEventually("Degraded State", this::isDegradedState, is(false)); | ||
} | ||
|
||
private boolean isDegradedState() { | ||
// If in degraded state, then the following command will return a warning | ||
final List<KsqlEntity> response = makeKsqlRequest( | ||
"Show Streams;"); | ||
|
||
final List<KsqlWarning> warnings = response.get(0).getWarnings(); | ||
return warnings.size() > 0 && | ||
(warnings.get(0).getMessage().contains( | ||
DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_CORRUPTED_ERROR_MESSAGE) || | ||
warnings.get(0).getMessage().contains( | ||
DefaultErrorMessages.COMMAND_RUNNER_DEGRADED_INCOMPATIBLE_COMMANDS_ERROR_MESSAGE)); | ||
} | ||
|
||
private List<String> showStreams() { | ||
return ((StreamsList)makeKsqlRequest("SHOW STREAMS;").get(0)) | ||
.getStreams().stream().map(SourceInfo::getName).collect(Collectors.toList()); | ||
} | ||
|
||
private List<KsqlEntity> makeKsqlRequest(final String sql) { | ||
return RestIntegrationTestUtil.makeKsqlRequest(REST_APP, sql); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we switch the order of these two statements? I'm not sure the check for the command topic will happen before the server has fully started up and skipped over any parts that may create a command topic. (I don't think
.start()
is blocking)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done. Swapped the order of two statements.