You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "eskabetxe (via GitHub)" <gi...@apache.org> on 2023/03/24 09:51:40 UTC

[GitHub] [flink] eskabetxe opened a new pull request, #22266: [FLINK-31525] Stabilize JdbcExactlyOnceSinkE2eTest

eskabetxe opened a new pull request, #22266:
URL: https://github.com/apache/flink/pull/22266

   ## What is the purpose of the change
   
   Stabilize JdbcExactlyOnceSinkE2eTest
   
   ## Brief change log
   
     - Remove job checkpoint configuration, default to cluster configuration
     - Change FailingMapper from random failure to expected failure, this allow to confirm the test behaviour
   
   ## Verifying this change
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on PR #22266:
URL: https://github.com/apache/flink/pull/22266#issuecomment-1489841744

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1151597857


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   sounds reasonable, thanks for explanation



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1150557681


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   This test use a FailureMapper that fails in a Random way Random times, and uses the min and max ElementsPerFailure for that..
   I change this FailureMapper to fail the expectedFailures and have a predictable behaviour 
   line 197: map(new FailingMapper(numElementsPerCheckpoint + (numElementsPerCheckpoint / 2)))
   
   The restartStrategy is change from a infinite (Integer.MAX_VALUE) to double of expected failures, so the test can fail in some way..
   
   What happen now with the Oracle test is that the test enters in a loop of failures but never get to max restarts and hangs up.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on PR #22266:
URL: https://github.com/apache/flink/pull/22266#issuecomment-1488290231

   lgtm, once ci is passed


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin merged pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin merged PR #22266:
URL: https://github.com/apache/flink/pull/22266


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1150532918


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   Can you please elaborate a bit more about the changed math?
   I'm not telling that it is wrong, just want to understand the reasoning



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1150557681


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   This test use a FailureMapper that fails in a Random way Random times, and uses the min and max ElementsPerFailure for that..
   I change this FailureMapper to fail the expectedFailures and have a predictable behaviour.
   
   The restartStrategy is change from a infinite (Integer.MAX_VALUE) to double of expected failures, so the test can fail in some way..
   
   What happen now with the Oracle test is that the test enters in a loop of failures but never get to max restarts and hangs up.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on PR #22266:
URL: https://github.com/apache/flink/pull/22266#issuecomment-1489194393

   based on exception this should help https://github.com/apache/flink/commit/36f39712d289b6b7b0b957a33bd1891266e3f992
   
   probably need to rebase


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on PR #22266:
URL: https://github.com/apache/flink/pull/22266#issuecomment-1489887387

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1150557681


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   This test use a FailureMapper that fails in a Random way Random times, and uses the min and max ElementsPerFailure for that.. I change this FailureMapper to fail in an a predictable behaviour .
   
   The restartStrategy is change from a infinite (Integer.MAX_VALUE) to double of expected failures, so the test can fail in some way..
   
   What happen now with the Oracle test is that the test enters in a loop of failures (by FailingMapper or checkpoint timeout) but never get to max restarts and hangs up until pipeline is cancel for time.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] flinkbot commented on pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "flinkbot (via GitHub)" <gi...@apache.org>.
flinkbot commented on PR #22266:
URL: https://github.com/apache/flink/pull/22266#issuecomment-1482534542

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3efc242e281d023a7434f3902ca94609d43d2299",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3efc242e281d023a7434f3902ca94609d43d2299",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3efc242e281d023a7434f3902ca94609d43d2299 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1150557681


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   This test use a FailureMapper that fails in a Random way Random times, and uses the min and max ElementsPerFailure for that.. I change this FailureMapper to fail in an a predictable behaviour .
   
   The restartStrategy is change from a infinite (Integer.MAX_VALUE) to double of expected failures, so the test can fail in some way..
   
   What happen now with the Oracle test is that the test enters in a loop of failures but never get to max restarts and hangs up.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on PR #22266:
