[ https://issues.apache.org/jira/browse/FLINK-9156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16435870#comment-16435870 ]
ASF GitHub Bot commented on FLINK-9156: --------------------------------------- Github user GJL commented on a diff in the pull request: https://github.com/apache/flink/pull/5838#discussion_r181140084 --- Diff: flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java --- @@ -677,6 +682,37 @@ public void testGetAccumulators() throws Exception { } } + /** + * Tests that command line options override the configuration settings. + */ + @Test + public void testRESTManualConfigurationOverride() throws Exception { + final String localhost = "localhost"; + final int port = 1234; + final Configuration configuration = new Configuration(); + + configuration.setString(JobManagerOptions.ADDRESS, localhost); + configuration.setInteger(JobManagerOptions.PORT, port); + configuration.setString(RestOptions.REST_ADDRESS, localhost); + configuration.setInteger(RestOptions.REST_PORT, port); + + final DefaultCLI defaultCLI = new DefaultCLI(configuration); + + final String manualHostname = "123.123.123.123"; + final int manualPort = 4321; + final String[] args = {"-m", manualHostname + ':' + manualPort}; + + CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + + final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine); + + final RestClusterClient<?> clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine)); + + URL webMonitorBaseUrl = clusterClient.getWebMonitorBaseUrl().get(); + assertThat(webMonitorBaseUrl.getHost(), Matchers.equalTo(manualHostname)); + assertThat(webMonitorBaseUrl.getPort(), Matchers.equalTo(manualPort)); --- End diff -- There is already a static import for `equalTo` in this file. I think it should be used, i.e., ``` assertThat(webMonitorBaseUrl.getHost(), equalTo(manualHostname)); assertThat(webMonitorBaseUrl.getPort(), equalTo(manualPort)); ``` > CLI does not respect -m,--jobmanager option > ------------------------------------------- > > Key: FLINK-9156 > URL: https://issues.apache.org/jira/browse/FLINK-9156 > Project: Flink > Issue Type: Bug > Components: Client > Affects Versions: 1.5.0 > Environment: 1.5 RC1 > Reporter: Gary Yao > Assignee: Chesnay Schepler > Priority: Blocker > Fix For: 1.5.0 > > > *Description* > The CLI does not respect the {{-m, --jobmanager}} option. For example > submitting a job using > {noformat} > bin/flink run -m 172.31.35.68:6123 [...] > {noformat} > results in the client trying to connect to what is specified in > {{flink-conf.yaml}} ({{jobmanager.rpc.address, jobmanager.rpc.port}}). > *Stacktrace* > {noformat} > ------------------------------------------------------------ > The program finished with the following exception: > org.apache.flink.client.program.ProgramInvocationException: Could not submit > job 99b0a48ec5cb4086740b1ffd38efd1af. > at > org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:244) > at > org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:464) > at > org.apache.flink.client.program.DetachedEnvironment.finalizeExecute(DetachedEnvironment.java:77) > at > org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:410) > at > org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:780) > at > org.apache.flink.client.cli.CliFrontend.runProgram(CliFrontend.java:274) > at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:209) > at > org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:1019) > at > org.apache.flink.client.cli.CliFrontend.lambda$main$9(CliFrontend.java:1095) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836) > at > org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) > at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1095) > Caused by: org.apache.flink.runtime.client.JobSubmissionException: Failed to > submit JobGraph. > at > org.apache.flink.client.program.rest.RestClusterClient.lambda$submitJob$4(RestClusterClient.java:351) > at > java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870) > at > java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852) > at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) > at > java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:561) > at > java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:929) > at > java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.util.concurrent.CompletionException: > org.apache.flink.util.FlinkException: Could not upload job jar files. > at > org.apache.flink.client.program.rest.RestClusterClient.lambda$submitJob$2(RestClusterClient.java:326) > at > java.util.concurrent.CompletableFuture.biApply(CompletableFuture.java:1105) > at > java.util.concurrent.CompletableFuture$BiApply.tryFire(CompletableFuture.java:1070) > ... 7 more > Caused by: org.apache.flink.util.FlinkException: Could not upload job jar > files. > ... 10 more > Caused by: java.io.IOException: Could not connect to BlobServer at address > /127.0.0.1:41909 > at org.apache.flink.runtime.blob.BlobClient.<init>(BlobClient.java:124) > at > org.apache.flink.runtime.blob.BlobClient.uploadJarFiles(BlobClient.java:547) > at > org.apache.flink.client.program.rest.RestClusterClient.lambda$submitJob$2(RestClusterClient.java:324) > ... 9 more > Caused by: java.net.ConnectException: Connection refused (Connection refused) > at java.net.PlainSocketImpl.socketConnect(Native Method) > at > java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350) > at > java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206) > at > java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188) > at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) > at java.net.Socket.connect(Socket.java:589) > at java.net.Socket.connect(Socket.java:538) > at org.apache.flink.runtime.blob.BlobClient.<init>(BlobClient.java:118) > ... 11 more > {noformat} -- This message was sent by Atlassian JIRA (v7.6.3#76005)