[ 
https://issues.apache.org/jira/browse/FLINK-22821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17355547#comment-17355547
 ] 

Yao Zhang commented on FLINK-22821:
-----------------------------------

Hi [~dwysakowicz]:
I reproduced this issue by creating two TestingServer with the same port, like:

{code:java}
TestingServer server = new TestingServer(2181, new File("/path/to/zk/"));
TestingServer server2 = new TestingServer(2181, new File("/path/to/zk/"));
{code}

Run this and it will result in those following errors:

{noformat}
java.net.BindException: Address already in use: bind
        at sun.nio.ch.Net.bind0(Native Method)
        at sun.nio.ch.Net.bind(Net.java:461)
        at sun.nio.ch.Net.bind(Net.java:453)
        at 
sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:222)
        at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:85)
        at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:78)
        at 
org.apache.zookeeper.server.NIOServerCnxnFactory.configure(NIOServerCnxnFactory.java:90)
        at 
org.apache.zookeeper.server.ZooKeeperServerMain.runFromConfig(ZooKeeperServerMain.java:111)
        at 
org.apache.curator.test.TestingZooKeeperMain.runFromConfig(TestingZooKeeperMain.java:93)
        at 
org.apache.curator.test.TestingZooKeeperServer$1.run(TestingZooKeeperServer.java:148)
        at java.lang.Thread.run(Thread.java:748)
{noformat}

According to this, I infer that the issue is caused by running test cases which 
create TestingServer concurrently. I reviewed the relevant codes in 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java:

{code:java}
@Override
public void prepare(Config config) throws Exception {
// ...
    zookeeper = new TestingServer(-1, tmpZkDir);
// ...
}
{code}
Though we create the TestingServer with the random port by passing -1 as the 
1st param, I suspect it is not thread safe. And I plan to add a lock around 
this step.

Could you pls assign this to me and let me have a try?


> FlinkKafkaProducerMigrationTest fails with "Address already in use"
> -------------------------------------------------------------------
>
>                 Key: FLINK-22821
>                 URL: https://issues.apache.org/jira/browse/FLINK-22821
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Kafka
>    Affects Versions: 1.14.0
>            Reporter: Dawid Wysakowicz
>            Priority: Critical
>              Labels: test-stability
>             Fix For: 1.14.0
>
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=18469&view=logs&j=c5f0071e-1851-543e-9a45-9ac140befc32&t=1fb1a56f-e8b5-5a82-00a0-a2db7757b4f5&l=6832
> {code}
> Jun 01 01:27:33 java.net.BindException: Address already in use
> Jun 01 01:27:33       at sun.nio.ch.Net.bind0(Native Method)
> Jun 01 01:27:33       at sun.nio.ch.Net.bind(Net.java:461)
> Jun 01 01:27:33       at sun.nio.ch.Net.bind(Net.java:453)
> Jun 01 01:27:33       at 
> sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:222)
> Jun 01 01:27:33       at 
> sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:85)
> Jun 01 01:27:33       at 
> sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:78)
> Jun 01 01:27:33       at 
> org.apache.zookeeper.server.NIOServerCnxnFactory.configure(NIOServerCnxnFactory.java:90)
> Jun 01 01:27:33       at 
> org.apache.zookeeper.server.ZooKeeperServerMain.runFromConfig(ZooKeeperServerMain.java:120)
> Jun 01 01:27:33       at 
> org.apache.curator.test.TestingZooKeeperMain.runFromConfig(TestingZooKeeperMain.java:93)
> Jun 01 01:27:33       at 
> org.apache.curator.test.TestingZooKeeperServer$1.run(TestingZooKeeperServer.java:148)
> Jun 01 01:27:33       at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to