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 04:23:56 UTC

[GitHub] [kafka] mjsax opened a new pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

mjsax opened a new pull request #8886:
URL: https://github.com/apache/kafka/pull/8886


   A StandbyTask must wipe its local state if EOS is enabled and no local
   checkpoint file is found.
   
   I left debug statement in on purpose to people can verify the test locally if they want.
    - when the first instance starts up, the store should be empty
    - if should process key 0 and put into the store
    - after the second instance is running and has replicated the state, the first instance should process key 1 and crash
    - after fail-over to instance 2, the store should only contain key 0
    - instance 2 should reprocess key 1 (we skip it to avoid the crash for this case; note that the store content in not modified; we only need to do this to make sure instance 1 can be restarted and rebuild the standby store)
    - instance 1 is restarted and rebuild the standby store
    - we stop instance 2 an the store is migrated to instance 1: the store should only contain key 0 (if the fix in `StandbyTask` is remove, one can observe that the store content is 0 and 1)
    - we re-enable error injection and re-inject the poison pill: instance 1 should fail again (without the fix, no error occurs because key 1 is detected as duplicated)
   
   Call for review @abbccdda @guozhangwang 


----------------------------------------------------------------
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



[GitHub] [kafka] mjsax commented on pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#issuecomment-645719839


   > Thanks for reproducing the test! IIUC, the main difference for this test vs mine is we use a single thread machine and mock the restart of instance-1?
   
   Just re-read you test again. Seems like it. But maybe the issue is really timing. I have a lot of `waitForConditions` in the test. Maybe in your test without them, you never hit the race condition?


----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441812617



##########
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:
       That is the tradeoff people buy in when using at-least-once. As long as state store recovery is expensive, it's not worth to change it IMHO. Also note, even if we wipe out the store for non-eos, the corrupted record might have been written into the changelog topic already, and thus, we would recover the corrupted state again, not gaining anything.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441813661



##########
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:
       Yeah, the fix is mainly for illustrating purpose atm. The problem is really, that the checkpoint file get's deleted for EOS before we reach this point in the code and thus we need to "hack" around it -- it would be cleaner if we could just check if a checkpoint exists of not...




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
abbccdda commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441874868



