gharris1727 commented on code in PR #12290: URL: https://github.com/apache/kafka/pull/12290#discussion_r1425904694
########## connect/runtime/src/test/java/org/apache/kafka/connect/integration/BlockingConnectorTest.java: ########## @@ -368,31 +374,54 @@ private static ConfigDef config() { ); } + /** + * {@link CountDownLatch#await() Wait} for the connector/task to reach the point in its lifecycle where + * it will block. + */ public static void waitForBlock() throws InterruptedException, TimeoutException { + CountDownLatch awaitBlockLatch; synchronized (Block.class) { - if (blockLatch == null) { - throw new IllegalArgumentException("No connector has been created yet"); - } + awaitBlockLatch = Block.awaitBlockLatch; + } + + if (awaitBlockLatch == null) { + throw new IllegalArgumentException("No connector has been created yet"); } log.debug("Waiting for connector to block"); - if (!blockLatch.await(CONNECTOR_BLOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { + if (!awaitBlockLatch.await(CONNECTOR_BLOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { throw new TimeoutException("Timed out waiting for connector to block."); Review Comment: If you add this stanza before the `log.debug("Connector should now be blocked")` the tests still pass: ``` boolean retry; synchronized (Block.class) { retry = Block.awaitBlockLatch != null && Block.awaitBlockLatch != awaitBlockLatch; } if (retry) { log.debug("New blocking instance was created, retrying wait"); waitForBlock(); } ``` For me, I see this being printed in: * testBlockInSinkTaskStart * testBlockInConnectorStart * testWorkerRestartWithBlockInConnectorStart * testBlockInSourceTaskStart * testBlockInConnectorInitialize This leads me to believe that this function is normally exiting before the blocking method of the last-instantiated instance happens. I don't immediately see how this could cause flakiness, but it's at least an instance of the method not doing what it says it does. -- 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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org