You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/06/17 15:36:05 UTC

[GitHub] [kafka] vvcephei commented on a change in pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

vvcephei commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441635399



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.integration;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.serialization.IntegerDeserializer;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StoreQueryParameters;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Transformer;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.QueryableStoreTypes;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning;
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+
+/**
+ * An integration test to verify the conversion of a dirty-closed EOS
+ * task towards a standby task is safe across restarts of the application.
+ */
+public class StandbyTaskEOSIntegrationTest {

Review comment:
       should this be parameterized for both flavors of eos?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
##########
@@ -85,6 +85,21 @@ public void initializeMetadata() {}
     public boolean initializeStateStores() {
         log.trace("Initializing state stores");
         registerStateStores();
+
+        if (eosEnabled) {
+            final Set<TopicPartition> partitionsToReinitialize = new HashSet<>();
+            for (final TopicPartition partition : stateMgr.checkpointed().keySet()) {

Review comment:
       I'm sure this works, but something seems odd about listing _checkpointed_ partitions to find ones that are _not checkpointed_. Maybe we need to encapsulate this search, or provide a more semantically appropriate "list all changelog partitions" method?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
##########
@@ -85,6 +85,21 @@ public void initializeMetadata() {}
     public boolean initializeStateStores() {
         log.trace("Initializing state stores");
         registerStateStores();
+
+        if (eosEnabled) {

Review comment:
       I have always been, and continue to be, mystified about why we're 100% a-ok with having corrupted state stores in ALOS mode. I understand that there are some kinds of corruption that are unavoidable without some extra work in the stores, but this check is just explicitly ignoring detectable corruption, which seems a bit extreme. Can we remove this conditional? The rest of these checks seem to apply equally well under all execution semantic modes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org