[ 
https://issues.apache.org/jira/browse/FLINK-12840?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Pavel Potseluev updated FLINK-12840:
------------------------------------
    Summary: Problems with ipv6 addresses  (was: Problems with ipv6 adresses)

> Problems with ipv6 addresses
> ----------------------------
>
>                 Key: FLINK-12840
>                 URL: https://issues.apache.org/jira/browse/FLINK-12840
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Configuration, Runtime / Network
>    Affects Versions: 1.8.0
>            Reporter: Pavel Potseluev
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> I found some problems when using flink with ipv6. The easiest way to 
> reproduce is add 
> {noformat}
> env.java.opts: -Djava.net.preferIPv6Addresses=true{noformat}
> to the flink-conf.yaml and run start-cluster.sh script.
>  Jobmanager log:
> {noformat}
> 2019-06-13 19:16:01,088 INFO  
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Trying to 
> start actor system at localhost:6123
> 2019-06-13 19:16:01,147 INFO  
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint         - Shutting 
> StandaloneSessionClusterEntrypoint down with application status FAILED. 
> Diagnostics java.lang.Exception: Could not create actor system
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:267)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:153)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:112)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:87)
> at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils.createRpcService(AkkaRpcServiceUtils.java:84)
> at 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.createRpcService(ClusterEntrypoint.java:296)
> at 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:264)
> at 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:216)
> at 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$0(ClusterEntrypoint.java:172)
> at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> at 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:171)
> at 
> org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:535)
> at 
> org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint.main(StandaloneSessionClusterEntrypoint.java:65)
> Caused by: com.typesafe.config.ConfigException$Parse: String: 56: Expecting 
> close brace } or a comma, got ':' (if you intended ':' to be part of a key or 
> string value, try enclosing the key or value in double quotes, or you may be 
> able to rename the file .properties rather than .conf)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseError(ConfigDocumentParser.java:201)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseError(ConfigDocumentParser.java:197)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:475)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parse(ConfigDocumentParser.java:595)
> at 
> com.typesafe.config.impl.ConfigDocumentParser.parse(ConfigDocumentParser.java:14)
> at com.typesafe.config.impl.Parseable.rawParseValue(Parseable.java:260)
> at com.typesafe.config.impl.Parseable.rawParseValue(Parseable.java:248)
> at com.typesafe.config.impl.Parseable.parseValue(Parseable.java:180)
> at com.typesafe.config.impl.Parseable.parseValue(Parseable.java:174)
> at com.typesafe.config.impl.Parseable.parse(Parseable.java:299)
> at com.typesafe.config.ConfigFactory.parseString(ConfigFactory.java:1046)
> at com.typesafe.config.ConfigFactory.parseString(ConfigFactory.java:1056)
> at 
> org.apache.flink.runtime.akka.AkkaUtils$.getRemoteAkkaConfig(AkkaUtils.scala:601)
> at org.apache.flink.runtime.akka.AkkaUtils$.getAkkaConfig(AkkaUtils.scala:218)
> at org.apache.flink.runtime.akka.AkkaUtils.getAkkaConfig(AkkaUtils.scala)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:247)
> ... 12 more{noformat}
> Taskmanager log:
> {noformat}
> 2019-06-13 19:16:12,260 INFO  
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Trying to 
> start actor system at [2a02:6b8:c02:7e8:0:1459:44c2:764a]:0
> 2019-06-13 19:16:12,310 ERROR 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner       - TaskManager 
> initialization failed.
> java.lang.Exception: Could not create actor system
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:267)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:153)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:112)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:87)
> at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils.createRpcService(AkkaRpcServiceUtils.java:84)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.createRpcService(TaskManagerRunner.java:412)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.<init>(TaskManagerRunner.java:134)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.runTaskManager(TaskManagerRunner.java:332)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner$1.call(TaskManagerRunner.java:302)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner$1.call(TaskManagerRunner.java:299)
> at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.main(TaskManagerRunner.java:299)
> Caused by: com.typesafe.config.ConfigException$Parse: String: 55: List should 
> have ended with ] or had a comma, instead had token: ':' (if you want ':' to 
> be part of a string value, then double-quote it)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseError(ConfigDocumentParser.java:201)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseError(ConfigDocumentParser.java:197)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseArray(ConfigDocumentParser.java:533)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:249)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.consolidateValues(ConfigDocumentParser.java:152)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:420)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseValue(ConfigDocumentParser.java:247)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parseObject(ConfigDocumentParser.java:405)
> at 
> com.typesafe.config.impl.ConfigDocumentParser$ParseContext.parse(ConfigDocumentParser.java:595)
> at 
> com.typesafe.config.impl.ConfigDocumentParser.parse(ConfigDocumentParser.java:14)
> at com.typesafe.config.impl.Parseable.rawParseValue(Parseable.java:260)
> at com.typesafe.config.impl.Parseable.rawParseValue(Parseable.java:248)
> at com.typesafe.config.impl.Parseable.parseValue(Parseable.java:180)
> at com.typesafe.config.impl.Parseable.parseValue(Parseable.java:174)
> at com.typesafe.config.impl.Parseable.parse(Parseable.java:299)
> at com.typesafe.config.ConfigFactory.parseString(ConfigFactory.java:1046)
> at com.typesafe.config.ConfigFactory.parseString(ConfigFactory.java:1056)
> at 
> org.apache.flink.runtime.akka.AkkaUtils$.getRemoteAkkaConfig(AkkaUtils.scala:601)
> at org.apache.flink.runtime.akka.AkkaUtils$.getAkkaConfig(AkkaUtils.scala:218)
> at org.apache.flink.runtime.akka.AkkaUtils.getAkkaConfig(AkkaUtils.scala)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:247)
> ... 11 more{noformat}
> The problem is the following code in the AkkaUtils:
>  
> {code:java}
> val hostnameConfigString =
>   s"""
>      |akka {
>      |  remote {
>      |    netty {
>      |      tcp {
>      |        hostname = $effectiveHostname
>      |        bind-hostname = $bindAddress
>      |      }
>      |    }
>      |  }
>      |}
>    """.stripMargin
> {code}
> I fixed it by adding double quotes. I ran flink with this fix and jobmanager 
> worked fine but there was a new problem with taskmanager:
> {noformat}
> 2019-06-13 21:12:37,451 INFO  
> org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Actor system 
> started at akka.tcp://flink@[2a02:6b8:c02:7e8:0:1459:44c2:764a]:14803
> 2019-06-13 21:12:37,470 ERROR 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner       - TaskManager 
> initialization failed.
> org.apache.flink.configuration.IllegalConfigurationException: The configured 
> hostname is not valid
> at 
> org.apache.flink.util.NetUtils.unresolvedHostToNormalizedString(NetUtils.java:150)
> at 
> org.apache.flink.util.NetUtils.unresolvedHostAndPortToNormalizedString(NetUtils.java:168)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:243)
> at 
> org.apache.flink.runtime.clusterframework.BootstrapTools.startActorSystem(BootstrapTools.java:153)
> at 
> org.apache.flink.runtime.metrics.util.MetricUtils.startMetricsActorSystem(MetricUtils.java:132)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.<init>(TaskManagerRunner.java:135)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.runTaskManager(TaskManagerRunner.java:332)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner$1.call(TaskManagerRunner.java:302)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner$1.call(TaskManagerRunner.java:299)
> at 
> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> at 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.main(TaskManagerRunner.java:299)
> Caused by: java.lang.IllegalArgumentException
> at org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:123)
> at 
> org.apache.flink.util.NetUtils.unresolvedHostToNormalizedString(NetUtils.java:148)
> ... 10 more{noformat}
>  
> This problem caused by NetUtils.unresolvedHostToNormalizedString fails when 
> gets an ipv6 address in brackets like [2a02:6b8:c02:7e8:0:1459:44c2:764a]. 
> It works fine with fixed AkkaUtils and NetUtils.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to