URL: https://github.com/apache/flink/pull/22266#issuecomment-1488359150

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1150557681


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -195,22 +181,20 @@ public void testInsert() throws Exception {
         long started = System.currentTimeMillis();
         LOG.info("Test insert for {}", dbEnv);
         int elementsPerSource = 50;
-        int numElementsPerCheckpoint = 7;
-        int minElementsPerFailure = numElementsPerCheckpoint / 3;
-        int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+        int numElementsPerCheckpoint = 10;
+        int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
 
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(dbEnv.getParallelism());
-        env.setRestartStrategy(fixedDelayRestart(Integer.MAX_VALUE, Time.milliseconds(100)));
-        env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+        env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+        env.getConfig().setAutoWatermarkInterval(0L);

Review Comment:
   This test use a FailureMapper that fails in a Random way Random times, and uses the min and max ElementsPerFailure for that..
   I change this FailureMapper to fail the expectedFailures and have a predictable behaviour .
   
   The restartStrategy is change from a infinite (Integer.MAX_VALUE) to double of expected failures, so the test can fail in some way..
   
   What happen now with the Oracle test is that the test enters in a loop of failures but never get to max restarts and hangs up.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1151599728


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -92,25 +89,14 @@
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkState;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.slf4j.event.Level.TRACE;
 
 /** A simple end-to-end test for {@link JdbcXaSinkFunction}. */
 @RunWith(Parameterized.class)
 public class JdbcExactlyOnceSinkE2eTest extends JdbcTestBase {
-    private static final Random RANDOM = new Random(System.currentTimeMillis());
-
     private static final Logger LOG = LoggerFactory.getLogger(JdbcExactlyOnceSinkE2eTest.class);
 
-    private static final long CHECKPOINT_TIMEOUT_MS = 20_000L;
-    private static final long TASK_CANCELLATION_TIMEOUT_MS = 20_000L;
-
-    // todo: remove after fixing FLINK-22889

Review Comment:
   I notice 2 more lines like about FLINK-22889 with todo remove after fixing.
   Should they also be cleaned?



##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -92,25 +89,14 @@
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkState;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.slf4j.event.Level.TRACE;
 
 /** A simple end-to-end test for {@link JdbcXaSinkFunction}. */
 @RunWith(Parameterized.class)
 public class JdbcExactlyOnceSinkE2eTest extends JdbcTestBase {
-    private static final Random RANDOM = new Random(System.currentTimeMillis());
-
     private static final Logger LOG = LoggerFactory.getLogger(JdbcExactlyOnceSinkE2eTest.class);
 
-    private static final long CHECKPOINT_TIMEOUT_MS = 20_000L;
-    private static final long TASK_CANCELLATION_TIMEOUT_MS = 20_000L;
-
-    // todo: remove after fixing FLINK-22889

Review Comment:
   I noticed 2 more lines like about FLINK-22889 with todo remove after fixing.
   Should they also be cleaned?



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] eskabetxe commented on a diff in pull request #22266: [FLINK-31525][tests] Stabilize JdbcExactlyOnceSinkE2eTest

Posted by "eskabetxe (via GitHub)" <gi...@apache.org>.
eskabetxe commented on code in PR #22266:
URL: https://github.com/apache/flink/pull/22266#discussion_r1151668036


##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -92,25 +89,14 @@
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkState;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.slf4j.event.Level.TRACE;
 
 /** A simple end-to-end test for {@link JdbcXaSinkFunction}. */
 @RunWith(Parameterized.class)
 public class JdbcExactlyOnceSinkE2eTest extends JdbcTestBase {
-    private static final Random RANDOM = new Random(System.currentTimeMillis());
-
     private static final Logger LOG = LoggerFactory.getLogger(JdbcExactlyOnceSinkE2eTest.class);
 
-    private static final long CHECKPOINT_TIMEOUT_MS = 20_000L;
-    private static final long TASK_CANCELLATION_TIMEOUT_MS = 20_000L;
-
-    // todo: remove after fixing FLINK-22889

Review Comment:
   yes removed



##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java:
##########
@@ -92,25 +89,14 @@
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkState;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.slf4j.event.Level.TRACE;
 
 /** A simple end-to-end test for {@link JdbcXaSinkFunction}. */
 @RunWith(Parameterized.class)
 public class JdbcExactlyOnceSinkE2eTest extends JdbcTestBase {
-    private static final Random RANDOM = new Random(System.currentTimeMillis());
-
     private static final Logger LOG = LoggerFactory.getLogger(JdbcExactlyOnceSinkE2eTest.class);
 
-    private static final long CHECKPOINT_TIMEOUT_MS = 20_000L;
-    private static final long TASK_CANCELLATION_TIMEOUT_MS = 20_000L;
-
-    // todo: remove after fixing FLINK-22889

Review Comment:
   yes, removed



-- 
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-unsubscribe@flink.apache.org

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