##########
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 {
+
+    private final AtomicBoolean skip = new AtomicBoolean(false);
+
+    private String appId;
+    private String inputTopic;
+    private String storeName;
+    private String outputTopic;
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(3);
+
+    @Before
+    public void createTopics() throws Exception {
+        appId = "standbyTest";
+        inputTopic = "testInputTopic";
+        outputTopic = "testOutputTopic";
+        storeName = "dedupStore";
+        CLUSTER.deleteTopicsAndWait(inputTopic, outputTopic);
+        CLUSTER.createTopic(inputTopic, 1, 3);
+        CLUSTER.createTopic(outputTopic, 1, 3);
+    }
+
+    @Test
+    public void shouldWipeOutStandbyStateDirectoryIfCheckpointIsMissing() throws Exception {
+        final String base = TestUtils.tempDirectory(appId).getPath();
+
+        IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
+            inputTopic,
+            Collections.singletonList(
+                new KeyValue<>(0, 0)
+            ),
+            TestUtils.producerConfig(
+                CLUSTER.bootstrapServers(),
+                IntegerSerializer.class,
+                IntegerSerializer.class,
+                new Properties()
+            ),
+            10L
+        );
+
+        try (
+            final KafkaStreams streamInstanceOne = buildWithDeduplicationTopology(base + "-1");
+            final KafkaStreams streamInstanceTwo = buildWithDeduplicationTopology(base + "-2");
+            final KafkaStreams streamInstanceOneRecovery = buildWithDeduplicationTopology(base + "-1")
+        ) {
+            // start first instance and wait for processing
+            startApplicationAndWaitUntilRunning(Collections.singletonList(streamInstanceOne), Duration.ofSeconds(30));
+            IntegrationTestUtils.waitUntilMinRecordsReceived(
+                TestUtils.consumerConfig(
+                    CLUSTER.bootstrapServers(),
+                    IntegerDeserializer.class,
+                    IntegerDeserializer.class
+                ),
+                outputTopic,
+                1
+            );
+
+            // start second instance and wait for standby replication
+            startApplicationAndWaitUntilRunning(Collections.singletonList(streamInstanceTwo), Duration.ofSeconds(30));
+            waitForCondition(
+                () -> streamInstanceTwo.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.<Integer, Integer>keyValueStore()
+                    ).enableStaleStores()
+                ).get(0) != null,

Review comment:
       Could we put key 0 and key 1 as variables?

##########
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 {
+
+    private final AtomicBoolean skip = new AtomicBoolean(false);
+
+    private String appId;
+    private String inputTopic;
+    private String storeName;
+    private String outputTopic;
+
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(3);
+
+    @Before
+    public void createTopics() throws Exception {
+        appId = "standbyTest";
+        inputTopic = "testInputTopic";
+        outputTopic = "testOutputTopic";
+        storeName = "dedupStore";
+        CLUSTER.deleteTopicsAndWait(inputTopic, outputTopic);
+        CLUSTER.createTopic(inputTopic, 1, 3);
+        CLUSTER.createTopic(outputTopic, 1, 3);
+    }
+
+    @Test
+    public void shouldWipeOutStandbyStateDirectoryIfCheckpointIsMissing() throws Exception {
+        final String base = TestUtils.tempDirectory(appId).getPath();
+
+        IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
+            inputTopic,
+            Collections.singletonList(
+                new KeyValue<>(0, 0)
+            ),
+            TestUtils.producerConfig(
+                CLUSTER.bootstrapServers(),
+                IntegerSerializer.class,
+                IntegerSerializer.class,
+                new Properties()
+            ),
+            10L
+        );
+
+        try (
+            final KafkaStreams streamInstanceOne = buildWithDeduplicationTopology(base + "-1");
+            final KafkaStreams streamInstanceTwo = buildWithDeduplicationTopology(base + "-2");
+            final KafkaStreams streamInstanceOneRecovery = buildWithDeduplicationTopology(base + "-1")
+        ) {
+            // start first instance and wait for processing
+            startApplicationAndWaitUntilRunning(Collections.singletonList(streamInstanceOne), Duration.ofSeconds(30));
+            IntegrationTestUtils.waitUntilMinRecordsReceived(
+                TestUtils.consumerConfig(
+                    CLUSTER.bootstrapServers(),
+                    IntegerDeserializer.class,
+                    IntegerDeserializer.class
+                ),
+                outputTopic,
+                1
+            );
+
+            // start second instance and wait for standby replication
+            startApplicationAndWaitUntilRunning(Collections.singletonList(streamInstanceTwo), Duration.ofSeconds(30));
+            waitForCondition(
+                () -> streamInstanceTwo.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.<Integer, Integer>keyValueStore()
+                    ).enableStaleStores()
+                ).get(0) != null,
+                120_000L, // use increased timeout to encounter for rebalancing time
+                "Could not get key from standby store"
+            );
+            // sanity check that first instance is still active
+            waitForCondition(
+                () -> streamInstanceOne.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.<Integer, Integer>keyValueStore()
+                    )
+                ).get(0) != null,
+                "Could not get key from main store"
+            );
+
+            // inject poison pill and wait for crash of first instance and recovery on second instance
+            IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
+                inputTopic,
+                Collections.singletonList(
+                    new KeyValue<>(1, 0)
+                ),
+                TestUtils.producerConfig(
+                    CLUSTER.bootstrapServers(),
+                    IntegerSerializer.class,
+                    IntegerSerializer.class,
+                    new Properties()
+                ),
+                10L
+            );
+            waitForCondition(
+                () -> streamInstanceOne.state() == KafkaStreams.State.ERROR,
+                "Stream instance 1 did not go into error state"
+            );
+            streamInstanceOne.close();
+
+            waitForCondition(
+                () -> streamInstanceTwo.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.keyValueStore()
+                    )
+                ).get(0) != null,
+                120_000L, // use increased timeout to encounter for rebalancing time
+                "Could not get key from recovered main store"
+            );
+
+            // "restart" first client and wait for standby recovery
+            startApplicationAndWaitUntilRunning(
+                Collections.singletonList(streamInstanceOneRecovery),
+                Duration.ofSeconds(30)
+            );
+            waitForCondition(
+                () -> streamInstanceOneRecovery.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.<Integer, Integer>keyValueStore()
+                    ).enableStaleStores()
+                ).get(0) != null,
+                "Could not get key from recovered standby store"
+            );
+            // sanity check that second instance is still active
+            waitForCondition(
+                () -> streamInstanceTwo.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.keyValueStore()
+                    )
+                ).get(0) != null,
+                "Could not get key from recovered main store"
+            );
+
+            streamInstanceTwo.close();
+            waitForCondition(
+                () -> streamInstanceOneRecovery.store(
+                    StoreQueryParameters.fromNameAndType(
+                        storeName,
+                        QueryableStoreTypes.<Integer, Integer>keyValueStore()
+                    )
+                ).get(0) != null,
+                120_000L, // use increased timeout to encounter for rebalancing time
+                "Could not get key from recovered main store"
+            );
+
+            // re-inject poison pill and wait for crash of first instance
+            skip.set(false);
+            IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
+                inputTopic,
+                Collections.singletonList(
+                    new KeyValue<>(1, 0)
+                ),
+                TestUtils.producerConfig(
+                    CLUSTER.bootstrapServers(),
+                    IntegerSerializer.class,
+                    IntegerSerializer.class,
+                    new Properties()
+                ),
+                10L
+            );
+            waitForCondition(
+                () -> streamInstanceOneRecovery.state() == KafkaStreams.State.ERROR,
+                "Stream instance 1 did not go into error state"
+            );
+        }
+    }
+
+    private KafkaStreams buildWithDeduplicationTopology(final String stateDirPath) {
+        final StreamsBuilder builder = new StreamsBuilder();
+
+        builder.addStateStore(Stores.keyValueStoreBuilder(
+            Stores.persistentKeyValueStore(storeName),
+            Serdes.Integer(),
+            Serdes.Integer())
+        );
+        builder.<Integer, Integer>stream(inputTopic)
+            .transform(
+                () -> new Transformer<Integer, Integer, KeyValue<Integer, Integer>>() {
+                    private KeyValueStore<Integer, Integer> store;
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public void init(final ProcessorContext context) {
+                        store = (KeyValueStore<Integer, Integer>) context.getStateStore(storeName);
+
+                        final KeyValueIterator<Integer, Integer> it = store.all();
+                        System.err.println("mjsax: store content begin");
+                        while (it.hasNext()) {
+                            final KeyValue<Integer, Integer> next = it.next();
+                            System.err.println("mjsax: key/value -> " + next.key + "/" + next.value);

Review comment:
       Remove the debug statements




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441814195



##########
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:
       This PR is for `2.5`, not trunk. I have another branch for `trunk` and there both cases are covered.




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441917481



##########
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:
       Not sure how to proceed? The naming issue might be a general problem, that we might want to fix in trunk/2.6? Is is worth to fix in 2.5?




----------------------------------------------------------------
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



[GitHub] [kafka] mjsax commented on pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#issuecomment-646427756


   Java 8 passed.
   Java 11:
   ```
   org.apache.kafka.connect.mirror.MirrorConnectorsIntegrationTest.testReplication
   ```


----------------------------------------------------------------
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



[GitHub] [kafka] mjsax commented on pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#issuecomment-646376402


   Retest this please.


----------------------------------------------------------------
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



[GitHub] [kafka] guozhangwang commented on pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#issuecomment-645633454


   cc @ableegoldman could you take a look at this PR as well?


----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441876159



##########
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 think `checkpointed` is just misleadingly named -- it's really more like checkpoint_able_ offsets IIUC




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441920056



##########
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 fine with leaving the poorly-named method as is, since it's fixed in trunk. But maybe you could leave a comment explaining why this seemingly contradictory set of logic actually makes sense (it took me more than a second to understand this at least)




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441876159



##########
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 think `checkpointed` is just misleadingly named -- it's really more like checkpoint-_able_ offsets IIUC




----------------------------------------------------------------
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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#discussion_r441876159



##########
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 think `checkpointed` is just misleadingly named -- it's really more like checkpoint _able_ offsets IIUC




----------------------------------------------------------------
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



[GitHub] [kafka] mjsax commented on pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #8886:
URL: https://github.com/apache/kafka/pull/8886#issuecomment-646376357


   Java 8 passed.
   Java 11:
   ```
   kafka.api.ConsumerBounceTest.testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup
   ```


----------------------------------------------------------------
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



[GitHub] [kafka] mjsax merged pull request #8886: KAFKA-9891: fix corrupted StandbyTask state

Posted by GitBox <gi...@apache.org>.
mjsax merged pull request #8886:
URL: https://github.com/apache/kafka/pull/8886


   


----------------------------------------------------------------
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