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 2022/02/24 23:09:19 UTC

[GitHub] [kafka] rhauch commented on a change in pull request #11797: KAFKA-12879: Revert changes from KAFKA-12339 and instead add retry capability to KafkaBasedLog

rhauch commented on a change in pull request #11797:
URL: https://github.com/apache/kafka/pull/11797#discussion_r814336427



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java
##########
@@ -703,6 +706,20 @@ public Config describeTopicConfig(String topic) {
         return result;
     }
 
+    protected Map<TopicPartition, Long> retryEndOffsets(Set<TopicPartition> partitions) {
+        int maxRetries = Integer.parseInt(adminConfig.getOrDefault(AdminClientConfig.RETRIES_CONFIG, DEFAULT_ADMIN_CLIENT_RETRIES).toString());
+        long retryBackoffMs = Long.parseLong(adminConfig.getOrDefault(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, DEFAULT_ADMIN_CLIENT_BACKOFF_MS).toString());
+
+        try {
+            return RetryUtil.retry(
+                    () -> endOffsets(partitions),
+                    maxRetries,

Review comment:
       The values of `maxRetries` and `retryBackoffMs` can each be `0`, and these are passed directly into the `RetryUtil.retry(...)` method. See my comment there. 

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/RetryUtilTest.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.connect.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.concurrent.Callable;
+
+@RunWith(PowerMockRunner.class)
+public class RetryUtilTest {
+
+    @Mock
+    private Callable<String> mockCallable;
+
+    @SuppressWarnings("unchecked")
+    @Before
+    public void setUp() throws Exception {
+        mockCallable = Mockito.mock(Callable.class);
+    }
+
+    @Test
+    public void TestSuccess() throws Exception {
+        Mockito.when(mockCallable.call()).thenReturn("success");
+        //EasyMock.expect(mockCallable.call()).andReturn("success").times(1);
+        try {
+            assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
+        } catch (Exception e) {
+            fail("Not expecting an exception: ", e.getCause());
+        }
+        Mockito.verify(mockCallable, Mockito.times(1)).call();
+    }
+
+    @Test
+    public void TestFailingRetries() throws Exception {
+        Mockito.when(mockCallable.call()).thenThrow(new TimeoutException());
+        try {
+            RetryUtil.retry(mockCallable, 10, 100);
+            fail("Expect exception being thrown here");
+        } catch (ConnectException e) {
+            // expecting a connect exception
+        } catch (Exception e) {
+            fail("Only expecting ConnectException");
+        }

Review comment:
       It's much better to use `Assertion.assertThrows(...)`:
   ```suggestion
           ConnectException e = assertThrows(ConnectException.class, RetryUtil.retry(mockCallable, 10, 100));
           assertTrue(e.getMesssage().startsWith("Fail to retry the operation after 10 attempts."));
   ```

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/RetryUtilTest.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.connect.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.concurrent.Callable;
+
+@RunWith(PowerMockRunner.class)
+public class RetryUtilTest {
+
+    @Mock
+    private Callable<String> mockCallable;
+
+    @SuppressWarnings("unchecked")
+    @Before
+    public void setUp() throws Exception {
+        mockCallable = Mockito.mock(Callable.class);
+    }
+
+    @Test
+    public void TestSuccess() throws Exception {
+        Mockito.when(mockCallable.call()).thenReturn("success");
+        //EasyMock.expect(mockCallable.call()).andReturn("success").times(1);

Review comment:
       Let's remove this line

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/RetryUtilTest.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.connect.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.concurrent.Callable;
+
+@RunWith(PowerMockRunner.class)
+public class RetryUtilTest {
+
+    @Mock
+    private Callable<String> mockCallable;
+
+    @SuppressWarnings("unchecked")
+    @Before
+    public void setUp() throws Exception {
+        mockCallable = Mockito.mock(Callable.class);
+    }
+
+    @Test
+    public void TestSuccess() throws Exception {
+        Mockito.when(mockCallable.call()).thenReturn("success");
+        //EasyMock.expect(mockCallable.call()).andReturn("success").times(1);
+        try {
+            assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
+        } catch (Exception e) {
+            fail("Not expecting an exception: ", e.getCause());
+        }
+        Mockito.verify(mockCallable, Mockito.times(1)).call();
+    }
+
+    @Test
+    public void TestFailingRetries() throws Exception {
+        Mockito.when(mockCallable.call()).thenThrow(new TimeoutException());
+        try {
+            RetryUtil.retry(mockCallable, 10, 100);
+            fail("Expect exception being thrown here");
+        } catch (ConnectException e) {
+            // expecting a connect exception
+        } catch (Exception e) {
+            fail("Only expecting ConnectException");
+        }
+        Mockito.verify(mockCallable, Mockito.times(10)).call();
+    }
+
+    @Test
+    public void RetriesEventuallySucceed() throws Exception {
+        Mockito.when(mockCallable.call())
+                .thenThrow(new TimeoutException())
+                .thenThrow(new TimeoutException())
+                .thenThrow(new TimeoutException())
+                .thenReturn("success");
+        try {
+            assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
+        } catch (Exception e) {
+            fail("Not expecting an exception: ", e.getCause());
+        }
+        Mockito.verify(mockCallable, Mockito.times(4)).call();
+    }
+
+    @Test
+    public void FailWithNonRetriableException() throws Exception {
+        Mockito.when(mockCallable.call())
+                .thenThrow(new TimeoutException("timeout"))
+                .thenThrow(new TimeoutException("timeout"))
+                .thenThrow(new TimeoutException("timeout"))
+                .thenThrow(new TimeoutException("timeout"))
+                .thenThrow(new TimeoutException("timeout"))
+                .thenThrow(new NullPointerException("Non retriable"));
+        try {
+            for (int i = 0; i < 10; i++) {
+                RetryUtil.retry(mockCallable, 10, 100);
+            }
+            fail("Not expecting an exception: ");
+        } catch (ConnectException e) {
+            fail("Should fail with NPE");
+        } catch (NullPointerException e) {
+            // good
+        }

Review comment:
       What is the purpose of the for loop? Why not just:
   ```suggestion
           NullPointerException e = assertThrows(NullPointerException.class, RetryUtil.retry(mockCallable, 10, 100));
           assertTrue(e.getMesssage().startsWith("Non retriable"));
   ```

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/RetryUtil.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.connect.util;
+
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Callable;
+
+public class RetryUtil {
+    private static final Logger log = LoggerFactory.getLogger(RetryUtil.class);
+
+    public static <T> T retry(Callable<T> callable, long maxRetries, long retryBackoffMs) throws Exception {
+        Throwable lastError = null;
+        int retries = 0;
+        while (retries++ < maxRetries) {
+            try {
+                return callable.call();

Review comment:
       Even though this method will not be part of our public API, we should still add JavaDoc so that we can document what the allowed values for `maxRetries` and `retryBackoffMs` can be, and what exceptions might be thrown by this method.
   
   Second, is `maxRetries` the total number of _attempts_, or the total number of _retries_ that are allowed. With the way this method is called by `KafkaBasedLog`, it is possible that both or either of these values are `0` -- which makes sense for "retries". But if `retries` is `0`, then this method will retry for a long period of times. Let's fix this logic to handle either or both `maxRetries` and/or `retryBackoffMs` being `0`, and add unit tests to covert this situation.

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/RetryUtilTest.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.connect.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.concurrent.Callable;
+
+@RunWith(PowerMockRunner.class)
+public class RetryUtilTest {
+
+    @Mock
+    private Callable<String> mockCallable;
+
+    @SuppressWarnings("unchecked")
+    @Before
+    public void setUp() throws Exception {
+        mockCallable = Mockito.mock(Callable.class);
+    }
+
+    @Test
+    public void TestSuccess() throws Exception {
+        Mockito.when(mockCallable.call()).thenReturn("success");
+        //EasyMock.expect(mockCallable.call()).andReturn("success").times(1);
+        try {
+            assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
+        } catch (Exception e) {
+            fail("Not expecting an exception: ", e.getCause());
+        }

Review comment:
       Is this any better than just:
   ```suggestion
           assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
   ```

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/RetryUtilTest.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.connect.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.connect.errors.ConnectException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.concurrent.Callable;
+
+@RunWith(PowerMockRunner.class)
+public class RetryUtilTest {
+
+    @Mock
+    private Callable<String> mockCallable;
+
+    @SuppressWarnings("unchecked")
+    @Before
+    public void setUp() throws Exception {
+        mockCallable = Mockito.mock(Callable.class);
+    }
+
+    @Test
+    public void TestSuccess() throws Exception {
+        Mockito.when(mockCallable.call()).thenReturn("success");
+        //EasyMock.expect(mockCallable.call()).andReturn("success").times(1);
+        try {
+            assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
+        } catch (Exception e) {
+            fail("Not expecting an exception: ", e.getCause());
+        }
+        Mockito.verify(mockCallable, Mockito.times(1)).call();
+    }
+
+    @Test
+    public void TestFailingRetries() throws Exception {
+        Mockito.when(mockCallable.call()).thenThrow(new TimeoutException());
+        try {
+            RetryUtil.retry(mockCallable, 10, 100);
+            fail("Expect exception being thrown here");
+        } catch (ConnectException e) {
+            // expecting a connect exception
+        } catch (Exception e) {
+            fail("Only expecting ConnectException");
+        }
+        Mockito.verify(mockCallable, Mockito.times(10)).call();
+    }
+
+    @Test
+    public void RetriesEventuallySucceed() throws Exception {
+        Mockito.when(mockCallable.call())
+                .thenThrow(new TimeoutException())
+                .thenThrow(new TimeoutException())
+                .thenThrow(new TimeoutException())
+                .thenReturn("success");
+        try {
+            assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
+        } catch (Exception e) {
+            fail("Not expecting an exception: ", e.getCause());
+        }

Review comment:
       The try-catch is not really needed, since any exception thrown will fail the test:
   ```suggestion
           assertEquals("success", RetryUtil.retry(mockCallable, 10, 100));
   ```




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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