dannycranmer commented on code in PR #110: URL: https://github.com/apache/flink-connector-aws/pull/110#discussion_r1385680247
########## flink-connector-aws/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutShardSubscriber.java: ########## @@ -249,22 +262,21 @@ private FanOutShardSubscription openSubscriptionToShard(final StartingPosition s kinesis.subscribeToShard(request, responseHandler); - boolean subscriptionEstablished = - waitForSubscriptionLatch.await( + boolean subscriptionTimedOut = + !waitForSubscriptionLatch.await( subscribeToShardTimeout.toMillis(), TimeUnit.MILLISECONDS); - if (!subscriptionEstablished) { + if (subscriptionTimedOut) { final String errorMessage = "Timed out acquiring subscription - " + shardId + " (" + consumerArn + ")"; LOG.error(errorMessage); subscription.cancelSubscription(); - handleError( - new RecoverableFanOutSubscriberException(new TimeoutException(errorMessage))); Review Comment: By wrapping this makes it retry forever ``` // Recoverable errors should be reattempted without contributing to the retry policy // A recoverable error would not result in the Flink job being cancelled ``` https://github.com/apache/flink-connector-aws/blob/main/flink-connector-aws/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutRecordPublisher.java#L185 ########## flink-connector-aws/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/FanOutShardSubscriber.java: ########## @@ -249,22 +262,21 @@ private FanOutShardSubscription openSubscriptionToShard(final StartingPosition s kinesis.subscribeToShard(request, responseHandler); - boolean subscriptionEstablished = - waitForSubscriptionLatch.await( + boolean subscriptionTimedOut = + !waitForSubscriptionLatch.await( subscribeToShardTimeout.toMillis(), TimeUnit.MILLISECONDS); - if (!subscriptionEstablished) { + if (subscriptionTimedOut) { final String errorMessage = "Timed out acquiring subscription - " + shardId + " (" + consumerArn + ")"; LOG.error(errorMessage); subscription.cancelSubscription(); - handleError( - new RecoverableFanOutSubscriberException(new TimeoutException(errorMessage))); Review Comment: Ah I see the bug, nice find. It is being wrapped again in `handleError` ########## flink-connector-aws/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/RecoverableErrorsConfig.java: ########## @@ -0,0 +1,99 @@ +/* + * 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.flink.streaming.connectors.kinesis.config; + +import org.apache.commons.collections.CollectionUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +/** + * Hosts the recoverable exception configuration. Recoverable exceptions are retried indefinitely. + */ +public class RecoverableErrorsConfig { + public static final String INVALID_CONFIG_MESSAGE = + "Invalid config for recoverable consumer exceptions. " + + "Valid config example: " + + "`flink.shard.consumer.error.recoverable[0].exception=net.java.UnknownHostException`. " + + "Your config array must use zero-based indexing as shown in the example."; + + /** + * Parses the array of recoverable error configs. + * + * @param config connector configuration + * @return an Optional of RecoverableErrorsConfig + */ + public static Optional<RecoverableErrorsConfig> createConfigFromPropertiesOrThrow( + final Properties config) { + List<ExceptionConfig> exConfs = new ArrayList<>(); + int idx = 0; + String exceptionConfigKey = + String.format( + "%s[%d].exception", + ConsumerConfigConstants.RECOVERABLE_EXCEPTIONS_PREFIX, idx); + while (config.containsKey(exceptionConfigKey)) { + String exPath = config.getProperty(exceptionConfigKey); + try { + Class<?> aClass = Class.forName(exPath); + if (!Throwable.class.isAssignableFrom(aClass)) { + throw new ClassCastException(); + } + exConfs.add(new ExceptionConfig(aClass)); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + "Provided recoverable exception class is not a Throwable: " + exPath); + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException( + "Provided recoverable exception class could not be found: " + exPath); + } + exceptionConfigKey = + String.format( + "%s[%d].exception", + ConsumerConfigConstants.RECOVERABLE_EXCEPTIONS_PREFIX, ++idx); + } + if (idx > 0) { + // We processed configs successfully + return Optional.of(new RecoverableErrorsConfig(exConfs)); + } + + // Check if user provided wrong config suffix, so they fail faster + for (Object key : config.keySet()) { + if (((String) key).startsWith(ConsumerConfigConstants.RECOVERABLE_EXCEPTIONS_PREFIX)) { + throw new IllegalArgumentException(RecoverableErrorsConfig.INVALID_CONFIG_MESSAGE); + } + } + + return Optional.empty(); + } + + private final List<ExceptionConfig> exceptionConfigs; + + public RecoverableErrorsConfig(List<ExceptionConfig> exceptionConfigs) { + this.exceptionConfigs = exceptionConfigs; + } Review Comment: Please move constructors and fields above methods, this is best practise for java. ########## flink-connector-aws/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java: ########## @@ -1021,4 +1021,28 @@ public void testGetV2ConsumerClientProperties() { .containsKey("aws.kinesis.client.user-agent-prefix") .hasSize(2); } + + @Test + public void testInvalidCustomRecoverableErrorConfiguration() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage( + "Provided recoverable exception class could not be found: com.NonExistentExceptionClass"); + + Properties testConfig = TestUtils.getStandardProperties(); + testConfig.setProperty( + ConsumerConfigConstants.RECOVERABLE_EXCEPTIONS_PREFIX + "[0].exception", + "com.NonExistentExceptionClass"); + + KinesisConfigUtil.validateConsumerConfiguration(testConfig); + } + + @Test + public void testValidCustomRecoverableErrorConfiguration() { + Properties testConfig = TestUtils.getStandardProperties(); + testConfig.setProperty( + ConsumerConfigConstants.RECOVERABLE_EXCEPTIONS_PREFIX + "[0].exception", Review Comment: Stutters? -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org