You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2018/01/12 22:25:22 UTC
[01/14] flink git commit: [FLINK-8341] [flip6] Remove not needed
options from CommandLineOptions
Repository: flink
Updated Branches:
refs/heads/master 4ade82631 -> 3b0168685
[FLINK-8341] [flip6] Remove not needed options from CommandLineOptions
This closes #5227.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d7e9dc19
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d7e9dc19
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d7e9dc19
Branch: refs/heads/master
Commit: d7e9dc1931f1f1cedbfee12aebe34dd76e9bac10
Parents: 30011b9
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Dec 20 16:32:18 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:02 2018 +0100
----------------------------------------------------------------------
.../flink/client/cli/CommandLineOptions.java | 16 -----------
.../org/apache/flink/api/scala/FlinkShell.scala | 28 ++++++++++++--------
2 files changed, 17 insertions(+), 27 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d7e9dc19/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
index a9a29b2..1b57620 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
@@ -20,7 +20,6 @@ package org.apache.flink.client.cli;
import org.apache.commons.cli.CommandLine;
-import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION;
/**
@@ -29,28 +28,13 @@ import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION;
*/
public abstract class CommandLineOptions {
- private final CommandLine commandLine;
-
- private final String jobManagerAddress;
-
private final boolean printHelp;
protected CommandLineOptions(CommandLine line) {
- this.commandLine = line;
this.printHelp = line.hasOption(HELP_OPTION.getOpt());
- this.jobManagerAddress = line.hasOption(ADDRESS_OPTION.getOpt()) ?
- line.getOptionValue(ADDRESS_OPTION.getOpt()) : null;
- }
-
- public CommandLine getCommandLine() {
- return commandLine;
}
public boolean isPrintHelp() {
return printHelp;
}
-
- public String getJobManagerAddress() {
- return jobManagerAddress;
- }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d7e9dc19/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index 9f29ce0..349d940 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala
import java.io._
-import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser}
+import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser, RunOptions}
import org.apache.flink.client.program.ClusterClient
import org.apache.flink.configuration.{Configuration, GlobalConfiguration, JobManagerOptions}
import org.apache.flink.runtime.minicluster.StandaloneMiniCluster
@@ -253,16 +253,19 @@ object FlinkShell {
yarnConfig.queue.foreach((queue) => args ++= Seq("-yqu", queue.toString))
yarnConfig.slots.foreach((slots) => args ++= Seq("-ys", slots.toString))
- val options = CliFrontendParser.parseRunCommand(args.toArray)
+ val commandLine = CliFrontendParser.parse(
+ CliFrontendParser.getRunCommandOptions,
+ args.toArray,
+ true)
+
val frontend = new CliFrontend(
configuration,
CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
- val config = frontend.getConfiguration
- val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
+ val customCLI = frontend.getActiveCustomCommandLine(commandLine)
- val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
+ val clusterDescriptor = customCLI.createClusterDescriptor(commandLine)
- val clusterSpecification = customCLI.getClusterSpecification(options.getCommandLine)
+ val clusterSpecification = customCLI.getClusterSpecification(commandLine)
val cluster = clusterDescriptor.deploySessionCluster(clusterSpecification)
@@ -281,16 +284,19 @@ object FlinkShell {
"-m", "yarn-cluster"
)
- val options = CliFrontendParser.parseRunCommand(args.toArray)
+ val commandLine = CliFrontendParser.parse(
+ CliFrontendParser.getRunCommandOptions,
+ args.toArray,
+ true)
+
val frontend = new CliFrontend(
configuration,
CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
- val config = frontend.getConfiguration
- val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
+ val customCLI = frontend.getActiveCustomCommandLine(commandLine)
- val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
+ val clusterDescriptor = customCLI.createClusterDescriptor(commandLine)
- val clusterId = customCLI.getClusterId(options.getCommandLine)
+ val clusterId = customCLI.getClusterId(commandLine)
val cluster = clusterDescriptor.retrieve(clusterId)
[10/14] flink git commit: [FLINK-8119] [flip6] Wire correct Flip6
components in Flip6YarnClusterDescriptor
Posted by tr...@apache.org.
[FLINK-8119] [flip6] Wire correct Flip6 components in Flip6YarnClusterDescriptor
Let the Flip6YarnClusterDescriptor create a RestClusterClient as ClusterClient.
Moreover, this commit makes the YarnResourceManager register under the REST port
at Yarn.
This closes #5234.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/dbe0e828
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/dbe0e828
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/dbe0e828
Branch: refs/heads/master
Commit: dbe0e8286d76a5facdb49589b638b87dbde80178
Parents: 7d986ce
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Jan 3 20:38:21 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:05 2018 +0100
----------------------------------------------------------------------
.../client/program/rest/RestClusterClient.java | 12 ++--
.../yarn/TestingYarnClusterDescriptor.java | 2 +-
.../java/org/apache/flink/yarn/YARNITCase.java | 10 +--
.../yarn/AbstractYarnClusterDescriptor.java | 37 ++++++-----
.../flink/yarn/Flip6YarnClusterDescriptor.java | 66 ++++++++++++++++++++
.../flink/yarn/YarnClusterDescriptor.java | 14 +++++
.../flink/yarn/YarnClusterDescriptorV2.java | 49 ---------------
.../apache/flink/yarn/YarnResourceManager.java | 16 ++++-
.../flink/yarn/cli/FlinkYarnSessionCli.java | 17 +++--
.../yarn/entrypoint/YarnEntrypointUtils.java | 7 +++
.../flink/yarn/AbstractYarnClusterTest.java | 6 ++
11 files changed, 150 insertions(+), 86 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
index 3e8b136..4d8656b 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
@@ -318,28 +318,24 @@ public class RestClusterClient<T> extends ClusterClient<T> {
return false;
}
- // ======================================
- // Legacy stuff we ignore
- // ======================================
-
@Override
public void waitForClusterToBeReady() {
- throw new UnsupportedOperationException();
+ // no op
}
@Override
public String getWebInterfaceURL() {
- throw new UnsupportedOperationException();
+ return "http://" + restClusterClientConfiguration.getRestServerAddress() + ':' + restClusterClientConfiguration.getRestServerPort();
}
@Override
public GetClusterStatusResponse getClusterStatus() {
- throw new UnsupportedOperationException();
+ return null;
}
@Override
public List<String> getNewMessages() {
- throw new UnsupportedOperationException();
+ return Collections.emptyList();
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
index e66d2e0..95a31be 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
@@ -35,7 +35,7 @@ import java.util.List;
* flink-yarn-tests-X-tests.jar and the flink-runtime-X-tests.jar to the set of files which
* are shipped to the yarn cluster. This is necessary to load the testing classes.
*/
-public class TestingYarnClusterDescriptor extends AbstractYarnClusterDescriptor {
+public class TestingYarnClusterDescriptor extends YarnClusterDescriptor {
public TestingYarnClusterDescriptor(Configuration configuration, String configurationDirectory) {
super(
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
index 069f68a..bea9001 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
@@ -19,6 +19,7 @@
package org.apache.flink.yarn;
import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.AkkaOptions;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
@@ -28,6 +29,7 @@ import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.junit.BeforeClass;
import org.junit.Ignore;
@@ -55,13 +57,13 @@ public class YARNITCase extends YarnTestBase {
configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s");
final YarnClient yarnClient = YarnClient.createYarnClient();
- try (final YarnClusterDescriptorV2 yarnClusterDescriptorV2 = new YarnClusterDescriptorV2(
+ try (final Flip6YarnClusterDescriptor flip6YarnClusterDescriptor = new Flip6YarnClusterDescriptor(
configuration,
System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR),
yarnClient)) {
- yarnClusterDescriptorV2.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath()));
- yarnClusterDescriptorV2.addShipFiles(Arrays.asList(flinkLibFolder.listFiles()));
+ flip6YarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath()));
+ flip6YarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles()));
final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
.setMasterMemoryMB(768)
@@ -83,7 +85,7 @@ public class YARNITCase extends YarnTestBase {
jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI()));
- YarnClusterClient clusterClient = yarnClusterDescriptorV2.deployJobCluster(clusterSpecification, jobGraph);
+ ClusterClient<ApplicationId> clusterClient = flip6YarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph);
clusterClient.shutdown();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index a7e8c36..f015235 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterDeploymentException;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterRetrieveException;
import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
@@ -30,6 +31,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.ResourceManagerOptions;
+import org.apache.flink.configuration.RestOptions;
import org.apache.flink.configuration.SecurityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.clusterframework.BootstrapTools;
@@ -332,7 +334,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
// -------------------------------------------------------------
@Override
- public YarnClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException {
+ public ClusterClient<ApplicationId> retrieve(ApplicationId applicationId) throws ClusterRetrieveException {
try {
// check if required Hadoop environment variables are set. If not, warn user
@@ -352,11 +354,17 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore.");
}
+ final String host = appReport.getHost();
+ final int rpcPort = appReport.getRpcPort();
+
LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'",
- appReport.getHost(), appReport.getRpcPort(), applicationId);
+ host, rpcPort, applicationId);
+
+ flinkConfiguration.setString(JobManagerOptions.ADDRESS, host);
+ flinkConfiguration.setInteger(JobManagerOptions.PORT, rpcPort);
- flinkConfiguration.setString(JobManagerOptions.ADDRESS, appReport.getHost());
- flinkConfiguration.setInteger(JobManagerOptions.PORT, appReport.getRpcPort());
+ flinkConfiguration.setString(RestOptions.REST_ADDRESS, host);
+ flinkConfiguration.setInteger(RestOptions.REST_PORT, rpcPort);
return createYarnClusterClient(
this,
@@ -371,7 +379,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
}
@Override
- public YarnClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
+ public ClusterClient<ApplicationId> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
try {
return deployInternal(
clusterSpecification,
@@ -383,7 +391,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
}
@Override
- public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException {
+ public ClusterClient<ApplicationId> deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException {
try {
return deployInternal(
clusterSpecification,
@@ -410,7 +418,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
* @param clusterSpecification Initial cluster specification for the to be deployed Flink cluster
* @param jobGraph A job graph which is deployed with the Flink cluster, null if none
*/
- protected YarnClusterClient deployInternal(
+ protected ClusterClient<ApplicationId> deployInternal(
ClusterSpecification clusterSpecification,
String yarnClusterEntrypoint,
@Nullable JobGraph jobGraph) throws Exception {
@@ -488,6 +496,9 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
flinkConfiguration.setString(JobManagerOptions.ADDRESS, host);
flinkConfiguration.setInteger(JobManagerOptions.PORT, port);
+ flinkConfiguration.setString(RestOptions.REST_ADDRESS, host);
+ flinkConfiguration.setInteger(RestOptions.REST_PORT, port);
+
// the Flink cluster is deployed in YARN. Represent cluster
return createYarnClusterClient(
this,
@@ -1534,20 +1545,12 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
/**
* Creates a YarnClusterClient; may be overriden in tests.
*/
- protected YarnClusterClient createYarnClusterClient(
+ protected abstract ClusterClient<ApplicationId> createYarnClusterClient(
AbstractYarnClusterDescriptor descriptor,
int numberTaskManagers,
int slotsPerTaskManager,
ApplicationReport report,
org.apache.flink.configuration.Configuration flinkConfiguration,
- boolean perJobCluster) throws Exception {
- return new YarnClusterClient(
- descriptor,
- numberTaskManagers,
- slotsPerTaskManager,
- report,
- flinkConfiguration,
- perJobCluster);
- }
+ boolean perJobCluster) throws Exception;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java
new file mode 100644
index 0000000..e3f9646
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.yarn;
+
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint;
+import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+
+/**
+ * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the
+ * new application master for a job under flip-6.
+ */
+public class Flip6YarnClusterDescriptor extends AbstractYarnClusterDescriptor {
+
+ public Flip6YarnClusterDescriptor(
+ Configuration flinkConfiguration,
+ String configurationDirectory,
+ YarnClient yarnCLient) {
+ super(flinkConfiguration, configurationDirectory, yarnCLient);
+ }
+
+ @Override
+ protected String getYarnSessionClusterEntrypoint() {
+ return YarnSessionClusterEntrypoint.class.getName();
+ }
+
+ @Override
+ protected String getYarnJobClusterEntrypoint() {
+ return YarnJobClusterEntrypoint.class.getName();
+ }
+
+ @Override
+ protected ClusterClient<ApplicationId> createYarnClusterClient(
+ AbstractYarnClusterDescriptor descriptor,
+ int numberTaskManagers,
+ int slotsPerTaskManager,
+ ApplicationReport report,
+ Configuration flinkConfiguration,
+ boolean perJobCluster) throws Exception {
+ return new RestClusterClient<>(
+ flinkConfiguration,
+ report.getApplicationId());
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
index 76f9154..9467326 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
@@ -19,9 +19,12 @@
package org.apache.flink.yarn;
import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.client.api.YarnClient;
/**
@@ -50,4 +53,15 @@ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor {
public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
throw new UnsupportedOperationException("Cannot deploy a per-job yarn cluster yet.");
}
+
+ @Override
+ protected ClusterClient<ApplicationId> createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception {
+ return new YarnClusterClient(
+ descriptor,
+ numberTaskManagers,
+ slotsPerTaskManager,
+ report,
+ flinkConfiguration,
+ perJobCluster);
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
deleted file mode 100644
index 6ce192c..0000000
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.yarn;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint;
-import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint;
-
-import org.apache.hadoop.yarn.client.api.YarnClient;
-
-/**
- * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the
- * new application master for a job under flip-6.
- */
-public class YarnClusterDescriptorV2 extends AbstractYarnClusterDescriptor {
-
- public YarnClusterDescriptorV2(
- Configuration flinkConfiguration,
- String configurationDirectory,
- YarnClient yarnCLient) {
- super(flinkConfiguration, configurationDirectory, yarnCLient);
- }
-
- @Override
- protected String getYarnSessionClusterEntrypoint() {
- return YarnSessionClusterEntrypoint.class.getName();
- }
-
- @Override
- protected String getYarnJobClusterEntrypoint() {
- return YarnJobClusterEntrypoint.class.getName();
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
index 5a71f41..0fa0dda 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
@@ -177,7 +177,21 @@ public class YarnResourceManager extends ResourceManager<YarnWorkerNode> impleme
//TODO: change akka address to tcp host and port, the getAddress() interface should return a standard tcp address
Tuple2<String, Integer> hostPort = parseHostPort(getAddress());
- resourceManagerClient.registerApplicationMaster(hostPort.f0, hostPort.f1, webInterfaceUrl);
+ final int restPort;
+
+ if (webInterfaceUrl != null) {
+ final int lastColon = webInterfaceUrl.lastIndexOf(':');
+
+ if (lastColon == -1) {
+ restPort = -1;
+ } else {
+ restPort = Integer.valueOf(webInterfaceUrl.substring(lastColon + 1));
+ }
+ } else {
+ restPort = -1;
+ }
+
+ resourceManagerClient.registerApplicationMaster(hostPort.f0, restPort, webInterfaceUrl);
return resourceManagerClient;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index fea8a5c..d4cf9ad 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -38,8 +38,8 @@ import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
+import org.apache.flink.yarn.Flip6YarnClusterDescriptor;
import org.apache.flink.yarn.YarnClusterDescriptor;
-import org.apache.flink.yarn.YarnClusterDescriptorV2;
import org.apache.flink.yarn.configuration.YarnConfigOptions;
import org.apache.commons.cli.CommandLine;
@@ -522,7 +522,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
for (Option option : commandLine.getOptions()) {
if (allOptions.hasOption(option.getOpt())) {
- if (!option.getOpt().equals(detached.getOpt())) {
+ if (!option.getOpt().equals(detached.getOpt()) && !option.getOpt().equals(flip6.getOpt())) {
// don't resume from properties file if yarn options have been specified
canApplyYarnProperties = false;
break;
@@ -653,14 +653,19 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
deleteYarnPropertiesFile();
+ ApplicationReport applicationReport;
+
try {
- final ApplicationReport applicationReport = yarnClusterDescriptor
+ applicationReport = yarnClusterDescriptor
.getYarnClient()
.getApplicationReport(yarnApplicationId);
-
- logFinalApplicationReport(applicationReport);
} catch (YarnException | IOException e) {
LOG.info("Could not log the final application report.", e);
+ applicationReport = null;
+ }
+
+ if (applicationReport != null) {
+ logFinalApplicationReport(applicationReport);
}
}
}
@@ -950,7 +955,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
private static AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) {
final YarnClient yarnClient = YarnClient.createYarnClient();
if (flip6) {
- return new YarnClusterDescriptorV2(configuration, configurationDirectory, yarnClient);
+ return new Flip6YarnClusterDescriptor(configuration, configurationDirectory, yarnClient);
} else {
return new YarnClusterDescriptor(configuration, configurationDirectory, yarnClient);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
index 256056c..a03ed08 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.ResourceManagerOptions;
+import org.apache.flink.configuration.RestOptions;
import org.apache.flink.configuration.SecurityOptions;
import org.apache.flink.configuration.WebOptions;
import org.apache.flink.runtime.clusterframework.BootstrapTools;
@@ -93,6 +94,7 @@ public class YarnEntrypointUtils {
ApplicationConstants.Environment.NM_HOST.key());
configuration.setString(JobManagerOptions.ADDRESS, hostname);
+ configuration.setString(RestOptions.REST_ADDRESS, hostname);
// TODO: Support port ranges for the AM
// final String portRange = configuration.getString(
@@ -112,6 +114,11 @@ public class YarnEntrypointUtils {
configuration.setInteger(WebOptions.PORT, 0);
}
+ if (configuration.getInteger(RestOptions.REST_PORT) >= 0) {
+ // set the REST port to 0 to select it randomly
+ configuration.setInteger(RestOptions.REST_PORT, 0);
+ }
+
// if the user has set the deprecated YARN-specific config keys, we add the
// corresponding generic config keys instead. that way, later code needs not
// deal with deprecated config keys
http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
index 4e68612..0859f03 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.yarn;
import org.apache.flink.client.deployment.ClusterRetrieveException;
+import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TestLogger;
@@ -131,5 +132,10 @@ public class AbstractYarnClusterTest extends TestLogger {
protected String getYarnJobClusterEntrypoint() {
throw new UnsupportedOperationException("Not needed for testing");
}
+
+ @Override
+ protected ClusterClient<ApplicationId> createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception {
+ throw new UnsupportedOperationException("Not needed for testing");
+ }
}
}
[02/14] flink git commit: [FLINK-8340] [flip6] Remove passing of
Configuration to CustomCommandLine
Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index 1b0f29a..d4ab41f 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -25,7 +25,6 @@ import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.runtime.clusterframework.ApplicationStatus;
@@ -34,6 +33,8 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
import org.apache.flink.runtime.security.SecurityConfiguration;
import org.apache.flink.runtime.security.SecurityUtils;
import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
import org.apache.flink.yarn.YarnClusterClient;
import org.apache.flink.yarn.YarnClusterDescriptor;
@@ -53,6 +54,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nullable;
+
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
@@ -70,7 +73,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
-import java.util.concurrent.Callable;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@@ -140,17 +142,36 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
private final boolean acceptInteractiveInput;
+ private final String configurationDirectory;
+
+ private final Properties yarnPropertiesFile;
+
+ private final String yarnApplicationIdFromYarnProperties;
+
//------------------------------------ Internal fields -------------------------
private YarnClusterClient yarnCluster;
private boolean detachedMode = false;
- public FlinkYarnSessionCli(String shortPrefix, String longPrefix) {
- this(shortPrefix, longPrefix, true);
+ public FlinkYarnSessionCli(
+ Configuration configuration,
+ String configurationDirectory,
+ String shortPrefix,
+ String longPrefix) throws FlinkException {
+ this(configuration, configurationDirectory, shortPrefix, longPrefix, true);
}
- public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) {
+ public FlinkYarnSessionCli(
+ Configuration configuration,
+ String configurationDirectory,
+ String shortPrefix,
+ String longPrefix,
+ boolean acceptInteractiveInput) throws FlinkException {
+ super(configuration);
+ this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
this.acceptInteractiveInput = acceptInteractiveInput;
+ // Create the command line options
+
query = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)");
applicationId = new Option(shortPrefix + "id", longPrefix + "applicationId", true, "Attach to running YARN session");
queue = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
@@ -183,93 +204,43 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
allOptions.addOption(applicationId);
allOptions.addOption(zookeeperNamespace);
allOptions.addOption(flip6);
- }
- /**
- * Tries to load a Flink Yarn properties file and returns the Yarn application id if successful.
- * @param cmdLine The command-line parameters
- * @param flinkConfiguration The flink configuration
- * @return Yarn application id or null if none could be retrieved
- */
- private String loadYarnPropertiesFile(CommandLine cmdLine, Configuration flinkConfiguration) {
-
- String jobManagerOption = cmdLine.getOptionValue(addressOption.getOpt(), null);
- if (jobManagerOption != null) {
- // don't resume from properties file if a JobManager has been specified
- return null;
- }
-
- for (Option option : cmdLine.getOptions()) {
- if (allOptions.hasOption(option.getOpt())) {
- if (!option.getOpt().equals(detached.getOpt())) {
- // don't resume from properties file if yarn options have been specified
- return null;
- }
- }
- }
+ // try loading a potential yarn properties file
+ final File yarnPropertiesLocation = getYarnPropertiesLocation(configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
- // load the YARN properties
- File propertiesFile = getYarnPropertiesLocation(flinkConfiguration);
- if (!propertiesFile.exists()) {
- return null;
- }
+ yarnPropertiesFile = new Properties();
- logAndSysout("Found YARN properties file " + propertiesFile.getAbsolutePath());
+ if (yarnPropertiesLocation.exists()) {
+ LOG.info("Found Yarn properties file under {}.", yarnPropertiesLocation.getAbsolutePath());
- Properties yarnProperties = new Properties();
- try {
- try (InputStream is = new FileInputStream(propertiesFile)) {
- yarnProperties.load(is);
+ try (InputStream is = new FileInputStream(yarnPropertiesLocation)) {
+ yarnPropertiesFile.load(is);
+ } catch (IOException ioe) {
+ throw new FlinkException("Could not read the Yarn properties file " + yarnPropertiesLocation + '.');
}
- }
- catch (IOException e) {
- throw new RuntimeException("Cannot read the YARN properties file", e);
- }
- // get the Yarn application id from the properties file
- String applicationID = yarnProperties.getProperty(YARN_APPLICATION_ID_KEY);
- if (applicationID == null) {
- throw new IllegalConfigurationException("Yarn properties file found but doesn't contain a " +
- "Yarn application id. Please delete the file at " + propertiesFile.getAbsolutePath());
- }
-
- try {
- // try converting id to ApplicationId
- ConverterUtils.toApplicationId(applicationID);
- }
- catch (Exception e) {
- throw new RuntimeException("YARN properties contains an invalid entry for " +
- "application id: " + applicationID, e);
- }
+ yarnApplicationIdFromYarnProperties = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
- logAndSysout("Using Yarn application id from YARN properties " + applicationID);
+ if (yarnApplicationIdFromYarnProperties == null) {
+ throw new FlinkException("Yarn properties file found but doesn't contain a " +
+ "Yarn application id. Please delete the file at " + yarnPropertiesLocation.getAbsolutePath());
+ }
- // configure the default parallelism from YARN
- String propParallelism = yarnProperties.getProperty(YARN_PROPERTIES_PARALLELISM);
- if (propParallelism != null) { // maybe the property is not set
try {
- int parallelism = Integer.parseInt(propParallelism);
- flinkConfiguration.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism);
-
- logAndSysout("YARN properties set default parallelism to " + parallelism);
+ // try converting id to ApplicationId
+ ConverterUtils.toApplicationId(yarnApplicationIdFromYarnProperties);
}
- catch (NumberFormatException e) {
- throw new RuntimeException("Error while parsing the YARN properties: " +
- "Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.");
+ catch (Exception e) {
+ throw new FlinkException("YARN properties contains an invalid entry for " +
+ "application id: " + yarnApplicationIdFromYarnProperties + ". Please delete the file at " +
+ yarnPropertiesLocation.getAbsolutePath(), e);
}
+ } else {
+ yarnApplicationIdFromYarnProperties = null;
}
-
- // handle the YARN client's dynamic properties
- String dynamicPropertiesEncoded = yarnProperties.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING);
- Map<String, String> dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded);
- for (Map.Entry<String, String> dynamicProperty : dynamicProperties.entrySet()) {
- flinkConfiguration.setString(dynamicProperty.getKey(), dynamicProperty.getValue());
- }
-
- return applicationID;
}
- public AbstractYarnClusterDescriptor createDescriptor(
+ private AbstractYarnClusterDescriptor createDescriptor(
Configuration configuration,
String configurationDirectory,
String defaultApplicationName,
@@ -364,7 +335,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
return yarnClusterDescriptor;
}
- public ClusterSpecification createClusterSpecification(Configuration configuration, CommandLine cmd) {
+ private ClusterSpecification createClusterSpecification(Configuration configuration, CommandLine cmd) {
if (!cmd.hasOption(container.getOpt())) { // number of containers is required option!
LOG.error("Missing required argument {}", container.getOpt());
printUsage();
@@ -374,27 +345,12 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
int numberTaskManagers = Integer.valueOf(cmd.getOptionValue(container.getOpt()));
// JobManager Memory
- final int jobManagerMemoryMB;
- if (cmd.hasOption(jmMemory.getOpt())) {
- jobManagerMemoryMB = Integer.valueOf(cmd.getOptionValue(this.jmMemory.getOpt()));
- } else {
- jobManagerMemoryMB = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY);
- }
+ final int jobManagerMemoryMB = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY);
// Task Managers memory
- final int taskManagerMemoryMB;
- if (cmd.hasOption(tmMemory.getOpt())) {
- taskManagerMemoryMB = Integer.valueOf(cmd.getOptionValue(this.tmMemory.getOpt()));
- } else {
- taskManagerMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY);
- }
+ final int taskManagerMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY);
- int slotsPerTaskManager;
- if (cmd.hasOption(slots.getOpt())) {
- slotsPerTaskManager = Integer.valueOf(cmd.getOptionValue(this.slots.getOpt()));
- } else {
- slotsPerTaskManager = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
- }
+ int slotsPerTaskManager = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
// convenience
int userParallelism = Integer.valueOf(cmd.getOptionValue(CliFrontendParser.PARALLELISM_OPTION.getOpt(), "-1"));
@@ -435,11 +391,11 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
}
@Override
- public boolean isActive(CommandLine commandLine, Configuration configuration) {
+ public boolean isActive(CommandLine commandLine) {
String jobManagerOption = commandLine.getOptionValue(addressOption.getOpt(), null);
boolean yarnJobManager = ID.equals(jobManagerOption);
boolean yarnAppId = commandLine.hasOption(applicationId.getOpt());
- return yarnJobManager || yarnAppId || loadYarnPropertiesFile(commandLine, configuration) != null;
+ return yarnJobManager || yarnAppId || (isYarnPropertiesFileMode(commandLine) && yarnApplicationIdFromYarnProperties != null);
}
@Override
@@ -463,11 +419,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
}
@Override
- public AbstractYarnClusterDescriptor createClusterDescriptor(
- Configuration configuration,
- String configurationDirectory,
- CommandLine commandLine) {
- final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration, commandLine);
+ public AbstractYarnClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException {
+ final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
return createDescriptor(
effectiveConfiguration,
@@ -477,17 +430,26 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
}
@Override
- public String getClusterId(Configuration configuration, CommandLine commandLine) {
- return commandLine.hasOption(applicationId.getOpt()) ? commandLine.getOptionValue(applicationId.getOpt()) : loadYarnPropertiesFile(commandLine, configuration);
+ @Nullable
+ public String getClusterId(CommandLine commandLine) {
+ if (commandLine.hasOption(applicationId.getOpt())) {
+ return commandLine.getOptionValue(applicationId.getOpt());
+ } else if (isYarnPropertiesFileMode(commandLine)) {
+ return yarnApplicationIdFromYarnProperties;
+ } else {
+ return null;
+ }
}
@Override
- public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
- return createClusterSpecification(configuration, commandLine);
+ public ClusterSpecification getClusterSpecification(CommandLine commandLine) throws FlinkException {
+ final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
+
+ return createClusterSpecification(effectiveConfiguration, commandLine);
}
@Override
- protected Configuration applyCommandLineOptionsToConfiguration(Configuration configuration, CommandLine commandLine) {
+ protected Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException {
// we ignore the addressOption because it can only contain "yarn-cluster"
final Configuration effectiveConfiguration = new Configuration(configuration);
@@ -496,7 +458,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
effectiveConfiguration.setString(HA_CLUSTER_ID, zkNamespace);
}
- final String applicationId = getClusterId(configuration, commandLine);
+ final String applicationId = getClusterId(commandLine);
if (applicationId != null) {
final String zooKeeperNamespace;
@@ -509,6 +471,66 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
effectiveConfiguration.setString(HA_CLUSTER_ID, zooKeeperNamespace);
}
+ if (commandLine.hasOption(jmMemory.getOpt())) {
+ effectiveConfiguration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, Integer.parseInt(commandLine.getOptionValue(jmMemory.getOpt())));
+ }
+
+ if (commandLine.hasOption(tmMemory.getOpt())) {
+ effectiveConfiguration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, Integer.parseInt(commandLine.getOptionValue(tmMemory.getOpt())));
+ }
+
+ if (commandLine.hasOption(slots.getOpt())) {
+ effectiveConfiguration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, Integer.parseInt(commandLine.getOptionValue(slots.getOpt())));
+ }
+
+ if (isYarnPropertiesFileMode(commandLine)) {
+ return applyYarnProperties(effectiveConfiguration);
+ } else {
+ return effectiveConfiguration;
+ }
+ }
+
+ private boolean isYarnPropertiesFileMode(CommandLine commandLine) {
+ boolean canApplyYarnProperties = !commandLine.hasOption(addressOption.getOpt());
+
+ for (Option option : commandLine.getOptions()) {
+ if (allOptions.hasOption(option.getOpt())) {
+ if (!option.getOpt().equals(detached.getOpt())) {
+ // don't resume from properties file if yarn options have been specified
+ canApplyYarnProperties = false;
+ break;
+ }
+ }
+ }
+
+ return canApplyYarnProperties;
+ }
+
+ private Configuration applyYarnProperties(Configuration configuration) throws FlinkException {
+ final Configuration effectiveConfiguration = new Configuration(configuration);
+
+ // configure the default parallelism from YARN
+ String propParallelism = yarnPropertiesFile.getProperty(YARN_PROPERTIES_PARALLELISM);
+ if (propParallelism != null) { // maybe the property is not set
+ try {
+ int parallelism = Integer.parseInt(propParallelism);
+ effectiveConfiguration.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism);
+
+ logAndSysout("YARN properties set default parallelism to " + parallelism);
+ }
+ catch (NumberFormatException e) {
+ throw new FlinkException("Error while parsing the YARN properties: " +
+ "Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.", e);
+ }
+ }
+
+ // handle the YARN client's dynamic properties
+ String dynamicPropertiesEncoded = yarnPropertiesFile.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING);
+ Map<String, String> dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded);
+ for (Map.Entry<String, String> dynamicProperty : dynamicProperties.entrySet()) {
+ effectiveConfiguration.setString(dynamicProperty.getKey(), dynamicProperty.getValue());
+ }
+
return effectiveConfiguration;
}
@@ -596,77 +618,82 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
}
} else {
- AbstractYarnClusterDescriptor yarnDescriptor;
- try {
- yarnDescriptor = createDescriptor(configuration, configurationDirectory, null, cmd);
- } catch (Exception e) {
- System.err.println("Error while starting the YARN Client: " + e.getMessage());
- e.printStackTrace(System.err);
- return 1;
- }
+ try (AbstractYarnClusterDescriptor yarnDescriptor = createClusterDescriptor(cmd)){
+ final ClusterSpecification clusterSpecification;
- final ClusterSpecification clusterSpecification = createClusterSpecification(yarnDescriptor.getFlinkConfiguration(), cmd);
+ try {
+ clusterSpecification = getClusterSpecification(cmd);
+ } catch (FlinkException e) {
+ System.err.println("Error while creating the cluster specification: " + e.getMessage());
+ e.printStackTrace();
+ return 1;
+ }
- try {
- yarnCluster = yarnDescriptor.deploySessionCluster(clusterSpecification);
- } catch (Exception e) {
- System.err.println("Error while deploying YARN cluster: " + e.getMessage());
- e.printStackTrace(System.err);
- return 1;
- }
- //------------------ ClusterClient deployed, handle connection details
- String jobManagerAddress =
- yarnCluster.getJobManagerAddress().getAddress().getHostName() +
- ":" + yarnCluster.getJobManagerAddress().getPort();
+ try {
+ yarnCluster = yarnDescriptor.deploySessionCluster(clusterSpecification);
+ } catch (Exception e) {
+ System.err.println("Error while deploying YARN cluster: " + e.getMessage());
+ e.printStackTrace(System.err);
+ return 1;
+ }
+ //------------------ ClusterClient deployed, handle connection details
+ String jobManagerAddress =
+ yarnCluster.getJobManagerAddress().getAddress().getHostName() +
+ ":" + yarnCluster.getJobManagerAddress().getPort();
- System.out.println("Flink JobManager is now running on " + jobManagerAddress);
- System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL());
+ System.out.println("Flink JobManager is now running on " + jobManagerAddress);
+ System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL());
- // file that we write into the conf/ dir containing the jobManager address and the dop.
- File yarnPropertiesFile = getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration());
+ // file that we write into the conf/ dir containing the jobManager address and the dop.
+ File yarnPropertiesFile = getYarnPropertiesLocation(configuration.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
- Properties yarnProps = new Properties();
- yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnCluster.getApplicationId().toString());
- if (clusterSpecification.getSlotsPerTaskManager() != -1) {
- String parallelism =
+ Properties yarnProps = new Properties();
+ yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnCluster.getApplicationId().toString());
+ if (clusterSpecification.getSlotsPerTaskManager() != -1) {
+ String parallelism =
Integer.toString(clusterSpecification.getSlotsPerTaskManager() * clusterSpecification.getNumberTaskManagers());
- yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
- }
- // add dynamic properties
- if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
- yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
+ yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
+ }
+ // add dynamic properties
+ if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
+ yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
yarnDescriptor.getDynamicPropertiesEncoded());
- }
- writeYarnProperties(yarnProps, yarnPropertiesFile);
+ }
+ writeYarnProperties(yarnProps, yarnPropertiesFile);
- //------------------ ClusterClient running, let user control it ------------
+ //------------------ ClusterClient running, let user control it ------------
- if (detachedMode) {
- // print info and quit:
- LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
+ if (detachedMode) {
+ // print info and quit:
+ LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
"yarn application -kill " + yarnCluster.getApplicationId());
- yarnCluster.waitForClusterToBeReady();
- yarnCluster.disconnect();
- } else {
+ yarnCluster.waitForClusterToBeReady();
+ yarnCluster.disconnect();
+ } else {
- ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+ ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
- try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
+ try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
yarnDescriptor.getYarnClient(),
yarnCluster.getApplicationId(),
- new ScheduledExecutorServiceAdapter(scheduledExecutorService))){
- runInteractiveCli(
- yarnCluster,
- yarnApplicationStatusMonitor,
- acceptInteractiveInput);
- } finally {
- // shut down the scheduled executor service
- ExecutorUtils.gracefulShutdown(
- 1000L,
- TimeUnit.MILLISECONDS,
- scheduledExecutorService);
+ new ScheduledExecutorServiceAdapter(scheduledExecutorService))) {
+ runInteractiveCli(
+ yarnCluster,
+ yarnApplicationStatusMonitor,
+ acceptInteractiveInput);
+ } finally {
+ // shut down the scheduled executor service
+ ExecutorUtils.gracefulShutdown(
+ 1000L,
+ TimeUnit.MILLISECONDS,
+ scheduledExecutorService);
+ }
}
+ } catch (FlinkException e) {
+ System.err.println("Error while deploying a Flink cluster: " + e.getMessage());
+ e.printStackTrace();
+ return 1;
}
}
return 0;
@@ -693,18 +720,20 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
}
public static void main(final String[] args) throws Exception {
- final FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session
-
final String configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv();
final Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration();
+
+ final FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
+ flinkConfiguration,
+ configurationDirectory,
+ "",
+ ""); // no prefix for the YARN session
+
SecurityUtils.install(new SecurityConfiguration(flinkConfiguration));
- int retCode = SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
- @Override
- public Integer call() {
- return cli.run(args, flinkConfiguration, configurationDirectory);
- }
- });
+
+ final int retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args, flinkConfiguration, configurationDirectory));
+
System.exit(retCode);
}
@@ -858,11 +887,17 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
}
}
- public static File getYarnPropertiesLocation(Configuration conf) {
- String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir");
+ public static File getYarnPropertiesLocation(@Nullable String yarnPropertiesFileLocation) {
+
+ final String propertiesFileLocation;
+
+ if (yarnPropertiesFileLocation != null) {
+ propertiesFileLocation = yarnPropertiesFileLocation;
+ } else {
+ propertiesFileLocation = System.getProperty("java.io.tmpdir");
+ }
+
String currentUser = System.getProperty("user.name");
- String propertiesFileLocation =
- conf.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation);
return new File(propertiesFileLocation, YARN_PROPERTIES_FILE + currentUser);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index 8f8359f..2f6c157 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -22,7 +22,7 @@ import org.apache.flink.client.cli.CliFrontendParser;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
import org.apache.flink.yarn.configuration.YarnConfigOptions;
@@ -69,6 +69,8 @@ public class FlinkYarnSessionCliTest extends TestLogger {
public void testDynamicProperties() throws Exception {
FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
+ new Configuration(),
+ tmp.getRoot().getAbsolutePath(),
"",
"",
false);
@@ -80,11 +82,7 @@ public class FlinkYarnSessionCliTest extends TestLogger {
CommandLine cmd = parser.parse(options, new String[]{"run", "-j", "fake.jar", "-n", "15",
"-D", "akka.ask.timeout=5 min", "-D", "env.java.opts=-DappName=foobar"});
- AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(
- new Configuration(),
- tmp.getRoot().getAbsolutePath(),
- null,
- cmd);
+ AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(cmd);
Assert.assertNotNull(flinkYarnDescriptor);
@@ -100,7 +98,11 @@ public class FlinkYarnSessionCliTest extends TestLogger {
String[] params =
new String[] {"-yn", "2", "-ys", "3", "-p", "7"};
- FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli("y", "yarn");
+ FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli(
+ new Configuration(),
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
Options options = new Options();
// TODO: Nasty workaround: We should get rid of the YarnCLI and run options coupling
@@ -110,7 +112,7 @@ public class FlinkYarnSessionCliTest extends TestLogger {
final CommandLine commandLine = CliFrontendParser.parse(options, params, true);
- ClusterSpecification clusterSpecification = yarnCLI.createClusterSpecification(new Configuration(), commandLine);
+ ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(commandLine);
// each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased.
assertEquals(4, clusterSpecification.getSlotsPerTaskManager());
@@ -122,21 +124,17 @@ public class FlinkYarnSessionCliTest extends TestLogger {
String[] params =
new String[] {"-yn", "2", "-ys", "3"};
- FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli("y", "yarn");
+ FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli(
+ new Configuration(),
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true);
- final Configuration configuration = new Configuration();
+ AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine);
- AbstractYarnClusterDescriptor descriptor = yarnCLI.createDescriptor(
- configuration,
- tmp.getRoot().getAbsolutePath(),
- "",
- commandLine);
-
- final ClusterSpecification clusterSpecification = yarnCLI.createClusterSpecification(
- configuration,
- commandLine);
+ final ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(commandLine);
// each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased.
assertEquals(3, clusterSpecification.getSlotsPerTaskManager());
@@ -149,15 +147,15 @@ public class FlinkYarnSessionCliTest extends TestLogger {
String[] params = new String[] {"-yn", "2", "-yz", zkNamespaceCliInput};
- FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli("y", "yarn");
+ FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli(
+ new Configuration(),
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true);
- AbstractYarnClusterDescriptor descriptor = yarnCLI.createDescriptor(
- new Configuration(),
- tmp.getRoot().getAbsolutePath(),
- "",
- commandLine);
+ AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine);
assertEquals(zkNamespaceCliInput, descriptor.getZookeeperNamespace());
}
@@ -170,16 +168,18 @@ public class FlinkYarnSessionCliTest extends TestLogger {
File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
- final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
final Configuration configuration = new Configuration();
configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath());
+ final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+ configuration,
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
+
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true);
- final String clusterId = flinkYarnSessionCli.getClusterId(
- configuration,
- commandLine);
+ final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
}
@@ -188,50 +188,49 @@ public class FlinkYarnSessionCliTest extends TestLogger {
* Tests that we fail when reading an invalid yarn properties file when retrieving
* the cluster id.
*/
- @Test(expected = IllegalConfigurationException.class)
+ @Test(expected = FlinkException.class)
public void testInvalidYarnPropertiesFile() throws Exception {
File directoryPath = writeYarnPropertiesFile(invalidPropertiesFile);
- final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
final Configuration configuration = new Configuration();
configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath());
- final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true);
-
- flinkYarnSessionCli.getClusterId(
+ new FlinkYarnSessionCli(
configuration,
- commandLine);
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
}
@Test
public void testResumeFromYarnID() throws Exception {
- final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
final Configuration configuration = new Configuration();
+ final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+ configuration,
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true);
- final String clusterId = flinkYarnSessionCli.getClusterId(
- configuration,
- commandLine);
+ final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
}
@Test
public void testResumeFromYarnIDZookeeperNamespace() throws Exception {
- final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
final Configuration configuration = new Configuration();
+ final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+ configuration,
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true);
- final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(
- configuration,
- tmp.getRoot().getAbsolutePath(),
- commandLine);
+ final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine);
final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration();
@@ -241,17 +240,18 @@ public class FlinkYarnSessionCliTest extends TestLogger {
@Test
public void testResumeFromYarnIDZookeeperNamespaceOverride() throws Exception {
- final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
+ final Configuration configuration = new Configuration();
+ final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+ configuration,
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
final String overrideZkNamespace = "my_cluster";
- final Configuration configuration = new Configuration();
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace}, true);
- final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(
- configuration,
- tmp.getRoot().getAbsolutePath(),
- commandLine);
+ final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine);
final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration();
@@ -263,17 +263,16 @@ public class FlinkYarnSessionCliTest extends TestLogger {
public void testYarnIDOverridesPropertiesFile() throws Exception {
File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
- final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
final Configuration configuration = new Configuration();
configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath());
- final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true);
-
- final String clusterId = flinkYarnSessionCli.getClusterId(
+ final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
configuration,
- commandLine);
-
+ tmp.getRoot().getAbsolutePath(),
+ "y",
+ "yarn");
+ final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true);
+ final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
assertEquals(TEST_YARN_APPLICATION_ID_2.toString(), clusterId);
}
[08/14] flink git commit: [hotfix] Add help command to
FlinkYarnSessionCli
Posted by tr...@apache.org.
[hotfix] Add help command to FlinkYarnSessionCli
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2ce64e79
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2ce64e79
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2ce64e79
Branch: refs/heads/master
Commit: 2ce64e791b3a2e2b8a8e4ac774f2ca45da5660dc
Parents: 402499f
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Dec 29 15:12:46 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:04 2018 +0100
----------------------------------------------------------------------
.../java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java | 8 ++++++++
1 file changed, 8 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2ce64e79/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index 75a270d..24a422e 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -127,6 +127,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
private final Option detached;
private final Option zookeeperNamespace;
private final Option flip6;
+ private final Option help;
/**
* @deprecated Streaming mode has been deprecated without replacement. Set the
@@ -198,6 +199,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
name = new Option(shortPrefix + "nm", longPrefix + "name", true, "Set a custom name for the application on YARN");
zookeeperNamespace = new Option(shortPrefix + "z", longPrefix + "zookeeperNamespace", true, "Namespace to create the Zookeeper sub-paths for high availability mode");
flip6 = new Option(shortPrefix + "f6", longPrefix + "flip6", false, "Specify this option to start a Flip-6 Yarn session cluster.");
+ help = new Option(shortPrefix + "h", longPrefix + "help", false, "Help for the Yarn session CLI.");
allOptions = new Options();
allOptions.addOption(flinkJar);
@@ -215,6 +217,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
allOptions.addOption(applicationId);
allOptions.addOption(zookeeperNamespace);
allOptions.addOption(flip6);
+ allOptions.addOption(help);
// try loading a potential yarn properties file
this.yarnPropertiesFileLocation = configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION);
@@ -564,6 +567,11 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
//
final CommandLine cmd = parseCommandLineOptions(args, true);
+ if (cmd.hasOption(help.getOpt())) {
+ printUsage();
+ return 0;
+ }
+
final AbstractYarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(cmd);
try {
[06/14] flink git commit: [FLINK-8347] [flip6] Make cluster id used
by ClusterDescriptor typesafe
Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
index b17b030..938b979 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.client.program.rest;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.deployment.StandaloneClusterId;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
@@ -115,7 +116,7 @@ public class RestClusterClientTest extends TestLogger {
private RestServerEndpointConfiguration restServerEndpointConfiguration;
- private RestClusterClient restClusterClient;
+ private RestClusterClient<StandaloneClusterId> restClusterClient;
@Before
public void setUp() throws Exception {
@@ -126,7 +127,7 @@ public class RestClusterClientTest extends TestLogger {
config.setString(JobManagerOptions.ADDRESS, "localhost");
restServerEndpointConfiguration = RestServerEndpointConfiguration.fromConfiguration(config);
mockGatewayRetriever = () -> CompletableFuture.completedFuture(mockRestfulGateway);
- restClusterClient = new RestClusterClient(config, (attempt) -> 0);
+ restClusterClient = new RestClusterClient(config, StandaloneClusterId.getInstance(), (attempt) -> 0);
}
@After
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
index 24cb0fc..f4ea659 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
@@ -20,7 +20,6 @@ package org.apache.flink.storm.api;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.JobWithJars;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.client.program.StandaloneClusterClient;
@@ -205,7 +204,7 @@ public class FlinkClient {
configuration.setString(JobManagerOptions.ADDRESS, jobManagerHost);
configuration.setInteger(JobManagerOptions.PORT, jobManagerPort);
- final ClusterClient client;
+ final StandaloneClusterClient client;
try {
client = new StandaloneClusterClient(configuration);
} catch (final Exception e) {
@@ -245,7 +244,7 @@ public class FlinkClient {
configuration.setString(JobManagerOptions.ADDRESS, this.jobManagerHost);
configuration.setInteger(JobManagerOptions.PORT, this.jobManagerPort);
- final ClusterClient client;
+ final StandaloneClusterClient client;
try {
client = new StandaloneClusterClient(configuration);
} catch (final Exception e) {
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
index 25ce10a..ac2f7eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.rpc;
import java.util.concurrent.CompletableFuture;
/**
- * Interface for self gateways
+ * Interface for self gateways.
*/
public interface RpcServer extends StartStoppable, MainThreadExecutable, RpcGateway {
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index 349d940..6fa6aa5 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -21,6 +21,7 @@ package org.apache.flink.api.scala
import java.io._
import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser, RunOptions}
+import org.apache.flink.client.deployment.{ClusterDescriptor, StandaloneClusterId}
import org.apache.flink.client.program.ClusterClient
import org.apache.flink.configuration.{Configuration, GlobalConfiguration, JobManagerOptions}
import org.apache.flink.runtime.minicluster.StandaloneMiniCluster
@@ -138,7 +139,7 @@ object FlinkShell {
def fetchConnectionInfo(
configuration: Configuration,
config: Config
- ): (String, Int, Option[Either[StandaloneMiniCluster, ClusterClient]]) = {
+ ): (String, Int, Option[Either[StandaloneMiniCluster, ClusterClient[_]]]) = {
config.executionMode match {
case ExecutionMode.LOCAL => // Local mode
val config = GlobalConfiguration.loadConfiguration()
@@ -294,7 +295,9 @@ object FlinkShell {
CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
val customCLI = frontend.getActiveCustomCommandLine(commandLine)
- val clusterDescriptor = customCLI.createClusterDescriptor(commandLine)
+ val clusterDescriptor = customCLI
+ .createClusterDescriptor(commandLine)
+ .asInstanceOf[ClusterDescriptor[Any]]
val clusterId = customCLI.getClusterId(commandLine)
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 74b1c68..a0b4a40 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -21,7 +21,6 @@ import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.JobWithJars;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.client.program.StandaloneClusterClient;
@@ -199,7 +198,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
configuration.setString(JobManagerOptions.ADDRESS, host);
configuration.setInteger(JobManagerOptions.PORT, port);
- ClusterClient client;
+ StandaloneClusterClient client;
try {
client = new StandaloneClusterClient(configuration);
client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled());
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
index 7f8572b..221f3fa 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
@@ -20,6 +20,7 @@
package org.apache.flink.test.example.client;
import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.deployment.StandaloneClusterId;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.StandaloneClusterClient;
import org.apache.flink.configuration.Configuration;
@@ -78,7 +79,7 @@ public class JobRetrievalITCase extends TestLogger {
final JobGraph jobGraph = new JobGraph(jobID, "testjob", imalock);
- final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
+ final ClusterClient<StandaloneClusterId> client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
// acquire the lock to make sure that the job cannot complete until the job client
// has been attached in resumingThread
@@ -123,7 +124,7 @@ public class JobRetrievalITCase extends TestLogger {
@Test
public void testNonExistingJobRetrieval() throws Exception {
final JobID jobID = new JobID();
- ClusterClient client = new StandaloneClusterClient(cluster.configuration());
+ ClusterClient<StandaloneClusterId> client = new StandaloneClusterClient(cluster.configuration());
try {
client.retrieveJob(jobID);
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index 897dc8c..55669f1 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -41,6 +41,7 @@ import akka.actor.PoisonPill;
import akka.testkit.JavaTestKit;
import org.apache.curator.test.TestingServer;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -125,7 +126,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
"@@" + FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY + "=" + fsStateHandlePath + "/checkpoints" +
"@@" + HighAvailabilityOptions.HA_STORAGE_PATH.key() + "=" + fsStateHandlePath + "/recovery");
- ClusterClient yarnCluster = null;
+ ClusterClient<ApplicationId> yarnCluster = null;
final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.MINUTES);
@@ -141,8 +142,6 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
try {
yarnCluster = flinkYarnClient.deploySessionCluster(clusterSpecification);
- final ClusterClient finalYarnCluster = yarnCluster;
-
highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
yarnCluster.getFlinkConfiguration(),
Executors.directExecutor(),
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index 15fe355..ec6c105 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -246,7 +246,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
.setSlotsPerTaskManager(1)
.createClusterSpecification();
// deploy
- ClusterClient yarnCluster = null;
+ ClusterClient<ApplicationId> yarnCluster = null;
try {
yarnCluster = clusterDescriptor.deploySessionCluster(clusterSpecification);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 4affb78..a7e8c36 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -19,9 +19,10 @@
package org.apache.flink.yarn;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.deployment.ClusterDeploymentException;
import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.client.deployment.ClusterRetrieveException;
import org.apache.flink.client.deployment.ClusterSpecification;
-import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
@@ -61,7 +62,6 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -101,7 +101,7 @@ import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties
/**
* The descriptor with deployment information for spawning or resuming a {@link YarnClusterClient}.
*/
-public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor {
+public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor<ApplicationId> {
private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class);
/**
@@ -332,7 +332,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
// -------------------------------------------------------------
@Override
- public ClusterClient retrieve(String applicationID) {
+ public YarnClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException {
try {
// check if required Hadoop environment variables are set. If not, warn user
@@ -343,18 +343,17 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
"configuration for accessing YARN.");
}
- final ApplicationId yarnAppId = ConverterUtils.toApplicationId(applicationID);
- final ApplicationReport appReport = yarnClient.getApplicationReport(yarnAppId);
+ final ApplicationReport appReport = yarnClient.getApplicationReport(applicationId);
if (appReport.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) {
// Flink cluster is not running anymore
LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}",
- applicationID, appReport.getFinalApplicationStatus());
- throw new RuntimeException("The Yarn application " + applicationID + " doesn't run anymore.");
+ applicationId, appReport.getFinalApplicationStatus());
+ throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore.");
}
LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'",
- appReport.getHost(), appReport.getRpcPort(), applicationID);
+ appReport.getHost(), appReport.getRpcPort(), applicationId);
flinkConfiguration.setString(JobManagerOptions.ADDRESS, appReport.getHost());
flinkConfiguration.setInteger(JobManagerOptions.PORT, appReport.getRpcPort());
@@ -367,43 +366,35 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
flinkConfiguration,
false);
} catch (Exception e) {
- throw new RuntimeException("Couldn't retrieve Yarn cluster", e);
+ throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e);
}
}
@Override
- public YarnClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) {
+ public YarnClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
try {
return deployInternal(
clusterSpecification,
getYarnSessionClusterEntrypoint(),
null);
} catch (Exception e) {
- throw new RuntimeException("Couldn't deploy Yarn session cluster", e);
+ throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e);
}
}
@Override
- public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+ public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException {
try {
return deployInternal(
clusterSpecification,
getYarnJobClusterEntrypoint(),
jobGraph);
} catch (Exception e) {
- throw new RuntimeException("Could not deploy Yarn job cluster.", e);
+ throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e);
}
}
@Override
- public void terminateCluster(String clusterId) throws FlinkException {
- try {
- yarnClient.killApplication(ConverterUtils.toApplicationId(clusterId));
- } catch (IOException | YarnException e) {
- throw new FlinkException("Could not terminate cluster with id " + clusterId + '.', e);
- }
- }
-
public void terminateCluster(ApplicationId applicationId) throws FlinkException {
try {
yarnClient.killApplication(applicationId);
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index 991b3b9..f2be264 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -41,7 +41,6 @@ import akka.util.Timeout;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.ConverterUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -58,7 +57,7 @@ import scala.concurrent.duration.FiniteDuration;
/**
* Java representation of a running Flink cluster within YARN.
*/
-public class YarnClusterClient extends ClusterClient {
+public class YarnClusterClient extends ClusterClient<ApplicationId> {
private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class);
@@ -170,11 +169,6 @@ public class YarnClusterClient extends ClusterClient {
}
}
- @Override
- public String getClusterIdentifier() {
- return ConverterUtils.toString(appReport.getApplicationId());
- }
-
/**
* This method is only available if the cluster hasn't been started in detached mode.
*/
@@ -234,6 +228,11 @@ public class YarnClusterClient extends ClusterClient {
}
@Override
+ public ApplicationId getClusterId() {
+ return appId;
+ }
+
+ @Override
public boolean isDetached() {
return super.isDetached() || clusterDescriptor.isDetachedMode();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index 24a422e..fea8a5c 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -86,7 +86,7 @@ import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_
/**
* Class handling the command line interface to the YARN session.
*/
-public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
+public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId> {
private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class);
//------------------------------------ Constants -------------------------
@@ -152,7 +152,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
private final Properties yarnPropertiesFile;
- private final String yarnApplicationIdFromYarnProperties;
+ private final ApplicationId yarnApplicationIdFromYarnProperties;
private final String yarnPropertiesFileLocation;
@@ -235,20 +235,20 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
". Please delete the file at " + yarnPropertiesLocation.getAbsolutePath() + '.', ioe);
}
- yarnApplicationIdFromYarnProperties = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
+ final String yarnApplicationIdString = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
- if (yarnApplicationIdFromYarnProperties == null) {
+ if (yarnApplicationIdString == null) {
throw new FlinkException("Yarn properties file found but doesn't contain a " +
"Yarn application id. Please delete the file at " + yarnPropertiesLocation.getAbsolutePath());
}
try {
// try converting id to ApplicationId
- ConverterUtils.toApplicationId(yarnApplicationIdFromYarnProperties);
+ yarnApplicationIdFromYarnProperties = ConverterUtils.toApplicationId(yarnApplicationIdString);
}
catch (Exception e) {
throw new FlinkException("YARN properties contains an invalid entry for " +
- "application id: " + yarnApplicationIdFromYarnProperties + ". Please delete the file at " +
+ "application id: " + yarnApplicationIdString + ". Please delete the file at " +
yarnPropertiesLocation.getAbsolutePath(), e);
}
} else {
@@ -458,9 +458,9 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
@Override
@Nullable
- public String getClusterId(CommandLine commandLine) {
+ public ApplicationId getClusterId(CommandLine commandLine) {
if (commandLine.hasOption(applicationId.getOpt())) {
- return commandLine.getOptionValue(applicationId.getOpt());
+ return ConverterUtils.toApplicationId(commandLine.getOptionValue(applicationId.getOpt()));
} else if (isYarnPropertiesFileMode(commandLine)) {
return yarnApplicationIdFromYarnProperties;
} else {
@@ -485,14 +485,14 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
effectiveConfiguration.setString(HA_CLUSTER_ID, zkNamespace);
}
- final String applicationId = getClusterId(commandLine);
+ final ApplicationId applicationId = getClusterId(commandLine);
if (applicationId != null) {
final String zooKeeperNamespace;
if (commandLine.hasOption(zookeeperNamespace.getOpt())){
zooKeeperNamespace = commandLine.getOptionValue(zookeeperNamespace.getOpt());
} else {
- zooKeeperNamespace = effectiveConfiguration.getString(HA_CLUSTER_ID, applicationId);
+ zooKeeperNamespace = effectiveConfiguration.getString(HA_CLUSTER_ID, applicationId.toString());
}
effectiveConfiguration.setString(HA_CLUSTER_ID, zooKeeperNamespace);
@@ -581,20 +581,20 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
System.out.println(description);
return 0;
} else {
- final ClusterClient clusterClient;
+ final ClusterClient<ApplicationId> clusterClient;
final ApplicationId yarnApplicationId;
if (cmd.hasOption(applicationId.getOpt())) {
yarnApplicationId = ConverterUtils.toApplicationId(cmd.getOptionValue(applicationId.getOpt()));
- clusterClient = yarnClusterDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
+ clusterClient = yarnClusterDescriptor.retrieve(yarnApplicationId);
} else {
final ClusterSpecification clusterSpecification = getClusterSpecification(cmd);
clusterClient = yarnClusterDescriptor.deploySessionCluster(clusterSpecification);
//------------------ ClusterClient deployed, handle connection details
- yarnApplicationId = ConverterUtils.toApplicationId(clusterClient.getClusterIdentifier());
+ yarnApplicationId = clusterClient.getClusterId();
String jobManagerAddress =
clusterClient.getJobManagerAddress().getAddress().getHostName() +
@@ -790,7 +790,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
}
private static void runInteractiveCli(
- ClusterClient clusterClient,
+ ClusterClient<?> clusterClient,
YarnApplicationStatusMonitor yarnApplicationStatusMonitor,
boolean readConsoleInput) {
try (BufferedReader in = new BufferedReader(new InputStreamReader(System.in))) {
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
index 2581e68..4e68612 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.yarn;
+import org.apache.flink.client.deployment.ClusterRetrieveException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.TestLogger;
@@ -49,10 +50,9 @@ public class AbstractYarnClusterTest extends TestLogger {
/**
* Tests that the cluster retrieval of a finished YARN application fails.
*/
- @Test(expected = RuntimeException.class)
- public void testClusterClientRetrievalOfFinishedYarnApplication() throws IOException {
+ @Test(expected = ClusterRetrieveException.class)
+ public void testClusterClientRetrievalOfFinishedYarnApplication() throws Exception {
final ApplicationId applicationId = ApplicationId.newInstance(System.currentTimeMillis(), 42);
- final String clusterId = applicationId.toString();
final ApplicationReport applicationReport = createApplicationReport(
applicationId,
YarnApplicationState.FINISHED,
@@ -65,30 +65,7 @@ public class AbstractYarnClusterTest extends TestLogger {
temporaryFolder.newFolder().getAbsolutePath(),
yarnClient);
- clusterDescriptor.retrieve(clusterId);
- }
-
- /**
- * Tests that the cluster retrieval fails if an invalid application id is provided.
- */
- @Test(expected = RuntimeException.class)
- public void testClusterClientRetrievalFromInvalidApplicationId() throws IOException {
- final ApplicationId applicationId = ApplicationId.newInstance(System.currentTimeMillis(), 42);
- final String clusterId = "foobar";
-
- final ApplicationReport applicationReport = createApplicationReport(
- applicationId,
- YarnApplicationState.RUNNING,
- FinalApplicationStatus.UNDEFINED);
-
- final YarnClient yarnClient = new TestingYarnClient(Collections.singletonMap(applicationId, applicationReport));
-
- final TestingAbstractYarnClusterDescriptor clusterDescriptor = new TestingAbstractYarnClusterDescriptor(
- new Configuration(),
- temporaryFolder.newFolder().getAbsolutePath(),
- yarnClient);
-
- clusterDescriptor.retrieve(clusterId);
+ clusterDescriptor.retrieve(applicationId);
}
private ApplicationReport createApplicationReport(
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index 2f6c157..9be9cb5 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -179,9 +179,9 @@ public class FlinkYarnSessionCliTest extends TestLogger {
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true);
- final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
+ final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine);
- assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
+ assertEquals(TEST_YARN_APPLICATION_ID, clusterId);
}
/**
@@ -214,9 +214,9 @@ public class FlinkYarnSessionCliTest extends TestLogger {
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true);
- final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
+ final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine);
- assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
+ assertEquals(TEST_YARN_APPLICATION_ID, clusterId);
}
@Test
@@ -272,8 +272,8 @@ public class FlinkYarnSessionCliTest extends TestLogger {
"y",
"yarn");
final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true);
- final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
- assertEquals(TEST_YARN_APPLICATION_ID_2.toString(), clusterId);
+ final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine);
+ assertEquals(TEST_YARN_APPLICATION_ID_2, clusterId);
}
///////////
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
index 5be54b0..e0f69cb 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.yarn;
+import org.apache.flink.client.deployment.ClusterDeploymentException;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
@@ -66,7 +67,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
}
@Test
- public void testFailIfTaskSlotsHigherThanMaxVcores() {
+ public void testFailIfTaskSlotsHigherThanMaxVcores() throws ClusterDeploymentException {
final YarnClient yarnClient = YarnClient.createYarnClient();
@@ -88,7 +89,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
clusterDescriptor.deploySessionCluster(clusterSpecification);
fail("The deploy call should have failed.");
- } catch (RuntimeException e) {
+ } catch (ClusterDeploymentException e) {
// we expect the cause to be an IllegalConfigurationException
if (!(e.getCause() instanceof IllegalConfigurationException)) {
throw e;
@@ -99,7 +100,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
}
@Test
- public void testConfigOverwrite() {
+ public void testConfigOverwrite() throws ClusterDeploymentException {
Configuration configuration = new Configuration();
// overwrite vcores in config
configuration.setInteger(YarnConfigOptions.VCORES, Integer.MAX_VALUE);
@@ -125,7 +126,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
clusterDescriptor.deploySessionCluster(clusterSpecification);
fail("The deploy call should have failed.");
- } catch (RuntimeException e) {
+ } catch (ClusterDeploymentException e) {
// we expect the cause to be an IllegalConfigurationException
if (!(e.getCause() instanceof IllegalConfigurationException)) {
throw e;
[07/14] flink git commit: [FLINK-8347] [flip6] Make cluster id used
by ClusterDescriptor typesafe
Posted by tr...@apache.org.
[FLINK-8347] [flip6] Make cluster id used by ClusterDescriptor typesafe
The ClusterDescriptor uses a typed cluster id for the ClusterClient retrieval.
Moreover, the ClusterClient and the CustomCommandLine are typed accordingly.
This closes #5232.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/38d37208
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/38d37208
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/38d37208
Branch: refs/heads/master
Commit: 38d3720863c6187153174d0df57fc414b0cf8e96
Parents: 2ce64e7
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Jan 11 23:46:33 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:04 2018 +0100
----------------------------------------------------------------------
.../org/apache/flink/client/RemoteExecutor.java | 2 +-
.../client/cli/AbstractCustomCommandLine.java | 2 +-
.../apache/flink/client/cli/CliFrontend.java | 462 ++++++++++---------
.../flink/client/cli/CliFrontendParser.java | 14 +-
.../flink/client/cli/CustomCommandLine.java | 6 +-
.../org/apache/flink/client/cli/DefaultCLI.java | 10 +-
.../flink/client/cli/Flip6DefaultCLI.java | 10 +-
.../client/deployment/ClusterDescriptor.java | 20 +-
.../deployment/ClusterRetrieveException.java | 41 ++
.../Flip6StandaloneClusterDescriptor.java | 16 +-
.../deployment/StandaloneClusterDescriptor.java | 12 +-
.../client/deployment/StandaloneClusterId.java | 32 ++
.../flink/client/program/ClusterClient.java | 10 +-
.../client/program/ContextEnvironment.java | 6 +-
.../program/ContextEnvironmentFactory.java | 4 +-
.../client/program/DetachedEnvironment.java | 2 +-
.../client/program/StandaloneClusterClient.java | 8 +-
.../client/program/rest/RestClusterClient.java | 27 +-
.../flink/client/cli/CliFrontendCancelTest.java | 10 +-
.../flink/client/cli/CliFrontendListTest.java | 6 +-
.../client/cli/CliFrontendSavepointTest.java | 18 +-
.../flink/client/cli/CliFrontendStopTest.java | 48 +-
.../apache/flink/client/cli/DefaultCLITest.java | 10 +-
.../client/cli/util/DummyClusterDescriptor.java | 16 +-
.../client/cli/util/DummyCustomCommandLine.java | 10 +-
.../client/program/ClientConnectionTest.java | 4 +-
.../apache/flink/client/program/ClientTest.java | 8 +-
.../flink/client/program/ClusterClientTest.java | 16 +-
.../program/rest/RestClusterClientTest.java | 5 +-
.../org/apache/flink/storm/api/FlinkClient.java | 5 +-
.../org/apache/flink/runtime/rpc/RpcServer.java | 2 +-
.../org/apache/flink/api/scala/FlinkShell.scala | 7 +-
.../environment/RemoteStreamEnvironment.java | 3 +-
.../test/example/client/JobRetrievalITCase.java | 5 +-
.../flink/yarn/YARNHighAvailabilityITCase.java | 5 +-
.../flink/yarn/YARNSessionFIFOITCase.java | 2 +-
.../yarn/AbstractYarnClusterDescriptor.java | 35 +-
.../apache/flink/yarn/YarnClusterClient.java | 13 +-
.../flink/yarn/cli/FlinkYarnSessionCli.java | 28 +-
.../flink/yarn/AbstractYarnClusterTest.java | 31 +-
.../flink/yarn/FlinkYarnSessionCliTest.java | 12 +-
.../flink/yarn/YarnClusterDescriptorTest.java | 9 +-
42 files changed, 528 insertions(+), 464 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
index 1ae9b07..fcf8bab 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
@@ -58,7 +58,7 @@ public class RemoteExecutor extends PlanExecutor {
private final Configuration clientConfiguration;
- private ClusterClient client;
+ private ClusterClient<?> client;
private int defaultParallelism = 1;
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
index da21556..59046bf 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
@@ -38,7 +38,7 @@ import static org.apache.flink.client.cli.CliFrontend.setJobManagerAddressInConf
* a ZooKeeper namespace.
*
*/
-public abstract class AbstractCustomCommandLine implements CustomCommandLine {
+public abstract class AbstractCustomCommandLine<T> implements CustomCommandLine<T> {
protected final Option zookeeperNamespaceOption = new Option("z", "zookeeperNamespace", true,
"Namespace to create the Zookeeper sub-paths for high availability mode");
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 847a5f8..d661aa9 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -76,9 +76,7 @@ import java.util.Comparator;
import java.util.Date;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import scala.concurrent.duration.FiniteDuration;
@@ -106,7 +104,7 @@ public class CliFrontend {
private final Configuration configuration;
- private final List<CustomCommandLine> customCommandLines;
+ private final List<CustomCommandLine<?>> customCommandLines;
private final Options customCommandLineOptions;
@@ -116,7 +114,7 @@ public class CliFrontend {
public CliFrontend(
Configuration configuration,
- List<CustomCommandLine> customCommandLines) throws Exception {
+ List<CustomCommandLine<?>> customCommandLines) throws Exception {
this.configuration = Preconditions.checkNotNull(configuration);
this.customCommandLines = Preconditions.checkNotNull(customCommandLines);
@@ -129,7 +127,7 @@ public class CliFrontend {
this.customCommandLineOptions = new Options();
- for (CustomCommandLine customCommandLine : customCommandLines) {
+ for (CustomCommandLine<?> customCommandLine : customCommandLines) {
customCommandLine.addGeneralOptions(customCommandLineOptions);
customCommandLine.addRunOptions(customCommandLineOptions);
}
@@ -196,14 +194,22 @@ public class CliFrontend {
throw new CliArgsException("Could not build the program from JAR file.", e);
}
- final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
+ final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
+ runProgram(customCommandLine, commandLine, runOptions, program);
+ }
+
+ private <T> void runProgram(
+ CustomCommandLine<T> customCommandLine,
+ CommandLine commandLine,
+ RunOptions runOptions,
+ PackagedProgram program) throws ProgramInvocationException, FlinkException {
+ final ClusterDescriptor<T> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
try {
- final String clusterId = customCommandLine.getClusterId(commandLine);
+ final T clusterId = customCommandLine.getClusterId(commandLine);
- final ClusterClient client;
+ final ClusterClient<T> client;
if (clusterId != null) {
client = clusterDescriptor.retrieve(clusterId);
@@ -235,7 +241,7 @@ public class CliFrontend {
if (clusterId == null && !client.isDetached()) {
// terminate the cluster only if we have started it before and if it's not detached
try {
- clusterDescriptor.terminateCluster(client.getClusterIdentifier());
+ clusterDescriptor.terminateCluster(client.getClusterId());
} catch (FlinkException e) {
LOG.info("Could not properly terminate the Flink cluster.", e);
}
@@ -351,100 +357,86 @@ public class CliFrontend {
return;
}
- boolean running = listOptions.getRunning();
- boolean scheduled = listOptions.getScheduled();
+ final boolean running;
+ final boolean scheduled;
// print running and scheduled jobs if not option supplied
- if (!running && !scheduled) {
+ if (!listOptions.getRunning() && !listOptions.getScheduled()) {
running = true;
scheduled = true;
+ } else {
+ running = listOptions.getRunning();
+ scheduled = listOptions.getScheduled();
}
- final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+ final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final String clusterId = activeCommandLine.getClusterId(commandLine);
-
- if (clusterId == null) {
- throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
- "you would like to connect.");
- }
+ runClusterAction(
+ activeCommandLine,
+ commandLine,
+ clusterClient -> listJobs(clusterClient, running, scheduled));
- final ClusterClient client = clusterDescriptor.retrieve(clusterId);
+ }
+ private <T> void listJobs(
+ ClusterClient<T> clusterClient,
+ boolean running,
+ boolean scheduled) throws FlinkException {
+ Collection<JobStatusMessage> jobDetails;
try {
- Collection<JobStatusMessage> jobDetails;
- try {
- CompletableFuture<Collection<JobStatusMessage>> jobDetailsFuture = client.listJobs();
+ CompletableFuture<Collection<JobStatusMessage>> jobDetailsFuture = clusterClient.listJobs();
- try {
- logAndSysout("Waiting for response...");
- jobDetails = jobDetailsFuture.get();
- }
- catch (ExecutionException ee) {
- Throwable cause = ExceptionUtils.stripExecutionException(ee);
- throw new Exception("Failed to retrieve job list.", cause);
- }
- } finally {
- client.shutdown();
- }
+ logAndSysout("Waiting for response...");
+ jobDetails = jobDetailsFuture.get();
- LOG.info("Successfully retrieved list of jobs");
+ } catch (Exception e) {
+ Throwable cause = ExceptionUtils.stripExecutionException(e);
+ throw new FlinkException("Failed to retrieve job list.", cause);
+ }
- SimpleDateFormat dateFormat = new SimpleDateFormat("dd.MM.yyyy HH:mm:ss");
- Comparator<JobStatusMessage> startTimeComparator = (o1, o2) -> (int) (o1.getStartTime() - o2.getStartTime());
+ LOG.info("Successfully retrieved list of jobs");
- final List<JobStatusMessage> runningJobs = new ArrayList<>();
- final List<JobStatusMessage> scheduledJobs = new ArrayList<>();
- jobDetails.forEach(details -> {
- if (details.getJobState() == JobStatus.CREATED) {
- scheduledJobs.add(details);
- } else {
- runningJobs.add(details);
- }
- });
+ SimpleDateFormat dateFormat = new SimpleDateFormat("dd.MM.yyyy HH:mm:ss");
+ Comparator<JobStatusMessage> startTimeComparator = (o1, o2) -> (int) (o1.getStartTime() - o2.getStartTime());
- if (running) {
- if (runningJobs.size() == 0) {
- System.out.println("No running jobs.");
- }
- else {
- runningJobs.sort(startTimeComparator);
+ final List<JobStatusMessage> runningJobs = new ArrayList<>();
+ final List<JobStatusMessage> scheduledJobs = new ArrayList<>();
+ jobDetails.forEach(details -> {
+ if (details.getJobState() == JobStatus.CREATED) {
+ scheduledJobs.add(details);
+ } else {
+ runningJobs.add(details);
+ }
+ });
- System.out.println("------------------ Running/Restarting Jobs -------------------");
- for (JobStatusMessage runningJob : runningJobs) {
- System.out.println(dateFormat.format(new Date(runningJob.getStartTime()))
- + " : " + runningJob.getJobId() + " : " + runningJob.getJobName() + " (" + runningJob.getJobState() + ")");
- }
- System.out.println("--------------------------------------------------------------");
- }
+ if (running) {
+ if (runningJobs.size() == 0) {
+ System.out.println("No running jobs.");
}
- if (scheduled) {
- if (scheduledJobs.size() == 0) {
- System.out.println("No scheduled jobs.");
- }
- else {
- scheduledJobs.sort(startTimeComparator);
+ else {
+ runningJobs.sort(startTimeComparator);
- System.out.println("----------------------- Scheduled Jobs -----------------------");
- for (JobStatusMessage scheduledJob : scheduledJobs) {
- System.out.println(dateFormat.format(new Date(scheduledJob.getStartTime()))
- + " : " + scheduledJob.getJobId() + " : " + scheduledJob.getJobName());
- }
- System.out.println("--------------------------------------------------------------");
+ System.out.println("------------------ Running/Restarting Jobs -------------------");
+ for (JobStatusMessage runningJob : runningJobs) {
+ System.out.println(dateFormat.format(new Date(runningJob.getStartTime()))
+ + " : " + runningJob.getJobId() + " : " + runningJob.getJobName() + " (" + runningJob.getJobState() + ")");
}
+ System.out.println("--------------------------------------------------------------");
}
- } finally {
- try {
- client.shutdown();
- } catch (Exception e) {
- LOG.info("Could not properly shut down the client.", e);
+ }
+ if (scheduled) {
+ if (scheduledJobs.size() == 0) {
+ System.out.println("No scheduled jobs.");
}
+ else {
+ scheduledJobs.sort(startTimeComparator);
- try {
- clusterDescriptor.close();
- } catch (Exception e) {
- LOG.info("Could not properly close the cluster descriptor.", e);
+ System.out.println("----------------------- Scheduled Jobs -----------------------");
+ for (JobStatusMessage scheduledJob : scheduledJobs) {
+ System.out.println(dateFormat.format(new Date(scheduledJob.getStartTime()))
+ + " : " + scheduledJob.getJobId() + " : " + scheduledJob.getJobName());
+ }
+ System.out.println("--------------------------------------------------------------");
}
}
}
@@ -477,41 +469,26 @@ public class CliFrontend {
if (stopArgs.length > 0) {
String jobIdString = stopArgs[0];
jobId = parseJobId(jobIdString);
- }
- else {
+ } else {
throw new CliArgsException("Missing JobID");
}
- final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
-
- final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+ final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final String clusterId = activeCommandLine.getClusterId(commandLine);
-
- if (clusterId == null) {
- throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
- "you would like to connect.");
- }
+ logAndSysout("Stopping job " + jobId + '.');
- final ClusterClient client = clusterDescriptor.retrieve(clusterId);
-
- try {
- logAndSysout("Stopping job " + jobId + '.');
- client.stop(jobId);
- logAndSysout("Stopped job " + jobId + '.');
- } finally {
- try {
- client.shutdown();
- } catch (Exception e) {
- LOG.info("Could not properly shut down the client.", e);
- }
+ runClusterAction(
+ activeCommandLine,
+ commandLine,
+ clusterClient -> {
+ try {
+ clusterClient.stop(jobId);
+ } catch (Exception e) {
+ throw new FlinkException("Could not stop the job " + jobId + '.', e);
+ }
+ });
- try {
- clusterDescriptor.close();
- } catch (Exception e) {
- LOG.info("Could not properly close the cluster descriptor.", e);
- }
- }
+ logAndSysout("Stopped job " + jobId + '.');
}
/**
@@ -536,71 +513,63 @@ public class CliFrontend {
return;
}
- String[] cleanedArgs = cancelOptions.getArgs();
+ final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- boolean withSavepoint = cancelOptions.isWithSavepoint();
- String targetDirectory = cancelOptions.getSavepointTargetDirectory();
+ final String[] cleanedArgs = cancelOptions.getArgs();
- JobID jobId;
+ if (cancelOptions.isWithSavepoint()) {
+ final JobID jobId;
+ final String targetDirectory;
- // Figure out jobID. This is a little overly complicated, because
- // we have to figure out whether the optional target directory
- // is set:
- // - cancel -s <jobID> => default target dir (JobID parsed as opt arg)
- // - cancel -s <targetDir> <jobID> => custom target dir (parsed correctly)
- if (cleanedArgs.length > 0) {
- String jobIdString = cleanedArgs[0];
+ if (cleanedArgs.length > 0) {
+ jobId = parseJobId(cleanedArgs[0]);
+ targetDirectory = cancelOptions.getSavepointTargetDirectory();
+ } else {
+ jobId = parseJobId(cancelOptions.getSavepointTargetDirectory());
+ targetDirectory = null;
+ }
- jobId = parseJobId(jobIdString);
- } else if (targetDirectory != null) {
- // Try this for case: cancel -s <jobID> (default savepoint target dir)
- String jobIdString = targetDirectory;
- targetDirectory = null;
+ if (targetDirectory == null) {
+ logAndSysout("Cancelling job " + jobId + " with savepoint to default savepoint directory.");
+ } else {
+ logAndSysout("Cancelling job " + jobId + " with savepoint to " + targetDirectory + '.');
+ }
- jobId = parseJobId(jobIdString);
+ runClusterAction(
+ activeCommandLine,
+ commandLine,
+ clusterClient -> {
+ final String savepointPath;
+ try {
+ savepointPath = clusterClient.cancelWithSavepoint(jobId, targetDirectory);
+ } catch (Exception e) {
+ throw new FlinkException("Could not cancel job " + jobId + '.', e);
+ }
+ logAndSysout("Cancelled job " + jobId + ". Savepoint stored in " + savepointPath + '.');
+ });
} else {
- throw new CliArgsException("Missing JobID in the command line arguments.");
- }
-
- final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
-
- final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+ final JobID jobId;
- final String clusterId = activeCommandLine.getClusterId(commandLine);
-
- if (clusterId == null) {
- throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
- "you would like to connect.");
- }
-
- final ClusterClient client = clusterDescriptor.retrieve(clusterId);
-
- try {
- if (withSavepoint) {
- if (targetDirectory == null) {
- logAndSysout("Cancelling job " + jobId + " with savepoint to default savepoint directory.");
- } else {
- logAndSysout("Cancelling job " + jobId + " with savepoint to " + targetDirectory + '.');
- }
- String savepointPath = client.cancelWithSavepoint(jobId, targetDirectory);
- logAndSysout("Cancelled job " + jobId + ". Savepoint stored in " + savepointPath + '.');
+ if (cleanedArgs.length > 0) {
+ jobId = parseJobId(cleanedArgs[0]);
} else {
- logAndSysout("Cancelling job " + jobId + '.');
- client.cancel(jobId);
- logAndSysout("Cancelled job " + jobId + '.');
- }
- } finally {
- try {
- client.shutdown();
- } catch (Exception e) {
- LOG.info("Could not properly shut down the client.", e);
+ throw new CliArgsException("Missing JobID. Specify a JobID to cancel a job.");
}
- try {
- clusterDescriptor.close();
- } catch (Exception e) {
- LOG.info("Could not properly close the cluster descriptor.", e);
- }
+ logAndSysout("Cancelling job " + jobId + '.');
+
+ runClusterAction(
+ activeCommandLine,
+ commandLine,
+ clusterClient -> {
+ try {
+ clusterClient.cancel(jobId);
+ } catch (Exception e) {
+ throw new FlinkException("Could not cancel job " + jobId + '.', e);
+ }
+ });
+
+ logAndSysout("Cancelled job " + jobId + '.');
}
}
@@ -626,69 +595,52 @@ public class CliFrontend {
return;
}
- CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
+ final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
-
- final String clusterId = customCommandLine.getClusterId(commandLine);
+ if (savepointOptions.isDispose()) {
+ runClusterAction(
+ activeCommandLine,
+ commandLine,
+ clusterClient -> disposeSavepoint(clusterClient, savepointOptions.getSavepointPath()));
+ } else {
+ String[] cleanedArgs = savepointOptions.getArgs();
- if (clusterId == null) {
- throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
- "you would like to connect.");
- }
+ final JobID jobId;
- final ClusterClient clusterClient = clusterDescriptor.retrieve(clusterId);
+ if (cleanedArgs.length >= 1) {
+ String jobIdString = cleanedArgs[0];
- try {
- if (savepointOptions.isDispose()) {
- // Discard
- disposeSavepoint(clusterClient, savepointOptions.getSavepointPath());
+ jobId = parseJobId(jobIdString);
} else {
- // Trigger
- String[] cleanedArgs = savepointOptions.getArgs();
- JobID jobId;
-
- if (cleanedArgs.length >= 1) {
- String jobIdString = cleanedArgs[0];
-
- jobId = parseJobId(jobIdString);
- } else {
- throw new CliArgsException("Error: The value for the Job ID is not a valid ID. " +
- "Specify a Job ID to trigger a savepoint.");
- }
-
- String savepointDirectory = null;
- if (cleanedArgs.length >= 2) {
- savepointDirectory = cleanedArgs[1];
- }
-
- // Print superfluous arguments
- if (cleanedArgs.length >= 3) {
- logAndSysout("Provided more arguments than required. Ignoring not needed arguments.");
- }
-
- triggerSavepoint(clusterClient, jobId, savepointDirectory);
+ throw new CliArgsException("Missing JobID. " +
+ "Specify a Job ID to trigger a savepoint.");
}
- } finally {
- try {
- clusterClient.shutdown();
- } catch (Exception e) {
- LOG.info("Could not shutdown the cluster client.", e);
+
+ final String savepointDirectory;
+ if (cleanedArgs.length >= 2) {
+ savepointDirectory = cleanedArgs[1];
+ } else {
+ savepointDirectory = null;
}
- try {
- clusterDescriptor.close();
- } catch (Exception e) {
- LOG.info("Could not properly close the cluster descriptor.", e);
+ // Print superfluous arguments
+ if (cleanedArgs.length >= 3) {
+ logAndSysout("Provided more arguments than required. Ignoring not needed arguments.");
}
+
+ runClusterAction(
+ activeCommandLine,
+ commandLine,
+ clusterClient -> triggerSavepoint(clusterClient, jobId, savepointDirectory));
}
+
}
/**
* Sends a {@link org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint}
* message to the job manager.
*/
- private String triggerSavepoint(ClusterClient clusterClient, JobID jobId, String savepointDirectory) throws FlinkException {
+ private String triggerSavepoint(ClusterClient<?> clusterClient, JobID jobId, String savepointDirectory) throws FlinkException {
logAndSysout("Triggering savepoint for job " + jobId + '.');
CompletableFuture<String> savepointPathFuture = clusterClient.triggerSavepoint(jobId, savepointDirectory);
@@ -713,7 +665,7 @@ public class CliFrontend {
/**
* Sends a {@link JobManagerMessages.DisposeSavepoint} message to the job manager.
*/
- private void disposeSavepoint(ClusterClient clusterClient, String savepointPath) throws FlinkException {
+ private void disposeSavepoint(ClusterClient<?> clusterClient, String savepointPath) throws FlinkException {
Preconditions.checkNotNull(savepointPath, "Missing required argument: savepoint path. " +
"Usage: bin/flink savepoint -d <savepoint-path>");
@@ -736,7 +688,7 @@ public class CliFrontend {
// Interaction with programs and JobManager
// --------------------------------------------------------------------------------------------
- protected void executeProgram(PackagedProgram program, ClusterClient client, int parallelism) throws ProgramMissingJobException, ProgramInvocationException {
+ protected void executeProgram(PackagedProgram program, ClusterClient<?> client, int parallelism) throws ProgramMissingJobException, ProgramInvocationException {
logAndSysout("Starting execution of program");
final JobSubmissionResult result = client.run(program, parallelism);
@@ -860,7 +812,7 @@ public class CliFrontend {
System.err.println(t.getCause().getMessage());
StackTraceElement[] trace = t.getCause().getStackTrace();
for (StackTraceElement ele: trace) {
- System.err.println("\t" + ele.toString());
+ System.err.println("\t" + ele);
if (ele.getMethodName().equals("main")) {
break;
}
@@ -890,6 +842,65 @@ public class CliFrontend {
return jobId;
}
+ /**
+ * Retrieves the {@link ClusterClient} from the given {@link CustomCommandLine} and runs the given
+ * {@link ClusterAction} against it.
+ *
+ * @param activeCommandLine to create the {@link ClusterDescriptor} from
+ * @param commandLine containing the parsed command line options
+ * @param clusterAction the cluster action to run against the retrieved {@link ClusterClient}.
+ * @param <T> type of the cluster id
+ * @throws FlinkException if something goes wrong
+ */
+ private <T> void runClusterAction(CustomCommandLine<T> activeCommandLine, CommandLine commandLine, ClusterAction<T> clusterAction) throws FlinkException {
+ final ClusterDescriptor<T> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+
+ final T clusterId = activeCommandLine.getClusterId(commandLine);
+
+ if (clusterId == null) {
+ throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
+ "you would like to connect.");
+ } else {
+ try {
+ final ClusterClient<T> clusterClient = clusterDescriptor.retrieve(clusterId);
+
+ try {
+ clusterAction.runAction(clusterClient);
+ } finally {
+ try {
+ clusterClient.shutdown();
+ } catch (Exception e) {
+ LOG.info("Could not properly shut down the cluster client.", e);
+ }
+ }
+ } finally {
+ try {
+ clusterDescriptor.close();
+ } catch (Exception e) {
+ LOG.info("Could not properly close the cluster descriptor.", e);
+ }
+ }
+ }
+ }
+
+ /**
+ * Internal interface to encapsulate cluster actions which are executed via
+ * the {@link ClusterClient}.
+ *
+ * @param <T> tyoe pf the cluster id
+ */
+ @FunctionalInterface
+ private interface ClusterAction<T> {
+
+ /**
+ * Run the cluster action with the given {@link ClusterClient}.
+ *
+ * @param clusterClient to run the cluster action against
+ * @throws FlinkException if something goes wrong
+ */
+ void runAction(ClusterClient<T> clusterClient) throws FlinkException;
+ }
+
// --------------------------------------------------------------------------------------------
// Entry point for executable
// --------------------------------------------------------------------------------------------
@@ -981,7 +992,7 @@ public class CliFrontend {
final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory);
// 3. load the custom command lines
- final List<CustomCommandLine> customCommandLines = loadCustomCommandLines(
+ final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines(
configuration,
configurationDirectory);
@@ -992,12 +1003,7 @@ public class CliFrontend {
SecurityUtils.install(new SecurityConfiguration(cli.configuration));
int retCode = SecurityUtils.getInstalledContext()
- .runSecured(new Callable<Integer>() {
- @Override
- public Integer call() {
- return cli.parseParameters(args);
- }
- });
+ .runSecured(() -> cli.parseParameters(args));
System.exit(retCode);
}
catch (Throwable t) {
@@ -1048,8 +1054,8 @@ public class CliFrontend {
config.setInteger(JobManagerOptions.PORT, address.getPort());
}
- public static List<CustomCommandLine> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
- List<CustomCommandLine> customCommandLines = new ArrayList<>(2);
+ public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
+ List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
// Command line interface of the YARN session, with a special initialization here
// to prefix all options with y/yarn.
@@ -1082,8 +1088,8 @@ public class CliFrontend {
* @param commandLine The input to the command-line.
* @return custom command-line which is active (may only be one at a time)
*/
- public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) {
- for (CustomCommandLine cli : customCommandLines) {
+ public CustomCommandLine<?> getActiveCustomCommandLine(CommandLine commandLine) {
+ for (CustomCommandLine<?> cli : customCommandLines) {
if (cli.isActive(commandLine)) {
return cli;
}
@@ -1096,7 +1102,7 @@ public class CliFrontend {
* @param className The fully-qualified class name to load.
* @param params The constructor parameters
*/
- private static CustomCommandLine loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
+ private static CustomCommandLine<?> loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
Class<? extends CustomCommandLine> customCliClass =
Class.forName(className).asSubclass(CustomCommandLine.class);
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 10507d6..475d854 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -236,7 +236,7 @@ public class CliFrontendParser {
/**
* Prints the help for the client.
*/
- public static void printHelp(Collection<CustomCommandLine> customCommandLines) {
+ public static void printHelp(Collection<CustomCommandLine<?>> customCommandLines) {
System.out.println("./flink <ACTION> [OPTIONS] [ARGUMENTS]");
System.out.println();
System.out.println("The following actions are available:");
@@ -251,7 +251,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForRun(Collection<CustomCommandLine> customCommandLines) {
+ public static void printHelpForRun(Collection<CustomCommandLine<?>> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -279,7 +279,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForList(Collection<CustomCommandLine> customCommandLines) {
+ public static void printHelpForList(Collection<CustomCommandLine<?>> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -294,7 +294,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForStop(Collection<CustomCommandLine> customCommandLines) {
+ public static void printHelpForStop(Collection<CustomCommandLine<?>> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -309,7 +309,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForCancel(Collection<CustomCommandLine> customCommandLines) {
+ public static void printHelpForCancel(Collection<CustomCommandLine<?>> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -324,7 +324,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForSavepoint(Collection<CustomCommandLine> customCommandLines) {
+ public static void printHelpForSavepoint(Collection<CustomCommandLine<?>> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -345,7 +345,7 @@ public class CliFrontendParser {
* @param runOptions True if the run options should be printed, False to print only general options
*/
private static void printCustomCliOptions(
- Collection<CustomCommandLine> customCommandLines,
+ Collection<CustomCommandLine<?>> customCommandLines,
HelpFormatter formatter,
boolean runOptions) {
// prints options from all available command-line classes
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index aabc224..e939974 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -30,7 +30,7 @@ import javax.annotation.Nullable;
/**
* Custom command-line interface to load hooks for the command-line interface.
*/
-public interface CustomCommandLine {
+public interface CustomCommandLine<T> {
/**
* Signals whether the custom command-line wants to execute or not.
@@ -66,7 +66,7 @@ public interface CustomCommandLine {
* @return ClusterDescriptor
* @throws FlinkException if the ClusterDescriptor could not be created
*/
- ClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException;
+ ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) throws FlinkException;
/**
* Returns the cluster id if a cluster id was specified on the command line, otherwise it
@@ -79,7 +79,7 @@ public interface CustomCommandLine {
* @return Cluster id identifying the cluster to deploy jobs to or null
*/
@Nullable
- String getClusterId(CommandLine commandLine);
+ T getClusterId(CommandLine commandLine);
/**
* Returns the {@link ClusterSpecification} specified by the configuration and the command
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index 5660765..d2694b8 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -18,9 +18,9 @@
package org.apache.flink.client.cli;
-import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
+import org.apache.flink.client.deployment.StandaloneClusterId;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.FlinkException;
@@ -31,7 +31,7 @@ import javax.annotation.Nullable;
/**
* The default CLI which is used for interaction with standalone clusters.
*/
-public class DefaultCLI extends AbstractCustomCommandLine {
+public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterId> {
public DefaultCLI(Configuration configuration) {
super(configuration);
@@ -49,7 +49,7 @@ public class DefaultCLI extends AbstractCustomCommandLine {
}
@Override
- public ClusterDescriptor createClusterDescriptor(
+ public StandaloneClusterDescriptor createClusterDescriptor(
CommandLine commandLine) throws FlinkException {
final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
@@ -58,8 +58,8 @@ public class DefaultCLI extends AbstractCustomCommandLine {
@Override
@Nullable
- public String getClusterId(CommandLine commandLine) {
- return "standalone";
+ public StandaloneClusterId getClusterId(CommandLine commandLine) {
+ return StandaloneClusterId.getInstance();
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
index 1a75aac..c8fab72 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
@@ -18,9 +18,9 @@
package org.apache.flink.client.cli;
-import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor;
+import org.apache.flink.client.deployment.StandaloneClusterId;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.FlinkException;
@@ -33,7 +33,7 @@ import javax.annotation.Nullable;
/**
* The default CLI which is used for interaction with standalone clusters.
*/
-public class Flip6DefaultCLI extends AbstractCustomCommandLine {
+public class Flip6DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterId> {
public static final Option FLIP_6 = new Option("flip6", "Switches the client to Flip-6 mode.");
@@ -62,7 +62,7 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine {
}
@Override
- public ClusterDescriptor createClusterDescriptor(
+ public Flip6StandaloneClusterDescriptor createClusterDescriptor(
CommandLine commandLine) throws FlinkException {
final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
@@ -71,8 +71,8 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine {
@Override
@Nullable
- public String getClusterId(CommandLine commandLine) {
- return "flip6Standalone";
+ public StandaloneClusterId getClusterId(CommandLine commandLine) {
+ return StandaloneClusterId.getInstance();
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
index b1f566c..aadb5fe 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
@@ -24,8 +24,10 @@ import org.apache.flink.util.FlinkException;
/**
* A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
+ *
+ * @param <T> Type of the cluster id
*/
-public interface ClusterDescriptor extends AutoCloseable {
+public interface ClusterDescriptor<T> extends AutoCloseable {
/**
* Returns a String containing details about the cluster (NodeManagers, available memory, ...).
@@ -35,19 +37,19 @@ public interface ClusterDescriptor extends AutoCloseable {
/**
* Retrieves an existing Flink Cluster.
- * @param applicationID The unique application identifier of the running cluster
+ * @param clusterId The unique identifier of the running cluster
* @return Client for the cluster
- * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
+ * @throws ClusterRetrieveException if the cluster client could not be retrieved
*/
- ClusterClient retrieve(String applicationID) throws UnsupportedOperationException;
+ ClusterClient<T> retrieve(T clusterId) throws ClusterRetrieveException;
/**
* Triggers deployment of a cluster.
* @param clusterSpecification Cluster specification defining the cluster to deploy
* @return Client for the cluster
- * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
+ * @throws ClusterDeploymentException if the cluster could not be deployed
*/
- ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException;
+ ClusterClient<T> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException;
/**
* Deploys a per-job cluster with the given job on the cluster.
@@ -57,9 +59,9 @@ public interface ClusterDescriptor extends AutoCloseable {
* @return Cluster client to talk to the Flink cluster
* @throws ClusterDeploymentException if the cluster could not be deployed
*/
- ClusterClient deployJobCluster(
+ ClusterClient<T> deployJobCluster(
final ClusterSpecification clusterSpecification,
- final JobGraph jobGraph);
+ final JobGraph jobGraph) throws ClusterDeploymentException;
/**
* Terminates the cluster with the given cluster id.
@@ -67,5 +69,5 @@ public interface ClusterDescriptor extends AutoCloseable {
* @param clusterId identifying the cluster to shut down
* @throws FlinkException if the cluster could not be terminated
*/
- void terminateCluster(String clusterId) throws FlinkException;
+ void terminateCluster(T clusterId) throws FlinkException;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java
new file mode 100644
index 0000000..7234d1b
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.client.deployment;
+
+import org.apache.flink.util.FlinkException;
+
+/**
+ * Exception which indicates that a cluster could not be retrieved.
+ */
+public class ClusterRetrieveException extends FlinkException {
+
+ private static final long serialVersionUID = 7718062507419172318L;
+
+ public ClusterRetrieveException(String message) {
+ super(message);
+ }
+
+ public ClusterRetrieveException(Throwable cause) {
+ super(cause);
+ }
+
+ public ClusterRetrieveException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
index 70fd9f7..8096da8 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
@@ -28,7 +28,7 @@ import org.apache.flink.util.Preconditions;
/**
* A deployment descriptor for an existing cluster.
*/
-public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
+public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterId> {
private final Configuration config;
@@ -44,27 +44,27 @@ public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
}
@Override
- public RestClusterClient retrieve(String applicationID) {
+ public RestClusterClient<StandaloneClusterId> retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException {
try {
- return new RestClusterClient(config);
+ return new RestClusterClient<>(config, standaloneClusterId);
} catch (Exception e) {
- throw new RuntimeException("Couldn't retrieve FLIP-6 standalone cluster", e);
+ throw new ClusterRetrieveException("Couldn't retrieve FLIP-6 standalone cluster", e);
}
}
@Override
- public RestClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+ public RestClusterClient<StandaloneClusterId> deploySessionCluster(ClusterSpecification clusterSpecification) {
throw new UnsupportedOperationException("Can't deploy a FLIP-6 standalone cluster.");
}
@Override
- public RestClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+ public RestClusterClient<StandaloneClusterId> deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
throw new UnsupportedOperationException("Can't deploy a standalone FLIP-6 per-job cluster.");
}
@Override
- public void terminateCluster(String clusterId) throws FlinkException {
- throw new UnsupportedOperationException("Cannot terminate a standalone Flip-6 cluster.");
+ public void terminateCluster(StandaloneClusterId clusterId) throws FlinkException {
+ throw new UnsupportedOperationException("Cannot terminate a Flip-6 standalone cluster.");
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
index 5638232..62908fe 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.FlinkException;
/**
* A deployment descriptor for an existing cluster.
*/
-public class StandaloneClusterDescriptor implements ClusterDescriptor {
+public class StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterId> {
private final Configuration config;
@@ -43,16 +43,16 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor {
}
@Override
- public StandaloneClusterClient retrieve(String applicationID) {
+ public StandaloneClusterClient retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException {
try {
return new StandaloneClusterClient(config);
} catch (Exception e) {
- throw new RuntimeException("Couldn't retrieve standalone cluster", e);
+ throw new ClusterRetrieveException("Couldn't retrieve standalone cluster", e);
}
}
@Override
- public StandaloneClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+ public StandaloneClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) {
throw new UnsupportedOperationException("Can't deploy a standalone cluster.");
}
@@ -62,8 +62,8 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor {
}
@Override
- public void terminateCluster(String clusterId) throws FlinkException {
- throw new UnsupportedOperationException("Cannot terminate a standalone cluster.");
+ public void terminateCluster(StandaloneClusterId clusterId) throws FlinkException {
+ throw new UnsupportedOperationException("Cannot terminate standalone clusters.");
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java
new file mode 100644
index 0000000..7201f1d
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.client.deployment;
+
+/**
+ * Identifier for standalone clusters.
+ */
+public class StandaloneClusterId {
+ private static final StandaloneClusterId INSTANCE = new StandaloneClusterId();
+
+ private StandaloneClusterId() {}
+
+ public static StandaloneClusterId getInstance() {
+ return INSTANCE;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index efa23fb..b992e60 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -93,8 +93,10 @@ import scala.concurrent.duration.FiniteDuration;
/**
* Encapsulates the functionality necessary to submit a program to a remote cluster.
+ *
+ * @param <T> type of the cluster id
*/
-public abstract class ClusterClient {
+public abstract class ClusterClient<T> {
protected final Logger log = LoggerFactory.getLogger(getClass());
@@ -937,9 +939,11 @@ public abstract class ClusterClient {
public abstract List<String> getNewMessages();
/**
- * Returns a string representation of the cluster.
+ * Returns the cluster id identifying the cluster to which the client is connected.
+ *
+ * @return cluster id of the connected cluster
*/
- public abstract String getClusterIdentifier();
+ public abstract T getClusterId();
/**
* Set the mode of this client (detached or blocking job execution).
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
index 7e47825..13f49d3 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
@@ -35,7 +35,7 @@ import java.util.List;
*/
public class ContextEnvironment extends ExecutionEnvironment {
- protected final ClusterClient client;
+ protected final ClusterClient<?> client;
protected final List<URL> jarFilesToAttach;
@@ -45,7 +45,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
protected final SavepointRestoreSettings savepointSettings;
- public ContextEnvironment(ClusterClient remoteConnection, List<URL> jarFiles, List<URL> classpaths,
+ public ContextEnvironment(ClusterClient<?> remoteConnection, List<URL> jarFiles, List<URL> classpaths,
ClassLoader userCodeClassLoader, SavepointRestoreSettings savepointSettings) {
this.client = remoteConnection;
this.jarFilesToAttach = jarFiles;
@@ -84,7 +84,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
+ ") : " + getIdString();
}
- public ClusterClient getClient() {
+ public ClusterClient<?> getClient() {
return this.client;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
index 6209254..64b1863 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
@@ -33,7 +33,7 @@ import java.util.List;
*/
public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
- private final ClusterClient client;
+ private final ClusterClient<?> client;
private final List<URL> jarFilesToAttach;
@@ -49,7 +49,7 @@ public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
private SavepointRestoreSettings savepointSettings;
- public ContextEnvironmentFactory(ClusterClient client, List<URL> jarFilesToAttach,
+ public ContextEnvironmentFactory(ClusterClient<?> client, List<URL> jarFilesToAttach,
List<URL> classpathsToAttach, ClassLoader userCodeClassLoader, int defaultParallelism,
boolean isDetached, SavepointRestoreSettings savepointSettings) {
this.client = client;
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
index 63aa811..9c7b639 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
@@ -44,7 +44,7 @@ public class DetachedEnvironment extends ContextEnvironment {
private static final Logger LOG = LoggerFactory.getLogger(DetachedEnvironment.class);
public DetachedEnvironment(
- ClusterClient remoteConnection,
+ ClusterClient<?> remoteConnection,
List<URL> jarFiles,
List<URL> classpaths,
ClassLoader userCodeClassLoader,
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
index 1782a25..df08c30 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -19,6 +19,7 @@
package org.apache.flink.client.program;
import org.apache.flink.api.common.JobSubmissionResult;
+import org.apache.flink.client.deployment.StandaloneClusterId;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.WebOptions;
import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
@@ -38,7 +39,7 @@ import scala.concurrent.Future;
* Cluster client for communication with an standalone (on-premise) cluster or an existing cluster that has been
* brought up independently of a specific job.
*/
-public class StandaloneClusterClient extends ClusterClient {
+public class StandaloneClusterClient extends ClusterClient<StandaloneClusterId> {
public StandaloneClusterClient(Configuration config) throws Exception {
super(config);
@@ -81,9 +82,8 @@ public class StandaloneClusterClient extends ClusterClient {
}
@Override
- public String getClusterIdentifier() {
- // Avoid blocking here by getting the address from the config without resolving the address
- return "Standalone cluster with JobManager at " + this.getJobManagerAddress();
+ public StandaloneClusterId getClusterId() {
+ return StandaloneClusterId.getInstance();
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
index a6bff1a..3e8b136 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
@@ -59,6 +59,7 @@ import org.apache.flink.runtime.rest.messages.queue.QueueStatus;
import org.apache.flink.runtime.util.ExecutorThreadFactory;
import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedThrowable;
import javax.annotation.Nullable;
@@ -83,23 +84,31 @@ import static org.apache.flink.util.Preconditions.checkState;
/**
* A {@link ClusterClient} implementation that communicates via HTTP REST requests.
*/
-public class RestClusterClient extends ClusterClient {
+public class RestClusterClient<T> extends ClusterClient<T> {
private final RestClusterClientConfiguration restClusterClientConfiguration;
+
private final RestClient restClient;
+
private final ExecutorService executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("Flink-RestClusterClient-IO"));
private final WaitStrategy waitStrategy;
- public RestClusterClient(Configuration config) throws Exception {
- this(config, new ExponentialWaitStrategy(10, 2000));
+ private final T clusterId;
+
+ public RestClusterClient(Configuration config, T clusterId) throws Exception {
+ this(
+ config,
+ clusterId,
+ new ExponentialWaitStrategy(10L, 2000L));
}
@VisibleForTesting
- RestClusterClient(Configuration configuration, WaitStrategy waitStrategy) throws Exception {
+ RestClusterClient(Configuration configuration, T clusterId, WaitStrategy waitStrategy) throws Exception {
super(configuration);
this.restClusterClientConfiguration = RestClusterClientConfiguration.fromConfiguration(configuration);
this.restClient = new RestClient(restClusterClientConfiguration.getRestClientConfiguration(), executorService);
this.waitStrategy = requireNonNull(waitStrategy);
+ this.clusterId = Preconditions.checkNotNull(clusterId);
}
@Override
@@ -295,16 +304,16 @@ public class RestClusterClient extends ClusterClient {
});
}
+ @Override
+ public T getClusterId() {
+ return clusterId;
+ }
+
// ======================================
// Legacy stuff we actually implement
// ======================================
@Override
- public String getClusterIdentifier() {
- return "Flip-6 Standalone cluster with dispatcher at " + restClusterClientConfiguration.getRestServerAddress() + '.';
- }
-
- @Override
public boolean hasUserJarsInClassPath(List<URL> userJarFiles) {
return false;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
index 60bd308..b2fa003 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
@@ -52,7 +52,7 @@ public class CliFrontendCancelTest extends TestLogger {
JobID jid = new JobID();
String[] parameters = { jid.toString() };
- final ClusterClient clusterClient = createClusterClient();
+ final ClusterClient<String> clusterClient = createClusterClient();
MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
testFrontend.cancel(parameters);
@@ -90,7 +90,7 @@ public class CliFrontendCancelTest extends TestLogger {
JobID jid = new JobID();
String[] parameters = { "-s", jid.toString() };
- final ClusterClient clusterClient = createClusterClient();
+ final ClusterClient<String> clusterClient = createClusterClient();
MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
testFrontend.cancel(parameters);
@@ -103,7 +103,7 @@ public class CliFrontendCancelTest extends TestLogger {
JobID jid = new JobID();
String[] parameters = { "-s", "targetDirectory", jid.toString() };
- final ClusterClient clusterClient = createClusterClient();
+ final ClusterClient<String> clusterClient = createClusterClient();
MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
testFrontend.cancel(parameters);
@@ -134,8 +134,8 @@ public class CliFrontendCancelTest extends TestLogger {
testFrontend.cancel(parameters);
}
- private static ClusterClient createClusterClient() throws Exception {
- final ClusterClient clusterClient = mock(ClusterClient.class);
+ private static ClusterClient<String> createClusterClient() throws Exception {
+ final ClusterClient<String> clusterClient = mock(ClusterClient.class);
return clusterClient;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
index 77d8016..760b376 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
@@ -49,7 +49,7 @@ public class CliFrontendListTest extends TestLogger {
// test list properly
{
String[] parameters = {"-r", "-s"};
- ClusterClient clusterClient = createClusterClient();
+ ClusterClient<String> clusterClient = createClusterClient();
MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
testFrontend.list(parameters);
Mockito.verify(clusterClient, times(1))
@@ -67,8 +67,8 @@ public class CliFrontendListTest extends TestLogger {
testFrontend.list(parameters);
}
- private static ClusterClient createClusterClient() throws Exception {
- final ClusterClient clusterClient = mock(ClusterClient.class);
+ private static ClusterClient<String> createClusterClient() throws Exception {
+ final ClusterClient<String> clusterClient = mock(ClusterClient.class);
when(clusterClient.listJobs()).thenReturn(CompletableFuture.completedFuture(Collections.emptyList()));
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
index 7b31a3e..d730344 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
@@ -82,7 +82,7 @@ public class CliFrontendSavepointTest extends TestLogger {
String savepointPath = "expectedSavepointPath";
- final ClusterClient clusterClient = createClusterClient(savepointPath);
+ final ClusterClient<String> clusterClient = createClusterClient(savepointPath);
try {
MockedCliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -110,7 +110,7 @@ public class CliFrontendSavepointTest extends TestLogger {
String expectedTestException = "expectedTestException";
Exception testException = new Exception(expectedTestException);
- final ClusterClient clusterClient = createFailingClusterClient(testException);
+ final ClusterClient<String> clusterClient = createFailingClusterClient(testException);
try {
MockedCliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -165,7 +165,7 @@ public class CliFrontendSavepointTest extends TestLogger {
String savepointDirectory = "customTargetDirectory";
- final ClusterClient clusterClient = createClusterClient(savepointDirectory);
+ final ClusterClient<String> clusterClient = createClusterClient(savepointDirectory);
try {
MockedCliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -224,7 +224,7 @@ public class CliFrontendSavepointTest extends TestLogger {
final CompletableFuture<String> disposeSavepointFuture = new CompletableFuture<>();
- final ClusterClient clusterClient = new DisposeSavepointClusterClient(
+ final DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient(
(String savepointPath, Time timeout) -> {
disposeSavepointFuture.complete(savepointPath);
return CompletableFuture.completedFuture(Acknowledge.get());
@@ -260,7 +260,7 @@ public class CliFrontendSavepointTest extends TestLogger {
Exception testException = new Exception("expectedTestException");
- ClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException));
+ DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException));
try {
CliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -313,8 +313,8 @@ public class CliFrontendSavepointTest extends TestLogger {
System.setErr(stdErr);
}
- private static ClusterClient createClusterClient(String expectedResponse) throws Exception {
- final ClusterClient clusterClient = mock(ClusterClient.class);
+ private static ClusterClient<String> createClusterClient(String expectedResponse) throws Exception {
+ final ClusterClient<String> clusterClient = mock(ClusterClient.class);
when(clusterClient.triggerSavepoint(any(JobID.class), anyString()))
.thenReturn(CompletableFuture.completedFuture(expectedResponse));
@@ -322,8 +322,8 @@ public class CliFrontendSavepointTest extends TestLogger {
return clusterClient;
}
- private static ClusterClient createFailingClusterClient(Exception expectedException) throws Exception {
- final ClusterClient clusterClient = mock(ClusterClient.class);
+ private static ClusterClient<String> createFailingClusterClient(Exception expectedException) throws Exception {
+ final ClusterClient<String> clusterClient = mock(ClusterClient.class);
when(clusterClient.triggerSavepoint(any(JobID.class), anyString()))
.thenReturn(FutureUtils.completedExceptionally(expectedException));
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
index 0120cdf..d6049e5 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID;
import org.apache.flink.client.cli.util.MockedCliFrontend;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;
@@ -29,9 +30,12 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
+import javax.annotation.Nullable;
+
import java.util.Collections;
import static org.apache.flink.client.cli.CliFrontendTestUtils.pipeSystemOutToNull;
+import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
@@ -51,18 +55,16 @@ public class CliFrontendStopTest extends TestLogger {
@Test
public void testStop() throws Exception {
// test stop properly
- {
- JobID jid = new JobID();
- String jidString = jid.toString();
+ JobID jid = new JobID();
+ String jidString = jid.toString();
- String[] parameters = { jidString };
- final ClusterClient clusterClient = createClusterClient(false);
- MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
+ String[] parameters = { jidString };
+ final ClusterClient<String> clusterClient = createClusterClient(null);
+ MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
- testFrontend.stop(parameters);
+ testFrontend.stop(parameters);
- Mockito.verify(clusterClient, times(1)).stop(any(JobID.class));
- }
+ Mockito.verify(clusterClient, times(1)).stop(any(JobID.class));
}
@Test(expected = CliArgsException.class)
@@ -87,32 +89,30 @@ public class CliFrontendStopTest extends TestLogger {
testFrontend.stop(parameters);
}
- @Test(expected = TestException.class)
+ @Test
public void testUnknownJobId() throws Exception {
// test unknown job Id
JobID jid = new JobID();
String[] parameters = { jid.toString() };
- final ClusterClient clusterClient = createClusterClient(true);
+ String expectedMessage = "Test exception";
+ FlinkException testException = new FlinkException(expectedMessage);
+ final ClusterClient<String> clusterClient = createClusterClient(testException);
MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
- testFrontend.stop(parameters);
- fail("Should have failed.");
- }
-
- private static final class TestException extends FlinkException {
- private static final long serialVersionUID = -2650760898729937583L;
-
- TestException(String message) {
- super(message);
+ try {
+ testFrontend.stop(parameters);
+ fail("Should have failed.");
+ } catch (FlinkException e) {
+ assertTrue(ExceptionUtils.findThrowableWithMessage(e, expectedMessage).isPresent());
}
}
- private static ClusterClient createClusterClient(boolean reject) throws Exception {
- final ClusterClient clusterClient = mock(ClusterClient.class);
+ private static ClusterClient<String> createClusterClient(@Nullable Exception exception) throws Exception {
+ final ClusterClient<String> clusterClient = mock(ClusterClient.class);
- if (reject) {
- doThrow(new TestException("Test Exception")).when(clusterClient).stop(any(JobID.class));
+ if (exception != null) {
+ doThrow(exception).when(clusterClient).stop(any(JobID.class));
}
return clusterClient;
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
index 6eb005d..aaca798 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
@@ -18,7 +18,7 @@
package org.apache.flink.client.cli;
-import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
@@ -62,9 +62,9 @@ public class DefaultCLITest extends TestLogger {
final InetSocketAddress expectedAddress = new InetSocketAddress(localhost, port);
- final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+ final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
- final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
+ final ClusterClient<?> clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
Assert.assertEquals(expectedAddress, clusterClient.getJobManagerAddress());
}
@@ -89,9 +89,9 @@ public class DefaultCLITest extends TestLogger {
CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
- final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+ final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
- final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
+ final ClusterClient<?> clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
final InetSocketAddress expectedAddress = new InetSocketAddress(manualHostname, manualPort);
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
index 2bb3ed0..0ceb95e 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
@@ -28,11 +28,11 @@ import org.apache.flink.util.Preconditions;
/**
* Dummy {@link ClusterDescriptor} implementation for testing purposes.
*/
-public class DummyClusterDescriptor implements ClusterDescriptor {
+public class DummyClusterDescriptor<T> implements ClusterDescriptor<T> {
- private final ClusterClient clusterClient;
+ private final ClusterClient<T> clusterClient;
- public DummyClusterDescriptor(ClusterClient clusterClient) {
+ public DummyClusterDescriptor(ClusterClient<T> clusterClient) {
this.clusterClient = Preconditions.checkNotNull(clusterClient);
}
@@ -42,23 +42,23 @@ public class DummyClusterDescriptor implements ClusterDescriptor {
}
@Override
- public ClusterClient retrieve(String applicationID) throws UnsupportedOperationException {
+ public ClusterClient<T> retrieve(T clusterId) {
return clusterClient;
}
@Override
- public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+ public ClusterClient<T> deploySessionCluster(ClusterSpecification clusterSpecification) {
return clusterClient;
}
@Override
- public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+ public ClusterClient<T> deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
return clusterClient;
}
@Override
- public void terminateCluster(String clusterId) throws FlinkException {
- throw new UnsupportedOperationException("DummyClusterDescriptor does not support cluster termination.");
+ public void terminateCluster(T clusterId) throws FlinkException {
+ throw new UnsupportedOperationException("Cannot terminate a dummy cluster.");
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
index 5279d85..12bea74 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
@@ -32,10 +32,10 @@ import javax.annotation.Nullable;
/**
* Dummy implementation of the {@link CustomCommandLine} for testing purposes.
*/
-public class DummyCustomCommandLine implements CustomCommandLine {
- private final ClusterClient clusterClient;
+public class DummyCustomCommandLine<T> implements CustomCommandLine {
+ private final ClusterClient<T> clusterClient;
- public DummyCustomCommandLine(ClusterClient clusterClient) {
+ public DummyCustomCommandLine(ClusterClient<T> clusterClient) {
this.clusterClient = Preconditions.checkNotNull(clusterClient);
}
@@ -60,8 +60,8 @@ public class DummyCustomCommandLine implements CustomCommandLine {
}
@Override
- public ClusterDescriptor createClusterDescriptor(CommandLine commandLine) {
- return new DummyClusterDescriptor(clusterClient);
+ public ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) {
+ return new DummyClusterDescriptor<>(clusterClient);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
index c2505ae..72767c7 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
@@ -101,7 +101,7 @@ public class ClientConnectionTest extends TestLogger {
config.setString(JobManagerOptions.ADDRESS, unreachableEndpoint.getHostName());
config.setInteger(JobManagerOptions.PORT, unreachableEndpoint.getPort());
- ClusterClient client = new StandaloneClusterClient(config);
+ StandaloneClusterClient client = new StandaloneClusterClient(config);
try {
// we have to query the cluster status to start the connection attempts
@@ -140,7 +140,7 @@ public class ClientConnectionTest extends TestLogger {
highAvailabilityServices.setJobMasterLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID, testingLeaderRetrievalService);
- ClusterClient client = new StandaloneClusterClient(configuration, highAvailabilityServices);
+ StandaloneClusterClient client = new StandaloneClusterClient(configuration, highAvailabilityServices);
ActorGateway gateway = client.getJobManagerGateway();
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
index 99f51ad..ca8b6fb 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
@@ -131,7 +131,7 @@ public class ClientTest extends TestLogger {
jobManagerSystem.actorOf(
Props.create(SuccessReturningActor.class),
JobMaster.JOB_MANAGER_NAME);
- ClusterClient out = new StandaloneClusterClient(config);
+ StandaloneClusterClient out = new StandaloneClusterClient(config);
out.setDetached(true);
try {
@@ -204,7 +204,7 @@ public class ClientTest extends TestLogger {
Props.create(SuccessReturningActor.class),
JobMaster.JOB_MANAGER_NAME);
- ClusterClient out = new StandaloneClusterClient(config);
+ StandaloneClusterClient out = new StandaloneClusterClient(config);
out.setDetached(true);
JobSubmissionResult result = out.run(program.getPlanWithJars(), 1);
@@ -222,7 +222,7 @@ public class ClientTest extends TestLogger {
Props.create(FailureReturningActor.class),
JobMaster.JOB_MANAGER_NAME);
- ClusterClient out = new StandaloneClusterClient(config);
+ StandaloneClusterClient out = new StandaloneClusterClient(config);
out.setDetached(true);
try {
@@ -259,7 +259,7 @@ public class ClientTest extends TestLogger {
}).when(packagedProgramMock).invokeInteractiveModeForExecution();
try {
- ClusterClient client = new StandaloneClusterClient(config);
+ StandaloneClusterClient client = new StandaloneClusterClient(config);
client.setDetached(true);
client.run(packagedProgramMock, 1);
fail("Creating the local execution environment should not be possible");
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
index 7b34d4a..e2eb88d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
@@ -71,7 +71,7 @@ public class ClusterClientTest extends TestLogger {
Configuration config = new Configuration();
HighAvailabilityServices highAvailabilityServices = mock(HighAvailabilityServices.class);
- ClusterClient clusterClient = new StandaloneClusterClient(config, highAvailabilityServices);
+ StandaloneClusterClient clusterClient = new StandaloneClusterClient(config, highAvailabilityServices);
clusterClient.shutdown();
@@ -87,7 +87,7 @@ public class ClusterClientTest extends TestLogger {
JobID jobID = new JobID();
TestStopActorGateway gateway = new TestStopActorGateway(jobID);
- ClusterClient clusterClient = new TestClusterClient(config, gateway);
+ TestClusterClient clusterClient = new TestClusterClient(config, gateway);
try {
clusterClient.stop(jobID);
Assert.assertTrue(gateway.messageArrived);
@@ -103,7 +103,7 @@ public class ClusterClientTest extends TestLogger {
JobID jobID = new JobID();
TestCancelActorGateway gateway = new TestCancelActorGateway(jobID);
- ClusterClient clusterClient = new TestClusterClient(config, gateway);
+ TestClusterClient clusterClient = new TestClusterClient(config, gateway);
try {
clusterClient.cancel(jobID);
Assert.assertTrue(gateway.messageArrived);
@@ -121,7 +121,7 @@ public class ClusterClientTest extends TestLogger {
String savepointDirectory = "/test/directory";
String savepointPath = "/test/path";
TestCancelWithSavepointActorGateway gateway = new TestCancelWithSavepointActorGateway(jobID, savepointDirectory, savepointPath);
- ClusterClient clusterClient = new TestClusterClient(config, gateway);
+ TestClusterClient clusterClient = new TestClusterClient(config, gateway);
try {
String path = clusterClient.cancelWithSavepoint(jobID, savepointDirectory);
Assert.assertTrue(gateway.messageArrived);
@@ -140,7 +140,7 @@ public class ClusterClientTest extends TestLogger {
String savepointDirectory = "/test/directory";
String savepointPath = "/test/path";
TestSavepointActorGateway gateway = new TestSavepointActorGateway(jobID, savepointDirectory, savepointPath);
- ClusterClient clusterClient = new TestClusterClient(config, gateway);
+ TestClusterClient clusterClient = new TestClusterClient(config, gateway);
try {
CompletableFuture<String> pathFuture = clusterClient.triggerSavepoint(jobID, savepointDirectory);
Assert.assertTrue(gateway.messageArrived);
@@ -156,7 +156,7 @@ public class ClusterClientTest extends TestLogger {
config.setString(JobManagerOptions.ADDRESS, "localhost");
TestListActorGateway gateway = new TestListActorGateway();
- ClusterClient clusterClient = new TestClusterClient(config, gateway);
+ TestClusterClient clusterClient = new TestClusterClient(config, gateway);
try {
CompletableFuture<Collection<JobStatusMessage>> jobDetailsFuture = clusterClient.listJobs();
Collection<JobStatusMessage> jobDetails = jobDetailsFuture.get();
@@ -179,7 +179,7 @@ public class ClusterClientTest extends TestLogger {
final ActorGateway jobManagerGateway = new TestDisposeWithWrongResponseActorGateway();
- final ClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
+ final TestClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
CompletableFuture<Acknowledge> acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath, timeout);
@@ -201,7 +201,7 @@ public class ClusterClientTest extends TestLogger {
final ActorGateway jobManagerGateway = new TestDisposeWithClassNotFoundExceptionActorGateway();
- final ClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
+ final TestClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
CompletableFuture<Acknowledge> acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath, timeout);
[04/14] flink git commit: [FLINK-8349] [flip6] Remove Yarn specific
commands from YarnClusterDescriptor
Posted by tr...@apache.org.
[FLINK-8349] [flip6] Remove Yarn specific commands from YarnClusterDescriptor
Remove Yarn specific commands from YarnClusterDescriptor. This is a preparational
step to make the FlinkYarnSessionCli work with the Flip-6 RestClusterClient.
This closes #5229.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/402499f0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/402499f0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/402499f0
Branch: refs/heads/master
Commit: 402499f06a4b590ac47df64ecc01055c06b0399b
Parents: 10e900b
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Jan 11 23:25:58 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:03 2018 +0100
----------------------------------------------------------------------
.../apache/flink/client/cli/CliFrontend.java | 9 +
.../client/deployment/ClusterDescriptor.java | 9 +
.../Flip6StandaloneClusterDescriptor.java | 6 +
.../deployment/StandaloneClusterDescriptor.java | 6 +
.../flink/client/program/ClusterClient.java | 13 +-
.../client/program/StandaloneClusterClient.java | 4 -
.../client/program/rest/RestClusterClient.java | 7 +-
.../client/cli/util/DummyClusterDescriptor.java | 6 +
.../YARNSessionCapacitySchedulerITCase.java | 24 +-
.../flink/yarn/YARNSessionFIFOITCase.java | 9 +-
.../org/apache/flink/yarn/YarnTestBase.java | 59 ++-
.../yarn/AbstractYarnClusterDescriptor.java | 21 +-
.../apache/flink/yarn/YarnClusterClient.java | 155 +------
.../flink/yarn/cli/FlinkYarnSessionCli.java | 415 ++++++++++---------
.../yarn/cli/YarnApplicationStatusMonitor.java | 3 +-
15 files changed, 352 insertions(+), 394 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 453d086..847a5f8 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -232,6 +232,15 @@ public class CliFrontend {
executeProgram(program, client, userParallelism);
} finally {
+ if (clusterId == null && !client.isDetached()) {
+ // terminate the cluster only if we have started it before and if it's not detached
+ try {
+ clusterDescriptor.terminateCluster(client.getClusterIdentifier());
+ } catch (FlinkException e) {
+ LOG.info("Could not properly terminate the Flink cluster.", e);
+ }
+ }
+
try {
client.shutdown();
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
index 07eeabc..b1f566c 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
@@ -20,6 +20,7 @@ package org.apache.flink.client.deployment;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
/**
* A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
@@ -59,4 +60,12 @@ public interface ClusterDescriptor extends AutoCloseable {
ClusterClient deployJobCluster(
final ClusterSpecification clusterSpecification,
final JobGraph jobGraph);
+
+ /**
+ * Terminates the cluster with the given cluster id.
+ *
+ * @param clusterId identifying the cluster to shut down
+ * @throws FlinkException if the cluster could not be terminated
+ */
+ void terminateCluster(String clusterId) throws FlinkException;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
index a35a68b..70fd9f7 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
/**
@@ -62,6 +63,11 @@ public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
}
@Override
+ public void terminateCluster(String clusterId) throws FlinkException {
+ throw new UnsupportedOperationException("Cannot terminate a standalone Flip-6 cluster.");
+ }
+
+ @Override
public void close() throws Exception {
// nothing to do
}
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
index ae25194..5638232 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.program.StandaloneClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
/**
* A deployment descriptor for an existing cluster.
@@ -61,6 +62,11 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor {
}
@Override
+ public void terminateCluster(String clusterId) throws FlinkException {
+ throw new UnsupportedOperationException("Cannot terminate a standalone cluster.");
+ }
+
+ @Override
public void close() throws Exception {
// nothing to do
}
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index 7c05e3e..efa23fb 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -262,11 +262,7 @@ public abstract class ClusterClient {
*/
public void shutdown() throws Exception {
synchronized (this) {
- try {
- finalizeCluster();
- } finally {
- actorSystemLoader.shutdown();
- }
+ actorSystemLoader.shutdown();
if (highAvailabilityServices != null) {
highAvailabilityServices.close();
@@ -938,7 +934,7 @@ public abstract class ClusterClient {
* May return new messages from the cluster.
* Messages can be for example about failed containers or container launch requests.
*/
- protected abstract List<String> getNewMessages();
+ public abstract List<String> getNewMessages();
/**
* Returns a string representation of the cluster.
@@ -946,11 +942,6 @@ public abstract class ClusterClient {
public abstract String getClusterIdentifier();
/**
- * Request the cluster to shut down or disconnect.
- */
- protected abstract void finalizeCluster();
-
- /**
* Set the mode of this client (detached or blocking job execution).
* @param isDetached If true, the client will submit programs detached via the {@code run} method
*/
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
index 10e1bdd..1782a25 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -105,8 +105,4 @@ public class StandaloneClusterClient extends ClusterClient {
return super.run(jobGraph, classLoader);
}
}
-
- @Override
- protected void finalizeCluster() {}
-
}
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
index 6b21cff..a6bff1a 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
@@ -329,12 +329,7 @@ public class RestClusterClient extends ClusterClient {
}
@Override
- protected List<String> getNewMessages() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- protected void finalizeCluster() {
+ public List<String> getNewMessages() {
throw new UnsupportedOperationException();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
index d300055..2bb3ed0 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
/**
@@ -56,6 +57,11 @@ public class DummyClusterDescriptor implements ClusterDescriptor {
}
@Override
+ public void terminateCluster(String clusterId) throws FlinkException {
+ throw new UnsupportedOperationException("DummyClusterDescriptor does not support cluster termination.");
+ }
+
+ @Override
public void close() {
// nothing to do
}
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 5bed22e..c806c5e 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.taskexecutor.TaskManagerServices;
import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
import org.apache.flink.test.testdata.WordCountData;
import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
import org.apache.flink.yarn.configuration.YarnConfigOptions;
@@ -74,6 +75,7 @@ import java.util.concurrent.ConcurrentMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import static junit.framework.TestCase.assertTrue;
import static org.apache.flink.yarn.UtilsTest.addTestAppender;
import static org.apache.flink.yarn.UtilsTest.checkForLogString;
@@ -98,7 +100,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* Test regular operation, including command line parameter parsing.
*/
@Test
- public void testClientStartup() {
+ public void testClientStartup() throws IOException {
LOG.info("Starting testClientStartup()");
runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
"-n", "1",
@@ -116,7 +118,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* The parallelism is requested at the YARN client (-ys).
*/
@Test
- public void perJobYarnCluster() {
+ public void perJobYarnCluster() throws IOException {
LOG.info("Starting perJobYarnCluster()");
addTestAppender(JobClient.class, Level.INFO);
File exampleJarLocation = new File("target/programs/BatchWordCount.jar");
@@ -145,7 +147,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* memory remains.
*/
@Test
- public void perJobYarnClusterOffHeap() {
+ public void perJobYarnClusterOffHeap() throws IOException {
LOG.info("Starting perJobYarnCluster()");
addTestAppender(JobClient.class, Level.INFO);
File exampleJarLocation = new File("target/programs/BatchWordCount.jar");
@@ -363,15 +365,19 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* target queue. With an error message, we can help users identifying the issue)
*/
@Test
- public void testNonexistingQueueWARNmessage() {
+ public void testNonexistingQueueWARNmessage() throws IOException {
LOG.info("Starting testNonexistingQueueWARNmessage()");
addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
- runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
+ try {
+ runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
"-t", flinkLibFolder.getAbsolutePath(),
"-n", "1",
"-jm", "768",
"-tm", "1024",
"-qu", "doesntExist"}, "to unknown queue: doesntExist", null, RunTypes.YARN_SESSION, 1);
+ } catch (Exception e) {
+ assertTrue(ExceptionUtils.findThrowableWithMessage(e, "to unknown queue: doesntExist").isPresent());
+ }
checkForLogString("The specified queue 'doesntExist' does not exist. Available queues");
LOG.info("Finished testNonexistingQueueWARNmessage()");
}
@@ -380,7 +386,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* Test per-job yarn cluster with the parallelism set at the CliFrontend instead of the YARN client.
*/
@Test
- public void perJobYarnClusterWithParallelism() {
+ public void perJobYarnClusterWithParallelism() throws IOException {
LOG.info("Starting perJobYarnClusterWithParallelism()");
// write log messages to stdout as well, so that the runWithArgs() method
// is catching the log output
@@ -407,7 +413,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* Test a fire-and-forget job submission to a YARN cluster.
*/
@Test(timeout = 60000)
- public void testDetachedPerJobYarnCluster() {
+ public void testDetachedPerJobYarnCluster() throws IOException {
LOG.info("Starting testDetachedPerJobYarnCluster()");
File exampleJarLocation = new File("target/programs/BatchWordCount.jar");
@@ -423,7 +429,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
* Test a fire-and-forget job submission to a YARN cluster.
*/
@Test(timeout = 60000)
- public void testDetachedPerJobYarnClusterWithStreamingJob() {
+ public void testDetachedPerJobYarnClusterWithStreamingJob() throws IOException {
LOG.info("Starting testDetachedPerJobYarnClusterWithStreamingJob()");
File exampleJarLocation = new File("target/programs/StreamingWordCount.jar");
@@ -435,7 +441,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
LOG.info("Finished testDetachedPerJobYarnClusterWithStreamingJob()");
}
- private void testDetachedPerJobYarnClusterInternal(String job) {
+ private void testDetachedPerJobYarnClusterInternal(String job) throws IOException {
YarnClient yc = YarnClient.createYarnClient();
yc.init(YARN_CONFIGURATION);
yc.start();
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index cc26350..15fe355 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -45,6 +45,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
+import java.io.IOException;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.List;
@@ -80,7 +81,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
* Test regular operation, including command line parameter parsing.
*/
@Test(timeout = 60000) // timeout after a minute.
- public void testDetachedMode() throws InterruptedException {
+ public void testDetachedMode() throws InterruptedException, IOException {
LOG.info("Starting testDetachedMode()");
addTestAppender(FlinkYarnSessionCli.class, Level.INFO);
Runner runner =
@@ -158,7 +159,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
* <p>This test validates through 666*2 cores in the "cluster".
*/
@Test
- public void testQueryCluster() {
+ public void testQueryCluster() throws IOException {
LOG.info("Starting testQueryCluster()");
runWithArgs(new String[] {"-q"}, "Summary: totalMemory 8192 totalCores 1332", null, RunTypes.YARN_SESSION, 0); // we have 666*2 cores.
LOG.info("Finished testQueryCluster()");
@@ -178,7 +179,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
*/
@Ignore("The test is too resource consuming (8.5 GB of memory)")
@Test
- public void testResourceComputation() {
+ public void testResourceComputation() throws IOException {
addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
LOG.info("Starting testResourceComputation()");
runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
@@ -206,7 +207,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
*/
@Ignore("The test is too resource consuming (8 GB of memory)")
@Test
- public void testfullAlloc() {
+ public void testfullAlloc() throws IOException {
addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
LOG.info("Starting testfullAlloc()");
runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index ed02892..eeda32d 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -61,6 +61,9 @@ import java.io.FileNotFoundException;
import java.io.FileWriter;
import java.io.FilenameFilter;
import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
import java.io.PrintStream;
import java.io.PrintWriter;
import java.util.ArrayList;
@@ -86,6 +89,7 @@ public abstract class YarnTestBase extends TestLogger {
protected static final PrintStream ORIGINAL_STDOUT = System.out;
protected static final PrintStream ORIGINAL_STDERR = System.err;
+ private static final InputStream ORIGINAL_STDIN = System.in;
protected static final String TEST_CLUSTER_NAME_KEY = "flink-yarn-minicluster-name";
@@ -510,13 +514,18 @@ public abstract class YarnTestBase extends TestLogger {
/**
* This method returns once the "startedAfterString" has been seen.
*/
- protected Runner startWithArgs(String[] args, String startedAfterString, RunTypes type) {
+ protected Runner startWithArgs(String[] args, String startedAfterString, RunTypes type) throws IOException {
LOG.info("Running with args {}", Arrays.toString(args));
outContent = new ByteArrayOutputStream();
errContent = new ByteArrayOutputStream();
+ PipedOutputStream out = new PipedOutputStream();
+ PipedInputStream in = new PipedInputStream(out);
+ PrintStream stdinPrintStream = new PrintStream(out);
+
System.setOut(new PrintStream(outContent));
System.setErr(new PrintStream(errContent));
+ System.setIn(in);
final int startTimeoutSeconds = 60;
@@ -525,7 +534,8 @@ public abstract class YarnTestBase extends TestLogger {
flinkConfiguration,
CliFrontend.getConfigurationDirectoryFromEnv(),
type,
- 0);
+ 0,
+ stdinPrintStream);
runner.setName("Frontend (CLI/YARN Client) runner thread (startWithArgs()).");
runner.start();
@@ -539,7 +549,7 @@ public abstract class YarnTestBase extends TestLogger {
}
// check if thread died
if (!runner.isAlive()) {
- sendOutput();
+ resetStreamsAndSendOutput();
if (runner.getRunnerError() != null) {
throw new RuntimeException("Runner failed with exception.", runner.getRunnerError());
}
@@ -547,13 +557,13 @@ public abstract class YarnTestBase extends TestLogger {
}
}
- sendOutput();
+ resetStreamsAndSendOutput();
Assert.fail("During the timeout period of " + startTimeoutSeconds + " seconds the " +
"expected string did not show up");
return null;
}
- protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnStrings, RunTypes type, int returnCode) {
+ protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnStrings, RunTypes type, int returnCode) throws IOException {
runWithArgs(args, terminateAfterString, failOnStrings, type, returnCode, false);
}
@@ -566,13 +576,17 @@ public abstract class YarnTestBase extends TestLogger {
* @param expectedReturnValue Expected return code from the runner.
* @param checkLogForTerminateString If true, the runner checks also the log4j logger for the terminate string
*/
- protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnPatterns, RunTypes type, int expectedReturnValue, boolean checkLogForTerminateString) {
+ protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnPatterns, RunTypes type, int expectedReturnValue, boolean checkLogForTerminateString) throws IOException {
LOG.info("Running with args {}", Arrays.toString(args));
outContent = new ByteArrayOutputStream();
errContent = new ByteArrayOutputStream();
+ PipedOutputStream out = new PipedOutputStream();
+ PipedInputStream in = new PipedInputStream(out);
+ PrintStream stdinPrintStream = new PrintStream(out);
System.setOut(new PrintStream(outContent));
System.setErr(new PrintStream(errContent));
+ System.setIn(in);
// we wait for at most three minutes
final int startTimeoutSeconds = 180;
@@ -583,11 +597,13 @@ public abstract class YarnTestBase extends TestLogger {
flinkConfiguration,
CliFrontend.getConfigurationDirectoryFromEnv(),
type,
- expectedReturnValue);
+ expectedReturnValue,
+ stdinPrintStream);
runner.start();
boolean expectedStringSeen = false;
boolean testPassedFromLog4j = false;
+ long shutdownTimeout = 30000L;
do {
sleep(1000);
String outContentString = outContent.toString();
@@ -597,9 +613,15 @@ public abstract class YarnTestBase extends TestLogger {
Pattern pattern = Pattern.compile(failOnString);
if (pattern.matcher(outContentString).find() || pattern.matcher(errContentString).find()) {
LOG.warn("Failing test. Output contained illegal string '" + failOnString + "'");
- sendOutput();
+ resetStreamsAndSendOutput();
// stopping runner.
runner.sendStop();
+ // wait for the thread to stop
+ try {
+ runner.join(shutdownTimeout);
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while stopping runner", e);
+ }
Assert.fail("Output contained illegal string '" + failOnString + "'");
}
}
@@ -622,7 +644,7 @@ public abstract class YarnTestBase extends TestLogger {
runner.sendStop();
// wait for the thread to stop
try {
- runner.join(30000);
+ runner.join(shutdownTimeout);
}
catch (InterruptedException e) {
LOG.warn("Interrupted while stopping runner", e);
@@ -639,7 +661,7 @@ public abstract class YarnTestBase extends TestLogger {
}
while (runner.getRunnerError() == null && !expectedStringSeen && System.currentTimeMillis() < deadline);
- sendOutput();
+ resetStreamsAndSendOutput();
if (runner.getRunnerError() != null) {
// this lets the test fail.
@@ -651,9 +673,10 @@ public abstract class YarnTestBase extends TestLogger {
LOG.info("Test was successful");
}
- protected static void sendOutput() {
+ protected static void resetStreamsAndSendOutput() {
System.setOut(ORIGINAL_STDOUT);
System.setErr(ORIGINAL_STDERR);
+ System.setIn(ORIGINAL_STDIN);
LOG.info("Sending stdout content through logger: \n\n{}\n\n", outContent.toString());
LOG.info("Sending stderr content through logger: \n\n{}\n\n", errContent.toString());
@@ -668,6 +691,8 @@ public abstract class YarnTestBase extends TestLogger {
private final String configurationDirectory;
private final int expectedReturnValue;
+ private final PrintStream stdinPrintStream;
+
private RunTypes type;
private FlinkYarnSessionCli yCli;
private Throwable runnerError;
@@ -677,13 +702,15 @@ public abstract class YarnTestBase extends TestLogger {
org.apache.flink.configuration.Configuration configuration,
String configurationDirectory,
RunTypes type,
- int expectedReturnValue) {
+ int expectedReturnValue,
+ PrintStream stdinPrintStream) {
this.args = args;
this.configuration = Preconditions.checkNotNull(configuration);
this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
this.type = type;
this.expectedReturnValue = expectedReturnValue;
+ this.stdinPrintStream = Preconditions.checkNotNull(stdinPrintStream);
}
@Override
@@ -697,8 +724,8 @@ public abstract class YarnTestBase extends TestLogger {
configurationDirectory,
"",
"",
- false);
- returnValue = yCli.run(args, configuration, configurationDirectory);
+ true);
+ returnValue = yCli.run(args);
break;
case CLI_FRONTEND:
try {
@@ -727,9 +754,7 @@ public abstract class YarnTestBase extends TestLogger {
/** Stops the Yarn session. */
public void sendStop() {
- if (yCli != null) {
- yCli.stop();
- }
+ stdinPrintStream.println("stop");
}
public Throwable getRunnerError() {
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 0a977df..4affb78 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -21,6 +21,7 @@ package org.apache.flink.yarn;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
@@ -33,6 +34,7 @@ import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.clusterframework.BootstrapTools;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.yarn.configuration.YarnConfigOptions;
@@ -330,7 +332,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
// -------------------------------------------------------------
@Override
- public YarnClusterClient retrieve(String applicationID) {
+ public ClusterClient retrieve(String applicationID) {
try {
// check if required Hadoop environment variables are set. If not, warn user
@@ -393,6 +395,23 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
}
}
+ @Override
+ public void terminateCluster(String clusterId) throws FlinkException {
+ try {
+ yarnClient.killApplication(ConverterUtils.toApplicationId(clusterId));
+ } catch (IOException | YarnException e) {
+ throw new FlinkException("Could not terminate cluster with id " + clusterId + '.', e);
+ }
+ }
+
+ public void terminateCluster(ApplicationId applicationId) throws FlinkException {
+ try {
+ yarnClient.killApplication(applicationId);
+ } catch (YarnException | IOException e) {
+ throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e);
+ }
+ }
+
/**
* This method will block until the ApplicationMaster/JobManager have been
* deployed on YARN.
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index 5fb7f90..991b3b9 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -24,7 +24,6 @@ import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
@@ -33,8 +32,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
-import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
-import org.apache.flink.yarn.configuration.YarnConfigOptions;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
@@ -43,17 +40,15 @@ import akka.pattern.Patterns;
import akka.util.Timeout;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.File;
import java.io.IOException;
import java.net.URL;
import java.util.ArrayList;
import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
import scala.Option;
import scala.concurrent.Await;
@@ -67,8 +62,6 @@ public class YarnClusterClient extends ClusterClient {
private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class);
- private Thread clientShutdownHook = new ClientShutdownHook();
-
//---------- Class internal fields -------------------
private final AbstractYarnClusterDescriptor clusterDescriptor;
@@ -80,8 +73,6 @@ public class YarnClusterClient extends ClusterClient {
private final ApplicationId appId;
private final String trackingURL;
- private boolean isConnected = true;
-
/** Indicator whether this cluster has just been created. */
private final boolean newlyCreatedCluster;
@@ -120,32 +111,6 @@ public class YarnClusterClient extends ClusterClient {
flinkConfig,
actorSystemLoader,
highAvailabilityServices);
-
- Runtime.getRuntime().addShutdownHook(clientShutdownHook);
- }
-
- /**
- * Disconnect from the Yarn cluster.
- */
- public void disconnect() {
-
- if (hasBeenShutDown.getAndSet(true)) {
- return;
- }
-
- if (!isConnected) {
- throw new IllegalStateException("Can not disconnect from an unconnected cluster.");
- }
-
- LOG.info("Disconnecting YarnClusterClient from ApplicationMaster");
-
- try {
- Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
- } catch (IllegalStateException e) {
- // we are already in the shutdown hook
- }
-
- isConnected = false;
}
// -------------------------- Interaction with the cluster ------------------------
@@ -207,7 +172,7 @@ public class YarnClusterClient extends ClusterClient {
@Override
public String getClusterIdentifier() {
- return "Yarn cluster with application id " + appReport.getApplicationId();
+ return ConverterUtils.toString(appReport.getApplicationId());
}
/**
@@ -215,13 +180,6 @@ public class YarnClusterClient extends ClusterClient {
*/
@Override
public GetClusterStatusResponse getClusterStatus() {
- if (!isConnected) {
- throw new IllegalStateException("The cluster is not connected to the cluster.");
- }
- if (hasBeenShutdown()) {
- throw new IllegalStateException("The cluster has already been shutdown.");
- }
-
try {
final Future<Object> clusterStatusOption =
getJobManagerGateway().ask(
@@ -236,15 +194,7 @@ public class YarnClusterClient extends ClusterClient {
@Override
public List<String> getNewMessages() {
- if (hasBeenShutdown()) {
- throw new RuntimeException("The YarnClusterClient has already been stopped");
- }
-
- if (!isConnected) {
- throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
- }
-
- List<String> ret = new ArrayList<String>();
+ List<String> ret = new ArrayList<>();
// get messages from ApplicationClient (locally)
while (true) {
Object result;
@@ -283,105 +233,6 @@ public class YarnClusterClient extends ClusterClient {
return ret;
}
- // -------------------------- Shutdown handling ------------------------
-
- private AtomicBoolean hasBeenShutDown = new AtomicBoolean(false);
-
- /**
- * Shuts down or disconnects from the YARN cluster.
- */
- @Override
- public void finalizeCluster() {
- if (isDetached() || !newlyCreatedCluster) {
- disconnect();
- } else {
- shutdownCluster();
- }
- }
-
- /**
- * Shuts down the Yarn application.
- */
- public void shutdownCluster() {
-
- if (hasBeenShutDown.getAndSet(true)) {
- return;
- }
-
- if (!isConnected) {
- throw new IllegalStateException("The cluster has been not been connected to the ApplicationMaster.");
- }
-
- try {
- Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
- } catch (IllegalStateException e) {
- // we are already in the shutdown hook
- }
-
- LOG.info("Sending shutdown request to the Application Master");
- try {
- Future<Object> response =
- Patterns.ask(applicationClient.get(),
- new YarnMessages.LocalStopYarnSession(ApplicationStatus.CANCELED,
- "Flink YARN Client requested shutdown"),
- new Timeout(akkaDuration));
- Await.ready(response, akkaDuration);
- } catch (Exception e) {
- LOG.warn("Error while stopping YARN cluster.", e);
- }
-
- try {
- File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
- if (propertiesFile.isFile()) {
- if (propertiesFile.delete()) {
- LOG.info("Deleted Yarn properties file at {}", propertiesFile.getAbsoluteFile().toString());
- } else {
- LOG.warn("Couldn't delete Yarn properties file at {}", propertiesFile.getAbsoluteFile().toString());
- }
- }
- } catch (Exception e) {
- LOG.warn("Exception while deleting the JobManager address file", e);
- }
-
- try {
- ApplicationReport appReport = clusterDescriptor.getYarnClient().getApplicationReport(appId);
-
- LOG.info("Application " + appId + " finished with state " + appReport
- .getYarnApplicationState() + " and final state " + appReport
- .getFinalApplicationStatus() + " at " + appReport.getFinishTime());
-
- if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState()
- == YarnApplicationState.KILLED) {
- LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics());
- LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve "
- + "the full application log using this command:"
- + System.lineSeparator()
- + "\tyarn logs -applicationId " + appReport.getApplicationId()
- + System.lineSeparator()
- + "(It sometimes takes a few seconds until the logs are aggregated)");
- }
- } catch (Exception e) {
- LOG.warn("Couldn't get final report", e);
- }
- }
-
- public boolean hasBeenShutdown() {
- return hasBeenShutDown.get();
- }
-
- private class ClientShutdownHook extends Thread {
- @Override
- public void run() {
- LOG.info("Shutting down YarnClusterClient from the client shutdown hook");
-
- try {
- shutdown();
- } catch (Throwable t) {
- LOG.warn("Could not properly shut down the yarn cluster client.", t);
- }
- }
- }
-
@Override
public boolean isDetached() {
return super.isDetached() || clusterDescriptor.isDetachedMode();
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d797f47..75a270d 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -19,9 +19,11 @@
package org.apache.flink.yarn.cli;
import org.apache.flink.client.cli.AbstractCustomCommandLine;
+import org.apache.flink.client.cli.CliArgsException;
import org.apache.flink.client.cli.CliFrontend;
import org.apache.flink.client.cli.CliFrontendParser;
import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
@@ -36,20 +38,21 @@ import org.apache.flink.util.ExecutorUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
-import org.apache.flink.yarn.YarnClusterClient;
import org.apache.flink.yarn.YarnClusterDescriptor;
import org.apache.flink.yarn.YarnClusterDescriptorV2;
import org.apache.flink.yarn.configuration.YarnConfigOptions;
import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -73,8 +76,10 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_ID;
@@ -96,7 +101,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
// YARN-session related constants
private static final String YARN_PROPERTIES_FILE = ".yarn-properties-";
- static final String YARN_APPLICATION_ID_KEY = "applicationID";
+ private static final String YARN_APPLICATION_ID_KEY = "applicationID";
private static final String YARN_PROPERTIES_PARALLELISM = "parallelism";
private static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString";
@@ -148,8 +153,9 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
private final String yarnApplicationIdFromYarnProperties;
+ private final String yarnPropertiesFileLocation;
+
//------------------------------------ Internal fields -------------------------
- private YarnClusterClient yarnCluster;
private boolean detachedMode = false;
public FlinkYarnSessionCli(
@@ -181,7 +187,12 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
tmMemory = new Option(shortPrefix + "tm", longPrefix + "taskManagerMemory", true, "Memory per TaskManager Container [in MB]");
container = new Option(shortPrefix + "n", longPrefix + "container", true, "Number of YARN container to allocate (=Number of Task Managers)");
slots = new Option(shortPrefix + "s", longPrefix + "slots", true, "Number of slots per TaskManager");
- dynamicproperties = new Option(shortPrefix + "D", true, "Dynamic properties");
+ dynamicproperties = Option.builder(shortPrefix + "D")
+ .argName("property=value")
+ .numberOfArgs(2)
+ .valueSeparator()
+ .desc("use value for given property")
+ .build();
detached = new Option(shortPrefix + "d", longPrefix + "detached", false, "Start detached");
streaming = new Option(shortPrefix + "st", longPrefix + "streaming", false, "Start Flink in streaming mode");
name = new Option(shortPrefix + "nm", longPrefix + "name", true, "Set a custom name for the application on YARN");
@@ -206,7 +217,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
allOptions.addOption(flip6);
// try loading a potential yarn properties file
- final File yarnPropertiesLocation = getYarnPropertiesLocation(configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
+ this.yarnPropertiesFileLocation = configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION);
+ final File yarnPropertiesLocation = getYarnPropertiesLocation(yarnPropertiesFileLocation);
yarnPropertiesFile = new Properties();
@@ -216,7 +228,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
try (InputStream is = new FileInputStream(yarnPropertiesLocation)) {
yarnPropertiesFile.load(is);
} catch (IOException ioe) {
- throw new FlinkException("Could not read the Yarn properties file " + yarnPropertiesLocation + '.');
+ throw new FlinkException("Could not read the Yarn properties file " + yarnPropertiesLocation +
+ ". Please delete the file at " + yarnPropertiesLocation.getAbsolutePath() + '.', ioe);
}
yarnApplicationIdFromYarnProperties = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
@@ -305,10 +318,21 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
yarnClusterDescriptor.setQueue(cmd.getOptionValue(queue.getOpt()));
}
- String[] dynamicProperties = null;
- if (cmd.hasOption(dynamicproperties.getOpt())) {
- dynamicProperties = cmd.getOptionValues(dynamicproperties.getOpt());
- }
+ final Properties properties = cmd.getOptionProperties(dynamicproperties.getOpt());
+
+ String[] dynamicProperties = properties.stringPropertyNames().stream()
+ .flatMap(
+ (String key) -> {
+ final String value = properties.getProperty(key);
+
+ if (value != null) {
+ return Stream.of(key + dynamicproperties.getValueSeparator() + value);
+ } else {
+ return Stream.empty();
+ }
+ })
+ .toArray(String[]::new);
+
String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR);
yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded);
@@ -534,184 +558,168 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
return effectiveConfiguration;
}
- public int run(
- String[] args,
- Configuration configuration,
- String configurationDirectory) {
+ public int run(String[] args) throws CliArgsException, FlinkException {
//
// Command Line Options
//
- Options options = new Options();
- addGeneralOptions(options);
- addRunOptions(options);
+ final CommandLine cmd = parseCommandLineOptions(args, true);
- CommandLineParser parser = new PosixParser();
- CommandLine cmd;
- try {
- cmd = parser.parse(options, args);
- } catch (Exception e) {
- System.out.println(e.getMessage());
- printUsage();
- return 1;
- }
+ final AbstractYarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(cmd);
- // Query cluster for metrics
- if (cmd.hasOption(query.getOpt())) {
- AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor(
- configuration,
- configurationDirectory,
- cmd.hasOption(flip6.getOpt()));
- String description;
- try {
- description = yarnDescriptor.getClusterDescription();
- } catch (Exception e) {
- System.err.println("Error while querying the YARN cluster for available resources: " + e.getMessage());
- e.printStackTrace(System.err);
- return 1;
- }
- System.out.println(description);
- return 0;
- } else if (cmd.hasOption(applicationId.getOpt())) {
+ try {
+ // Query cluster for metrics
+ if (cmd.hasOption(query.getOpt())) {
+ final String description = yarnClusterDescriptor.getClusterDescription();
+ System.out.println(description);
+ return 0;
+ } else {
+ final ClusterClient clusterClient;
+ final ApplicationId yarnApplicationId;
- AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor(
- configuration,
- configurationDirectory,
- cmd.hasOption(flip6.getOpt()));
+ if (cmd.hasOption(applicationId.getOpt())) {
+ yarnApplicationId = ConverterUtils.toApplicationId(cmd.getOptionValue(applicationId.getOpt()));
- //configure ZK namespace depending on the value passed
- String zkNamespace = cmd.hasOption(zookeeperNamespace.getOpt()) ?
- cmd.getOptionValue(zookeeperNamespace.getOpt())
- : yarnDescriptor.getFlinkConfiguration()
- .getString(HA_CLUSTER_ID, cmd.getOptionValue(applicationId.getOpt()));
- LOG.info("Going to use the ZK namespace: {}", zkNamespace);
- yarnDescriptor.getFlinkConfiguration().setString(HA_CLUSTER_ID, zkNamespace);
+ clusterClient = yarnClusterDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
+ } else {
+ final ClusterSpecification clusterSpecification = getClusterSpecification(cmd);
- try {
- yarnCluster = yarnDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
- } catch (Exception e) {
- throw new RuntimeException("Could not retrieve existing Yarn application", e);
- }
+ clusterClient = yarnClusterDescriptor.deploySessionCluster(clusterSpecification);
- if (detachedMode) {
- LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
- "Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
- "yarn application -kill " + applicationId.getOpt());
- yarnCluster.disconnect();
- } else {
- ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
-
- try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
- yarnDescriptor.getYarnClient(),
- yarnCluster.getApplicationId(),
- new ScheduledExecutorServiceAdapter(scheduledExecutorService))) {
- runInteractiveCli(
- yarnCluster,
- yarnApplicationStatusMonitor,
- true);
- } finally {
- // shut down the scheduled executor service
- ExecutorUtils.gracefulShutdown(
- 1000L,
- TimeUnit.MILLISECONDS,
- scheduledExecutorService);
- }
- }
- } else {
+ //------------------ ClusterClient deployed, handle connection details
+ yarnApplicationId = ConverterUtils.toApplicationId(clusterClient.getClusterIdentifier());
- try (AbstractYarnClusterDescriptor yarnDescriptor = createClusterDescriptor(cmd)){
- final ClusterSpecification clusterSpecification;
+ String jobManagerAddress =
+ clusterClient.getJobManagerAddress().getAddress().getHostName() +
+ ':' + clusterClient.getJobManagerAddress().getPort();
- try {
- clusterSpecification = getClusterSpecification(cmd);
- } catch (FlinkException e) {
- System.err.println("Error while creating the cluster specification: " + e.getMessage());
- e.printStackTrace();
- return 1;
- }
+ System.out.println("Flink JobManager is now running on " + jobManagerAddress);
+ System.out.println("JobManager Web Interface: " + clusterClient.getWebInterfaceURL());
- try {
- yarnCluster = yarnDescriptor.deploySessionCluster(clusterSpecification);
- } catch (Exception e) {
- System.err.println("Error while deploying YARN cluster: " + e.getMessage());
- e.printStackTrace(System.err);
- return 1;
- }
- //------------------ ClusterClient deployed, handle connection details
- String jobManagerAddress =
- yarnCluster.getJobManagerAddress().getAddress().getHostName() +
- ":" + yarnCluster.getJobManagerAddress().getPort();
-
- System.out.println("Flink JobManager is now running on " + jobManagerAddress);
- System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL());
-
- // file that we write into the conf/ dir containing the jobManager address and the dop.
- File yarnPropertiesFile = getYarnPropertiesLocation(configuration.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
-
- Properties yarnProps = new Properties();
- yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnCluster.getApplicationId().toString());
- if (clusterSpecification.getSlotsPerTaskManager() != -1) {
- String parallelism =
- Integer.toString(clusterSpecification.getSlotsPerTaskManager() * clusterSpecification.getNumberTaskManagers());
- yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
- }
- // add dynamic properties
- if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
- yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
- yarnDescriptor.getDynamicPropertiesEncoded());
+ writeYarnPropertiesFile(
+ yarnApplicationId,
+ clusterSpecification.getNumberTaskManagers() * clusterSpecification.getSlotsPerTaskManager(),
+ yarnClusterDescriptor.getDynamicPropertiesEncoded());
}
- writeYarnProperties(yarnProps, yarnPropertiesFile);
-
- //------------------ ClusterClient running, let user control it ------------
if (detachedMode) {
- // print info and quit:
LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
- "yarn application -kill " + yarnCluster.getApplicationId());
- yarnCluster.waitForClusterToBeReady();
- yarnCluster.disconnect();
+ "yarn application -kill " + applicationId.getOpt());
} else {
+ ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
- ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+ final YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
+ yarnClusterDescriptor.getYarnClient(),
+ yarnApplicationId,
+ new ScheduledExecutorServiceAdapter(scheduledExecutorService));
- try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
- yarnDescriptor.getYarnClient(),
- yarnCluster.getApplicationId(),
- new ScheduledExecutorServiceAdapter(scheduledExecutorService))) {
+ try {
runInteractiveCli(
- yarnCluster,
+ clusterClient,
yarnApplicationStatusMonitor,
acceptInteractiveInput);
} finally {
+ try {
+ yarnApplicationStatusMonitor.close();
+ } catch (Exception e) {
+ LOG.info("Could not properly close the Yarn application status monitor.", e);
+ }
+
+ try {
+ clusterClient.shutdown();
+ } catch (Exception e) {
+ LOG.info("Could not properly shutdown cluster client.", e);
+ }
+
+ try {
+ yarnClusterDescriptor.terminateCluster(yarnApplicationId);
+ } catch (FlinkException e) {
+ LOG.info("Could not properly terminate the Flink cluster.", e);
+ }
+
// shut down the scheduled executor service
ExecutorUtils.gracefulShutdown(
1000L,
TimeUnit.MILLISECONDS,
scheduledExecutorService);
+
+ deleteYarnPropertiesFile();
+
+ try {
+ final ApplicationReport applicationReport = yarnClusterDescriptor
+ .getYarnClient()
+ .getApplicationReport(yarnApplicationId);
+
+ logFinalApplicationReport(applicationReport);
+ } catch (YarnException | IOException e) {
+ LOG.info("Could not log the final application report.", e);
+ }
}
}
- } catch (FlinkException e) {
- System.err.println("Error while deploying a Flink cluster: " + e.getMessage());
- e.printStackTrace();
- return 1;
+ }
+ } finally {
+ try {
+ yarnClusterDescriptor.close();
+ } catch (Exception e) {
+ LOG.info("Could not properly close the yarn cluster descriptor.", e);
}
}
+
return 0;
}
- /**
- * Utility method for tests.
- */
- public void stop() {
- if (yarnCluster != null) {
- LOG.info("Command line interface is shutting down the yarnCluster");
+ private void logFinalApplicationReport(ApplicationReport appReport) {
+ LOG.info("Application " + appReport.getApplicationId() + " finished with state " + appReport
+ .getYarnApplicationState() + " and final state " + appReport
+ .getFinalApplicationStatus() + " at " + appReport.getFinishTime());
+
+ if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState()
+ == YarnApplicationState.KILLED) {
+ LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics());
+ LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve "
+ + "the full application log using this command:"
+ + System.lineSeparator()
+ + "\tyarn logs -applicationId " + appReport.getApplicationId()
+ + System.lineSeparator()
+ + "(It sometimes takes a few seconds until the logs are aggregated)");
+ }
+ }
- try {
- yarnCluster.shutdown();
- } catch (Throwable t) {
- LOG.warn("Could not properly shutdown the yarn cluster.", t);
+ private void deleteYarnPropertiesFile() {
+ // try to clean up the old yarn properties file
+ try {
+ File propertiesFile = getYarnPropertiesLocation(yarnPropertiesFileLocation);
+ if (propertiesFile.isFile()) {
+ if (propertiesFile.delete()) {
+ LOG.info("Deleted Yarn properties file at {}", propertiesFile.getAbsoluteFile());
+ } else {
+ LOG.warn("Couldn't delete Yarn properties file at {}", propertiesFile.getAbsoluteFile());
+ }
}
+ } catch (Exception e) {
+ LOG.warn("Exception while deleting the JobManager address file", e);
+ }
+ }
+
+ private void writeYarnPropertiesFile(
+ ApplicationId yarnApplicationId,
+ int parallelism,
+ @Nullable String dynamicProperties) {
+ // file that we write into the conf/ dir containing the jobManager address and the dop.
+ final File yarnPropertiesFile = getYarnPropertiesLocation(yarnPropertiesFileLocation);
+
+ Properties yarnProps = new Properties();
+ yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnApplicationId.toString());
+ if (parallelism > 0) {
+ yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, Integer.toString(parallelism));
+ }
+
+ // add dynamic properties
+ if (dynamicProperties != null) {
+ yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING, dynamicProperties);
}
+
+ writeYarnProperties(yarnProps, yarnPropertiesFile);
}
private void logAndSysout(String message) {
@@ -719,28 +727,64 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
System.out.println(message);
}
- public static void main(final String[] args) throws Exception {
+ public static Map<String, String> getDynamicProperties(String dynamicPropertiesEncoded) {
+ if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) {
+ Map<String, String> properties = new HashMap<>();
+
+ String[] propertyLines = dynamicPropertiesEncoded.split(YARN_DYNAMIC_PROPERTIES_SEPARATOR);
+ for (String propLine : propertyLines) {
+ if (propLine == null) {
+ continue;
+ }
+
+ int firstEquals = propLine.indexOf("=");
+
+ if (firstEquals >= 0) {
+ String key = propLine.substring(0, firstEquals).trim();
+ String value = propLine.substring(firstEquals + 1, propLine.length()).trim();
+
+ if (!key.isEmpty()) {
+ properties.put(key, value);
+ }
+ }
+ }
+ return properties;
+ }
+ else {
+ return Collections.emptyMap();
+ }
+ }
+
+ public static void main(final String[] args) {
final String configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv();
final Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration();
- final FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
- flinkConfiguration,
- configurationDirectory,
- "",
- ""); // no prefix for the YARN session
+ int retCode;
+
+ try {
+ final FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
+ flinkConfiguration,
+ configurationDirectory,
+ "",
+ ""); // no prefix for the YARN session
- SecurityUtils.install(new SecurityConfiguration(flinkConfiguration));
+ SecurityUtils.install(new SecurityConfiguration(flinkConfiguration));
- final int retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args, flinkConfiguration, configurationDirectory));
+ retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args));
+ } catch (CliArgsException e) {
+ retCode = handleCliArgsException(e);
+ } catch (Exception e) {
+ retCode = handleError(e);
+ }
System.exit(retCode);
}
private static void runInteractiveCli(
- YarnClusterClient clusterClient,
- YarnApplicationStatusMonitor yarnApplicationStatusMonitor,
- boolean readConsoleInput) {
+ ClusterClient clusterClient,
+ YarnApplicationStatusMonitor yarnApplicationStatusMonitor,
+ boolean readConsoleInput) {
try (BufferedReader in = new BufferedReader(new InputStreamReader(System.in))) {
boolean continueRepl = true;
int numTaskmanagers = 0;
@@ -799,7 +843,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
}
}
- private static void printClusterMessages(YarnClusterClient clusterClient) {
+ private static void printClusterMessages(ClusterClient clusterClient) {
final List<String> messages = clusterClient.getNewMessages();
if (!messages.isEmpty()) {
System.err.println("New messages from the YARN cluster: ");
@@ -819,8 +863,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
* @throws InterruptedException
*/
private static boolean repStep(
- BufferedReader in,
- boolean readConsoleInput) throws IOException, InterruptedException {
+ BufferedReader in,
+ boolean readConsoleInput) throws IOException, InterruptedException {
// wait until CLIENT_POLLING_INTERVAL is over or the user entered something.
long startTime = System.currentTimeMillis();
@@ -859,32 +903,25 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
propertiesFile.setReadable(true, false); // readable for all.
}
- public static Map<String, String> getDynamicProperties(String dynamicPropertiesEncoded) {
- if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) {
- Map<String, String> properties = new HashMap<>();
+ private static int handleCliArgsException(CliArgsException e) {
+ LOG.error("Could not parse the command line arguments.", e);
- String[] propertyLines = dynamicPropertiesEncoded.split(YARN_DYNAMIC_PROPERTIES_SEPARATOR);
- for (String propLine : propertyLines) {
- if (propLine == null) {
- continue;
- }
+ System.out.println(e.getMessage());
+ System.out.println();
+ System.out.println("Use the help option (-h or --help) to get help on the command.");
+ return 1;
+ }
- int firstEquals = propLine.indexOf("=");
+ private static int handleError(Exception e) {
+ LOG.error("Error while running the Flink Yarn session.", e);
- if (firstEquals >= 0) {
- String key = propLine.substring(0, firstEquals).trim();
- String value = propLine.substring(firstEquals + 1, propLine.length()).trim();
+ System.err.println();
+ System.err.println("------------------------------------------------------------");
+ System.err.println(" The program finished with the following exception:");
+ System.err.println();
- if (!key.isEmpty()) {
- properties.put(key, value);
- }
- }
- }
- return properties;
- }
- else {
- return Collections.emptyMap();
- }
+ e.printStackTrace();
+ return 1;
}
public static File getYarnPropertiesLocation(@Nullable String yarnPropertiesFileLocation) {
@@ -902,7 +939,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
return new File(propertiesFileLocation, YARN_PROPERTIES_FILE + currentUser);
}
- protected AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) {
+ private static AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) {
final YarnClient yarnClient = YarnClient.createYarnClient();
if (flip6) {
return new YarnClusterDescriptorV2(configuration, configurationDirectory, yarnClient);
http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
index 88d7747..f96b581 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
@@ -59,7 +59,7 @@ public class YarnApplicationStatusMonitor implements AutoCloseable {
applicationStatusUpdateFuture = scheduledExecutor.scheduleWithFixedDelay(
this::updateApplicationStatus,
- UPDATE_INTERVAL,
+ 0L,
UPDATE_INTERVAL,
TimeUnit.MILLISECONDS);
@@ -83,6 +83,7 @@ public class YarnApplicationStatusMonitor implements AutoCloseable {
applicationReport = yarnClient.getApplicationReport(yarnApplicationId);
} catch (Exception e) {
LOG.info("Could not retrieve the Yarn application report for {}.", yarnApplicationId);
+ applicationStatus = ApplicationStatus.UNKNOWN;
return;
}
[13/14] flink git commit: [FLINK-8420] [flip6] Recognize
TimeoutException in RetryingRegistration
Posted by tr...@apache.org.
[FLINK-8420] [flip6] Recognize TimeoutException in RetryingRegistration
A timeout exception will trigger an exponential backoff wrt the connection timeout.
This will guarantee that we don't overload the network with connection requests but
also to quickly connect to a newly available target.
This closes #5286.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3c99ae89
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3c99ae89
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3c99ae89
Branch: refs/heads/master
Commit: 3c99ae8959f69325bb9b7d810b41c60e42e602c5
Parents: d0bc300
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Jan 12 14:02:05 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 17:54:10 2018 +0100
----------------------------------------------------------------------
.../registration/RetryingRegistration.java | 3 +-
.../registration/RetryingRegistrationTest.java | 48 ++++++++++++++++----
2 files changed, 42 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/3c99ae89/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
index 802d361..279714b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.rpc.FencedRpcGateway;
import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.ExceptionUtils;
import org.slf4j.Logger;
@@ -263,7 +264,7 @@ public abstract class RetryingRegistration<F extends Serializable, G extends Rpc
registrationAcceptFuture.whenCompleteAsync(
(Void v, Throwable failure) -> {
if (failure != null && !isCanceled()) {
- if (failure instanceof TimeoutException) {
+ if (ExceptionUtils.stripCompletionException(failure) instanceof TimeoutException) {
// we simply have not received a response in time. maybe the timeout was
// very low (initial fast registration attempts), maybe the target endpoint is
// currently down.
http://git-wip-us.apache.org/repos/asf/flink/blob/3c99ae89/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
index 7fc6897..885a7f5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
@@ -160,7 +160,7 @@ public class RetryingRegistrationTest extends TestLogger {
}
}
- @Test
+ @Test(timeout = 10000)
public void testRetriesOnTimeouts() throws Exception {
final String testId = "rien ne va plus";
final String testEndpointAddress = "<test-address>";
@@ -178,7 +178,15 @@ public class RetryingRegistrationTest extends TestLogger {
try {
rpc.registerGateway(testEndpointAddress, testGateway);
- TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+ final long initialTimeout = 20L;
+ TestRetryingRegistration registration = new TestRetryingRegistration(
+ rpc,
+ testEndpointAddress,
+ leaderId,
+ initialTimeout,
+ 1000L,
+ 15000L, // make sure that we timeout in case of an error
+ 15000L);
long started = System.nanoTime();
registration.startRegistration();
@@ -195,7 +203,7 @@ public class RetryingRegistrationTest extends TestLogger {
assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
// validate that some retry-delay / back-off behavior happened
- assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
+ assertTrue("retries did not properly back off", elapsedMillis >= 3 * initialTimeout);
}
finally {
rpc.stopService();
@@ -346,11 +354,35 @@ public class RetryingRegistrationTest extends TestLogger {
static final long DELAY_ON_DECLINE = 200;
public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
- super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
- rpc, "TestEndpoint",
- TestRegistrationGateway.class,
- targetAddress, leaderId,
- INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
+ this(
+ rpc,
+ targetAddress,
+ leaderId,
+ INITIAL_TIMEOUT,
+ MAX_TIMEOUT,
+ DELAY_ON_ERROR,
+ DELAY_ON_DECLINE);
+ }
+
+ public TestRetryingRegistration(
+ RpcService rpc,
+ String targetAddress,
+ UUID leaderId,
+ long initialTimeout,
+ long maxTimeout,
+ long delayOnError,
+ long delayOnDecline) {
+ super(
+ LoggerFactory.getLogger(RetryingRegistrationTest.class),
+ rpc,
+ "TestEndpoint",
+ TestRegistrationGateway.class,
+ targetAddress,
+ leaderId,
+ initialTimeout,
+ maxTimeout,
+ delayOnError,
+ delayOnDecline);
}
@Override
[09/14] flink git commit: [FLINK-8348] [flip6] Print help for
DefaultCLI
Posted by tr...@apache.org.
[FLINK-8348] [flip6] Print help for DefaultCLI
This closes #5233.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7d986ce0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7d986ce0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7d986ce0
Branch: refs/heads/master
Commit: 7d986ce0482562efd90fec416c4b2f4638a4058a
Parents: 38d3720
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Dec 29 15:29:06 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:05 2018 +0100
----------------------------------------------------------------------
.../apache/flink/client/cli/CliFrontendParser.java | 16 +++++++---------
.../org/apache/flink/client/cli/DefaultCLI.java | 2 +-
2 files changed, 8 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/7d986ce0/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 475d854..402b4ae 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -350,16 +350,14 @@ public class CliFrontendParser {
boolean runOptions) {
// prints options from all available command-line classes
for (CustomCommandLine cli: customCommandLines) {
- if (cli.getId() != null) {
- formatter.setSyntaxPrefix(" Options for " + cli.getId() + " mode:");
- Options customOpts = new Options();
- cli.addGeneralOptions(customOpts);
- if (runOptions) {
- cli.addRunOptions(customOpts);
- }
- formatter.printHelp(" ", customOpts);
- System.out.println();
+ formatter.setSyntaxPrefix(" Options for " + cli.getId() + " mode:");
+ Options customOpts = new Options();
+ cli.addGeneralOptions(customOpts);
+ if (runOptions) {
+ cli.addRunOptions(customOpts);
}
+ formatter.printHelp(" ", customOpts);
+ System.out.println();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7d986ce0/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index d2694b8..2ca5a65 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -45,7 +45,7 @@ public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterId> {
@Override
public String getId() {
- return null;
+ return "Default CLI";
}
@Override
[14/14] flink git commit: [FLINK-8288] [runtime] register job master
rest endpoint url to yarn
Posted by tr...@apache.org.
[FLINK-8288] [runtime] register job master rest endpoint url to yarn
This closes #5186.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3b016868
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3b016868
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3b016868
Branch: refs/heads/master
Commit: 3b01686851e8281642924c4a620fb43b008de174
Parents: 3c99ae8
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Wed Dec 20 10:10:35 2017 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 17:55:01 2018 +0100
----------------------------------------------------------------------
.../entrypoint/JobClusterEntrypoint.java | 20 ++++++++++----------
1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/3b016868/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
index 21a7ba4..cb1b086 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
@@ -92,16 +92,6 @@ public abstract class JobClusterEntrypoint extends ClusterEntrypoint {
HeartbeatServices heartbeatServices,
MetricRegistry metricRegistry) throws Exception {
- resourceManager = createResourceManager(
- configuration,
- ResourceID.generate(),
- rpcService,
- highAvailabilityServices,
- heartbeatServices,
- metricRegistry,
- this,
- null);
-
jobManagerServices = JobManagerServices.fromConfiguration(configuration, blobServer);
resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever();
@@ -134,6 +124,16 @@ public abstract class JobClusterEntrypoint extends ClusterEntrypoint {
LOG.debug("Starting JobMaster REST endpoint.");
jobMasterRestEndpoint.start();
+ resourceManager = createResourceManager(
+ configuration,
+ ResourceID.generate(),
+ rpcService,
+ highAvailabilityServices,
+ heartbeatServices,
+ metricRegistry,
+ this,
+ jobMasterRestEndpoint.getRestAddress());
+
jobManagerRunner = createJobManagerRunner(
configuration,
ResourceID.generate(),
[03/14] flink git commit: [FLINK-8340] [flip6] Remove passing of
Configuration to CustomCommandLine
Posted by tr...@apache.org.
[FLINK-8340] [flip6] Remove passing of Configuration to CustomCommandLine
Since the Configuration does not change over the lifetime of a CustomCommandLine,
we can safely pass it as a constructor argument instead of method argument.
This closes #5226.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/30011b9b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/30011b9b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/30011b9b
Branch: refs/heads/master
Commit: 30011b9b110aad0e1c28e7e0a025b73986781a72
Parents: 4ade826
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Jan 3 13:25:09 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:02 2018 +0100
----------------------------------------------------------------------
.../client/cli/AbstractCustomCommandLine.java | 16 +-
.../apache/flink/client/cli/CliFrontend.java | 84 ++--
.../flink/client/cli/CustomCommandLine.java | 23 +-
.../org/apache/flink/client/cli/DefaultCLI.java | 17 +-
.../flink/client/cli/Flip6DefaultCLI.java | 17 +-
.../flink/client/cli/CliFrontendCancelTest.java | 24 +-
.../flink/client/cli/CliFrontendInfoTest.java | 24 +-
.../flink/client/cli/CliFrontendListTest.java | 6 +-
.../cli/CliFrontendPackageProgramTest.java | 251 +++++-------
.../flink/client/cli/CliFrontendRunTest.java | 34 +-
.../flink/client/cli/CliFrontendStopTest.java | 12 +-
.../apache/flink/client/cli/DefaultCLITest.java | 49 +--
.../flink/client/cli/Flip6DefaultCLITest.java | 4 +-
.../client/cli/util/DummyCustomCommandLine.java | 12 +-
.../client/cli/util/MockedCliFrontend.java | 4 +-
.../flink/client/program/ClusterClientTest.java | 4 +
.../org/apache/flink/api/scala/FlinkShell.scala | 27 +-
.../YARNSessionCapacitySchedulerITCase.java | 4 +-
.../flink/yarn/YARNSessionFIFOITCase.java | 5 +-
.../org/apache/flink/yarn/YarnTestBase.java | 5 +-
.../apache/flink/yarn/YarnClusterClient.java | 3 +-
.../flink/yarn/cli/FlinkYarnSessionCli.java | 389 ++++++++++---------
.../flink/yarn/FlinkYarnSessionCliTest.java | 121 +++---
23 files changed, 541 insertions(+), 594 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
index 70ffc5c..c7a1672 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
@@ -22,6 +22,9 @@ import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
@@ -47,6 +50,16 @@ public abstract class AbstractCustomCommandLine<C extends ClusterClient> impleme
"Address of the JobManager (master) to which to connect. " +
"Use this flag to connect to a different JobManager than the one specified in the configuration.");
+ protected final Configuration configuration;
+
+ protected AbstractCustomCommandLine(Configuration configuration) {
+ this.configuration = new UnmodifiableConfiguration(Preconditions.checkNotNull(configuration));
+ }
+
+ public Configuration getConfiguration() {
+ return configuration;
+ }
+
@Override
public void addRunOptions(Options baseOptions) {
// nothing to add here
@@ -61,11 +74,10 @@ public abstract class AbstractCustomCommandLine<C extends ClusterClient> impleme
/**
* Override configuration settings by specified command line options.
*
- * @param configuration to use as the base configuration
* @param commandLine containing the overriding values
* @return Effective configuration with the overriden configuration settings
*/
- protected Configuration applyCommandLineOptionsToConfiguration(Configuration configuration, CommandLine commandLine) {
+ protected Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException {
final Configuration resultingConfiguration = new Configuration(configuration);
if (commandLine.hasOption(addressOption.getOpt())) {
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 4b5040e..630154c 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -108,8 +108,6 @@ public class CliFrontend {
private final List<CustomCommandLine<?>> customCommandLines;
- private final String configurationDirectory;
-
private final Options customCommandLineOptions;
private final FiniteDuration clientTimeout;
@@ -118,11 +116,9 @@ public class CliFrontend {
public CliFrontend(
Configuration configuration,
- List<CustomCommandLine<?>> customCommandLines,
- String configurationDirectory) throws Exception {
+ List<CustomCommandLine<?>> customCommandLines) throws Exception {
this.configuration = Preconditions.checkNotNull(configuration);
this.customCommandLines = Preconditions.checkNotNull(customCommandLines);
- this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
try {
FileSystem.initialize(this.configuration);
@@ -139,9 +135,9 @@ public class CliFrontend {
}
this.clientTimeout = AkkaUtils.getClientTimeout(this.configuration);
- this.defaultParallelism = GlobalConfiguration.loadConfiguration().getInteger(
- ConfigConstants.DEFAULT_PARALLELISM_KEY,
- ConfigConstants.DEFAULT_PARALLELISM);
+ this.defaultParallelism = configuration.getInteger(
+ ConfigConstants.DEFAULT_PARALLELISM_KEY,
+ ConfigConstants.DEFAULT_PARALLELISM);
}
// --------------------------------------------------------------------------------------------
@@ -161,15 +157,6 @@ public class CliFrontend {
return copiedConfiguration;
}
- /**
- * Returns the configuration directory for the CLI frontend.
- *
- * @return Configuration directory
- */
- public String getConfigurationDirectory() {
- return configurationDirectory;
- }
-
// --------------------------------------------------------------------------------------------
// Execute Actions
// --------------------------------------------------------------------------------------------
@@ -211,20 +198,17 @@ public class CliFrontend {
final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
try {
- final String clusterId = customCommandLine.getClusterId(configuration, commandLine);
+ final String clusterId = customCommandLine.getClusterId(commandLine);
final ClusterClient client;
if (clusterId != null) {
client = clusterDescriptor.retrieve(clusterId);
} else {
- final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(configuration, commandLine);
+ final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine);
client = clusterDescriptor.deploySessionCluster(clusterSpecification);
}
@@ -368,14 +352,9 @@ public class CliFrontend {
}
final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
- final String clusterId = activeCommandLine.getClusterId(
- configuration,
- commandLine);
+ final String clusterId = activeCommandLine.getClusterId(commandLine);
if (clusterId == null) {
throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -496,12 +475,9 @@ public class CliFrontend {
final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
- final String clusterId = activeCommandLine.getClusterId(configuration, commandLine);
+ final String clusterId = activeCommandLine.getClusterId(commandLine);
if (clusterId == null) {
throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -579,12 +555,9 @@ public class CliFrontend {
final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
- final String clusterId = activeCommandLine.getClusterId(configuration, commandLine);
+ final String clusterId = activeCommandLine.getClusterId(commandLine);
if (clusterId == null) {
throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -646,14 +619,9 @@ public class CliFrontend {
CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
- final String clusterId = customCommandLine.getClusterId(
- configuration,
- commandLine);
+ final String clusterId = customCommandLine.getClusterId(commandLine);
if (clusterId == null) {
throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -1004,13 +972,14 @@ public class CliFrontend {
final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory);
// 3. load the custom command lines
- final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines();
+ final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines(
+ configuration,
+ configurationDirectory);
try {
final CliFrontend cli = new CliFrontend(
configuration,
- customCommandLines,
- configurationDirectory);
+ customCommandLines);
SecurityUtils.install(new SecurityConfiguration(cli.configuration));
int retCode = SecurityUtils.getInstalledContext()
@@ -1070,7 +1039,7 @@ public class CliFrontend {
config.setInteger(JobManagerOptions.PORT, address.getPort());
}
- public static List<CustomCommandLine<?>> loadCustomCommandLines() {
+ public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
// Command line interface of the YARN session, with a special initialization here
@@ -1079,13 +1048,18 @@ public class CliFrontend {
// active CustomCommandLine in order and DefaultCLI isActive always return true.
final String flinkYarnSessionCLI = "org.apache.flink.yarn.cli.FlinkYarnSessionCli";
try {
- customCommandLines.add(loadCustomCommandLine(flinkYarnSessionCLI, "y", "yarn"));
+ customCommandLines.add(
+ loadCustomCommandLine(flinkYarnSessionCLI,
+ configuration,
+ configurationDirectory,
+ "y",
+ "yarn"));
} catch (Exception e) {
LOG.warn("Could not load CLI class {}.", flinkYarnSessionCLI, e);
}
- customCommandLines.add(new Flip6DefaultCLI());
- customCommandLines.add(new DefaultCLI());
+ customCommandLines.add(new Flip6DefaultCLI(configuration));
+ customCommandLines.add(new DefaultCLI(configuration));
return customCommandLines;
}
@@ -1101,7 +1075,7 @@ public class CliFrontend {
*/
public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) {
for (CustomCommandLine cli : customCommandLines) {
- if (cli.isActive(commandLine, configuration)) {
+ if (cli.isActive(commandLine)) {
return cli;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index cfa0827..f642484 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -21,7 +21,7 @@ package org.apache.flink.client.cli;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.FlinkException;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Options;
@@ -36,10 +36,9 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
/**
* Signals whether the custom command-line wants to execute or not.
* @param commandLine The command-line options
- * @param configuration The Flink configuration
* @return True if the command-line wants to run, False otherwise
*/
- boolean isActive(CommandLine commandLine, Configuration configuration);
+ boolean isActive(CommandLine commandLine);
/**
* Gets the unique identifier of this CustomCommandLine.
@@ -55,6 +54,7 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
/**
* Adds custom options to the existing general options.
+ *
* @param baseOptions The existing options.
*/
void addGeneralOptions(Options baseOptions);
@@ -63,15 +63,11 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
* Create a {@link ClusterDescriptor} from the given configuration, configuration directory
* and the command line.
*
- * @param configuration to create the ClusterDescriptor with
- * @param configurationDirectory where the configuration was loaded from
* @param commandLine containing command line options relevant for the ClusterDescriptor
* @return ClusterDescriptor
+ * @throws FlinkException if the ClusterDescriptor could not be created
*/
- ClusterDescriptor<ClusterType> createClusterDescriptor(
- Configuration configuration,
- String configurationDirectory,
- CommandLine commandLine);
+ ClusterDescriptor<ClusterType> createClusterDescriptor(CommandLine commandLine) throws FlinkException;
/**
* Returns the cluster id if a cluster id was specified on the command line, otherwise it
@@ -80,24 +76,21 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
* <p>A cluster id identifies a running cluster, e.g. the Yarn application id for a Flink
* cluster running on Yarn.
*
- * @param configuration to be used for the cluster id retrieval
* @param commandLine containing command line options relevant for the cluster id retrieval
* @return Cluster id identifying the cluster to deploy jobs to or null
*/
@Nullable
- String getClusterId(Configuration configuration, CommandLine commandLine);
+ String getClusterId(CommandLine commandLine);
/**
* Returns the {@link ClusterSpecification} specified by the configuration and the command
* line options. This specification can be used to deploy a new Flink cluster.
*
- * @param configuration to be used for the ClusterSpecification values
* @param commandLine containing command line options relevant for the ClusterSpecification
* @return ClusterSpecification for a new Flink cluster
+ * @throws FlinkException if the ClusterSpecification could not be created
*/
- ClusterSpecification getClusterSpecification(
- Configuration configuration,
- CommandLine commandLine);
+ ClusterSpecification getClusterSpecification(CommandLine commandLine) throws FlinkException;
default CommandLine parseCommandLineOptions(String[] args, boolean stopAtNonOptions) throws CliArgsException {
final Options options = new Options();
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index d34e307..c29c5b7 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
import org.apache.flink.client.program.StandaloneClusterClient;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.FlinkException;
import org.apache.commons.cli.CommandLine;
@@ -33,8 +34,12 @@ import javax.annotation.Nullable;
*/
public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClient> {
+ public DefaultCLI(Configuration configuration) {
+ super(configuration);
+ }
+
@Override
- public boolean isActive(CommandLine commandLine, Configuration configuration) {
+ public boolean isActive(CommandLine commandLine) {
// always active because we can try to read a JobManager address from the config
return true;
}
@@ -46,22 +51,20 @@ public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClien
@Override
public ClusterDescriptor<StandaloneClusterClient> createClusterDescriptor(
- Configuration configuration,
- String configurationDirectory,
- CommandLine commandLine) {
- final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration, commandLine);
+ CommandLine commandLine) throws FlinkException {
+ final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
return new StandaloneClusterDescriptor(effectiveConfiguration);
}
@Override
@Nullable
- public String getClusterId(Configuration configuration, CommandLine commandLine) {
+ public String getClusterId(CommandLine commandLine) {
return "standalone";
}
@Override
- public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
+ public ClusterSpecification getClusterSpecification(CommandLine commandLine) {
return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
index 0e80f44..3adeca6 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor;
import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.FlinkException;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
@@ -41,8 +42,12 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient
FLIP_6.setRequired(false);
}
+ public Flip6DefaultCLI(Configuration configuration) {
+ super(configuration);
+ }
+
@Override
- public boolean isActive(CommandLine commandLine, Configuration configuration) {
+ public boolean isActive(CommandLine commandLine) {
return commandLine.hasOption(FLIP_6.getOpt());
}
@@ -59,22 +64,20 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient
@Override
public ClusterDescriptor<RestClusterClient> createClusterDescriptor(
- Configuration configuration,
- String configurationDirectory,
- CommandLine commandLine) {
- final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration, commandLine);
+ CommandLine commandLine) throws FlinkException {
+ final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
return new Flip6StandaloneClusterDescriptor(effectiveConfiguration);
}
@Override
@Nullable
- public String getClusterId(Configuration configuration, CommandLine commandLine) {
+ public String getClusterId(CommandLine commandLine) {
return "flip6Standalone";
}
@Override
- public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
+ public ClusterSpecification getClusterSpecification(CommandLine commandLine) {
return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
index b0a308d..60bd308 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
@@ -63,20 +63,20 @@ public class CliFrontendCancelTest extends TestLogger {
@Test(expected = CliArgsException.class)
public void testMissingJobId() throws Exception {
String[] parameters = {};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.cancel(parameters);
}
@Test(expected = CliArgsException.class)
public void testUnrecognizedOption() throws Exception {
String[] parameters = {"-v", "-l"};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.cancel(parameters);
}
@@ -116,10 +116,10 @@ public class CliFrontendCancelTest extends TestLogger {
public void testCancelWithSavepointWithoutJobId() throws Exception {
// Cancel with savepoint (with target directory), but no job ID
String[] parameters = { "-s", "targetDirectory" };
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.cancel(parameters);
}
@@ -127,10 +127,10 @@ public class CliFrontendCancelTest extends TestLogger {
public void testCancelWithSavepointWithoutParameters() throws Exception {
// Cancel with savepoint (no target directory) and no job ID
String[] parameters = { "-s" };
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.cancel(parameters);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
index fa212cb..c284c61 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
@@ -42,20 +42,20 @@ public class CliFrontendInfoTest extends TestLogger {
@Test(expected = CliArgsException.class)
public void testMissingOption() throws Exception {
String[] parameters = {};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.cancel(parameters);
}
@Test(expected = CliArgsException.class)
public void testUnrecognizedOption() throws Exception {
String[] parameters = {"-v", "-l"};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.cancel(parameters);
}
@@ -65,10 +65,10 @@ public class CliFrontendInfoTest extends TestLogger {
try {
String[] parameters = new String[]{CliFrontendTestUtils.getTestJarPath(), "-f", "true"};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.info(parameters);
assertTrue(buffer.toString().contains("\"parallelism\": \"1\""));
}
@@ -82,10 +82,10 @@ public class CliFrontendInfoTest extends TestLogger {
replaceStdOut();
try {
String[] parameters = {"-p", "17", CliFrontendTestUtils.getTestJarPath()};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.info(parameters);
assertTrue(buffer.toString().contains("\"parallelism\": \"17\""));
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
index 6639e25..77d8016 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
@@ -60,10 +60,10 @@ public class CliFrontendListTest extends TestLogger {
@Test(expected = CliArgsException.class)
public void testUnrecognizedOption() throws Exception {
String[] parameters = {"-v", "-k"};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.list(parameters);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
index a862dc1..6873e68 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
@@ -22,7 +22,6 @@ import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.CompilerException;
import org.apache.flink.optimizer.DataStatistics;
import org.apache.flink.optimizer.Optimizer;
import org.apache.flink.optimizer.costs.DefaultCostEstimator;
@@ -64,189 +63,145 @@ public class CliFrontendPackageProgramTest extends TestLogger {
@Before
public void setup() throws Exception {
+ final Configuration configuration = new Configuration();
frontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
}
@Test
- public void testNonExistingJarFile() {
- try {
- ProgramOptions options = mock(ProgramOptions.class);
- when(options.getJarFilePath()).thenReturn("/some/none/existing/path");
+ public void testNonExistingJarFile() throws Exception {
+ ProgramOptions options = mock(ProgramOptions.class);
+ when(options.getJarFilePath()).thenReturn("/some/none/existing/path");
- try {
- frontend.buildProgram(options);
- fail("should throw an exception");
- }
- catch (FileNotFoundException e) {
- // that's what we want
- }
+ try {
+ frontend.buildProgram(options);
+ fail("should throw an exception");
}
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
+ catch (FileNotFoundException e) {
+ // that's what we want
}
}
@Test
- public void testFileNotJarFile() {
- try {
- ProgramOptions options = mock(ProgramOptions.class);
- when(options.getJarFilePath()).thenReturn(getNonJarFilePath());
+ public void testFileNotJarFile() throws Exception {
+ ProgramOptions options = mock(ProgramOptions.class);
+ when(options.getJarFilePath()).thenReturn(getNonJarFilePath());
- try {
- frontend.buildProgram(options);
- fail("should throw an exception");
- }
- catch (ProgramInvocationException e) {
- // that's what we want
- }
+ try {
+ frontend.buildProgram(options);
+ fail("should throw an exception");
}
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
+ catch (ProgramInvocationException e) {
+ // that's what we want
}
}
@Test
- public void testVariantWithExplicitJarAndArgumentsOption() {
- try {
- String[] arguments = {
- "--classpath", "file:///tmp/foo",
- "--classpath", "file:///tmp/bar",
- "-j", getTestJarPath(),
- "-a", "--debug", "true", "arg1", "arg2" };
- URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
- String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
-
- RunOptions options = CliFrontendParser.parseRunCommand(arguments);
- assertEquals(getTestJarPath(), options.getJarFilePath());
- assertArrayEquals(classpath, options.getClasspaths().toArray());
- assertArrayEquals(reducedArguments, options.getProgramArgs());
-
- PackagedProgram prog = frontend.buildProgram(options);
-
- Assert.assertArrayEquals(reducedArguments, prog.getArguments());
- Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
- }
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
+ public void testVariantWithExplicitJarAndArgumentsOption() throws Exception {
+ String[] arguments = {
+ "--classpath", "file:///tmp/foo",
+ "--classpath", "file:///tmp/bar",
+ "-j", getTestJarPath(),
+ "-a", "--debug", "true", "arg1", "arg2" };
+ URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+ String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
+
+ RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+ assertEquals(getTestJarPath(), options.getJarFilePath());
+ assertArrayEquals(classpath, options.getClasspaths().toArray());
+ assertArrayEquals(reducedArguments, options.getProgramArgs());
+
+ PackagedProgram prog = frontend.buildProgram(options);
+
+ Assert.assertArrayEquals(reducedArguments, prog.getArguments());
+ Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
}
@Test
- public void testVariantWithExplicitJarAndNoArgumentsOption() {
- try {
- String[] arguments = {
- "--classpath", "file:///tmp/foo",
- "--classpath", "file:///tmp/bar",
- "-j", getTestJarPath(),
- "--debug", "true", "arg1", "arg2" };
- URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
- String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
-
- RunOptions options = CliFrontendParser.parseRunCommand(arguments);
- assertEquals(getTestJarPath(), options.getJarFilePath());
- assertArrayEquals(classpath, options.getClasspaths().toArray());
- assertArrayEquals(reducedArguments, options.getProgramArgs());
-
- PackagedProgram prog = frontend.buildProgram(options);
-
- Assert.assertArrayEquals(reducedArguments, prog.getArguments());
- Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
- }
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
+ public void testVariantWithExplicitJarAndNoArgumentsOption() throws Exception {
+ String[] arguments = {
+ "--classpath", "file:///tmp/foo",
+ "--classpath", "file:///tmp/bar",
+ "-j", getTestJarPath(),
+ "--debug", "true", "arg1", "arg2" };
+ URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+ String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
+
+ RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+ assertEquals(getTestJarPath(), options.getJarFilePath());
+ assertArrayEquals(classpath, options.getClasspaths().toArray());
+ assertArrayEquals(reducedArguments, options.getProgramArgs());
+
+ PackagedProgram prog = frontend.buildProgram(options);
+
+ Assert.assertArrayEquals(reducedArguments, prog.getArguments());
+ Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
}
@Test
- public void testValidVariantWithNoJarAndNoArgumentsOption() {
- try {
- String[] arguments = {
- "--classpath", "file:///tmp/foo",
- "--classpath", "file:///tmp/bar",
- getTestJarPath(),
- "--debug", "true", "arg1", "arg2" };
- URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
- String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
-
- RunOptions options = CliFrontendParser.parseRunCommand(arguments);
- assertEquals(getTestJarPath(), options.getJarFilePath());
- assertArrayEquals(classpath, options.getClasspaths().toArray());
- assertArrayEquals(reducedArguments, options.getProgramArgs());
-
- PackagedProgram prog = frontend.buildProgram(options);
-
- Assert.assertArrayEquals(reducedArguments, prog.getArguments());
- Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
- }
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
+ public void testValidVariantWithNoJarAndNoArgumentsOption() throws Exception {
+ String[] arguments = {
+ "--classpath", "file:///tmp/foo",
+ "--classpath", "file:///tmp/bar",
+ getTestJarPath(),
+ "--debug", "true", "arg1", "arg2" };
+ URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+ String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
+
+ RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+ assertEquals(getTestJarPath(), options.getJarFilePath());
+ assertArrayEquals(classpath, options.getClasspaths().toArray());
+ assertArrayEquals(reducedArguments, options.getProgramArgs());
+
+ PackagedProgram prog = frontend.buildProgram(options);
+
+ Assert.assertArrayEquals(reducedArguments, prog.getArguments());
+ Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
}
@Test(expected = CliArgsException.class)
public void testNoJarNoArgumentsAtAll() throws Exception {
frontend.run(new String[0]);
-
- fail("Should have failed.");
}
@Test
- public void testNonExistingFileWithArguments() {
- try {
- String[] arguments = {
- "--classpath", "file:///tmp/foo",
- "--classpath", "file:///tmp/bar",
- "/some/none/existing/path",
- "--debug", "true", "arg1", "arg2" };
- URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
- String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
+ public void testNonExistingFileWithArguments() throws Exception {
+ String[] arguments = {
+ "--classpath", "file:///tmp/foo",
+ "--classpath", "file:///tmp/bar",
+ "/some/none/existing/path",
+ "--debug", "true", "arg1", "arg2" };
+ URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+ String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
+
+ RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+ assertEquals(arguments[4], options.getJarFilePath());
+ assertArrayEquals(classpath, options.getClasspaths().toArray());
+ assertArrayEquals(reducedArguments, options.getProgramArgs());
- RunOptions options = CliFrontendParser.parseRunCommand(arguments);
- assertEquals(arguments[4], options.getJarFilePath());
- assertArrayEquals(classpath, options.getClasspaths().toArray());
- assertArrayEquals(reducedArguments, options.getProgramArgs());
-
- try {
- frontend.buildProgram(options);
- fail("Should fail with an exception");
- }
- catch (FileNotFoundException e) {
- // that's what we want
- }
+ try {
+ frontend.buildProgram(options);
+ fail("Should fail with an exception");
}
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
+ catch (FileNotFoundException e) {
+ // that's what we want
}
}
@Test
- public void testNonExistingFileWithoutArguments() {
- try {
- String[] arguments = {"/some/none/existing/path"};
+ public void testNonExistingFileWithoutArguments() throws Exception {
+ String[] arguments = {"/some/none/existing/path"};
- RunOptions options = CliFrontendParser.parseRunCommand(arguments);
- assertEquals(arguments[0], options.getJarFilePath());
- assertArrayEquals(new String[0], options.getProgramArgs());
+ RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+ assertEquals(arguments[0], options.getJarFilePath());
+ assertArrayEquals(new String[0], options.getProgramArgs());
- try {
- frontend.buildProgram(options);
- }
- catch (FileNotFoundException e) {
- // that's what we want
- }
+ try {
+ frontend.buildProgram(options);
}
- catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
+ catch (FileNotFoundException e) {
+ // that's what we want
}
}
@@ -284,7 +239,7 @@ public class CliFrontendPackageProgramTest extends TestLogger {
* </ul>
*/
@Test
- public void testPlanWithExternalClass() throws CompilerException, ProgramInvocationException {
+ public void testPlanWithExternalClass() throws Exception {
final boolean[] callme = { false }; // create a final object reference, to be able to change its val later
try {
@@ -334,9 +289,5 @@ public class CliFrontendPackageProgramTest extends TestLogger {
}
assertTrue("Classloader was not called", callme[0]);
}
- catch (Exception e) {
- e.printStackTrace();
- fail("Program failed with the wrong exception: " + e.getClass().getName());
- }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
index fa1c76c..ebb76d8 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
@@ -46,31 +46,32 @@ public class CliFrontendRunTest {
@Test
public void testRun() throws Exception {
+ final Configuration configuration = GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir());
// test without parallelism
{
String[] parameters = {"-v", getTestJarPath()};
- RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(1, true, false);
+ RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 1, true, false);
testFrontend.run(parameters);
}
// test configure parallelism
{
String[] parameters = {"-v", "-p", "42", getTestJarPath()};
- RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(42, true, false);
+ RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 42, true, false);
testFrontend.run(parameters);
}
// test configure sysout logging
{
String[] parameters = {"-p", "2", "-q", getTestJarPath()};
- RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false, false);
+ RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 2, false, false);
testFrontend.run(parameters);
}
// test detached mode
{
String[] parameters = {"-p", "2", "-d", getTestJarPath()};
- RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, true, true);
+ RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 2, true, true);
testFrontend.run(parameters);
}
@@ -111,10 +112,10 @@ public class CliFrontendRunTest {
public void testUnrecognizedOption() throws Exception {
// test unrecognized option
String[] parameters = {"-v", "-l", "-a", "some", "program", "arguments"};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.run(parameters);
}
@@ -122,10 +123,10 @@ public class CliFrontendRunTest {
public void testInvalidParallelismOption() throws Exception {
// test configure parallelism with non integer value
String[] parameters = {"-v", "-p", "text", getTestJarPath()};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.run(parameters);
}
@@ -133,10 +134,10 @@ public class CliFrontendRunTest {
public void testParallelismWithOverflow() throws Exception {
// test configure parallelism with overflow integer value
String[] parameters = {"-v", "-p", "475871387138", getTestJarPath()};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.run(parameters);
}
@@ -148,11 +149,10 @@ public class CliFrontendRunTest {
private final boolean sysoutLogging;
private final boolean isDetached;
- public RunTestingCliFrontend(int expectedParallelism, boolean logging, boolean isDetached) throws Exception {
+ public RunTestingCliFrontend(Configuration configuration, int expectedParallelism, boolean logging, boolean isDetached) throws Exception {
super(
- GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir()),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
this.expectedParallelism = expectedParallelism;
this.sysoutLogging = logging;
this.isDetached = isDetached;
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
index fbbc739..0120cdf 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
@@ -69,10 +69,10 @@ public class CliFrontendStopTest extends TestLogger {
public void testUnrecognizedOption() throws Exception {
// test unrecognized option
String[] parameters = { "-v", "-l" };
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.stop(parameters);
}
@@ -80,10 +80,10 @@ public class CliFrontendStopTest extends TestLogger {
public void testMissingJobId() throws Exception {
// test missing job id
String[] parameters = {};
+ Configuration configuration = new Configuration();
CliFrontend testFrontend = new CliFrontend(
- new Configuration(),
- Collections.singletonList(new DefaultCLI()),
- CliFrontendTestUtils.getConfigDir());
+ configuration,
+ Collections.singletonList(new DefaultCLI(configuration)));
testFrontend.stop(parameters);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
index 3f05f90..e73b9c9 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
@@ -46,30 +46,25 @@ public class DefaultCLITest extends TestLogger {
*/
@Test
public void testConfigurationPassing() throws Exception {
- final DefaultCLI defaultCLI = new DefaultCLI();
-
- final String configurationDirectory = temporaryFolder.newFolder().getAbsolutePath();
- final String[] args = {};
-
- CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
+ final Configuration configuration = new Configuration();
final String localhost = "localhost";
final int port = 1234;
- final Configuration configuration = new Configuration();
configuration.setString(JobManagerOptions.ADDRESS, localhost);
configuration.setInteger(JobManagerOptions.PORT, port);
+ final DefaultCLI defaultCLI = new DefaultCLI(configuration);
+
+ final String[] args = {};
+
+ CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
+
final InetSocketAddress expectedAddress = new InetSocketAddress(localhost, port);
- final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
- final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(
- configuration,
- commandLine));
+ final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
Assert.assertEquals(expectedAddress, clusterClient.getJobManagerAddress());
}
@@ -79,15 +74,6 @@ public class DefaultCLITest extends TestLogger {
*/
@Test
public void testManualConfigurationOverride() throws Exception {
- final DefaultCLI defaultCLI = new DefaultCLI();
-
- final String manualHostname = "123.123.123.123";
- final int manualPort = 4321;
- final String configurationDirectory = temporaryFolder.newFolder().getAbsolutePath();
- final String[] args = {"-m", manualHostname + ':' + manualPort};
-
- CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
-
final String localhost = "localhost";
final int port = 1234;
final Configuration configuration = new Configuration();
@@ -95,14 +81,17 @@ public class DefaultCLITest extends TestLogger {
configuration.setString(JobManagerOptions.ADDRESS, localhost);
configuration.setInteger(JobManagerOptions.PORT, port);
- final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(
- configuration,
- configurationDirectory,
- commandLine);
+ 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 ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
- final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(
- configuration,
- commandLine));
+ final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
final InetSocketAddress expectedAddress = new InetSocketAddress(manualHostname, manualPort);
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
index 6a538d4..c3299bc 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
@@ -37,7 +37,7 @@ public class Flip6DefaultCLITest extends TestLogger {
@Test
public void testFlip6Switch() throws CliArgsException {
final String[] args = {"-flip6"};
- final Flip6DefaultCLI flip6DefaultCLI = new Flip6DefaultCLI();
+ final Flip6DefaultCLI flip6DefaultCLI = new Flip6DefaultCLI(new Configuration());
final Options options = new Options();
flip6DefaultCLI.addGeneralOptions(options);
@@ -46,6 +46,6 @@ public class Flip6DefaultCLITest extends TestLogger {
final CommandLine commandLine = CliFrontendParser.parse(options, args, false);
Assert.assertTrue(commandLine.hasOption(Flip6DefaultCLI.FLIP_6.getOpt()));
- Assert.assertTrue(flip6DefaultCLI.isActive(commandLine, new Configuration()));
+ Assert.assertTrue(flip6DefaultCLI.isActive(commandLine));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
index f9c2bfa..a36e8e9 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
@@ -22,7 +22,6 @@ import org.apache.flink.client.cli.CustomCommandLine;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.Preconditions;
import org.apache.commons.cli.CommandLine;
@@ -43,7 +42,7 @@ public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCo
}
@Override
- public boolean isActive(CommandLine commandLine, Configuration configuration) {
+ public boolean isActive(CommandLine commandLine) {
return true;
}
@@ -63,21 +62,18 @@ public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCo
}
@Override
- public ClusterDescriptor<T> createClusterDescriptor(
- Configuration configuration,
- String configurationDirectory,
- CommandLine commandLine) {
+ public ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) {
return new DummyClusterDescriptor<>(clusterClient);
}
@Override
@Nullable
- public String getClusterId(Configuration configuration, CommandLine commandLine) {
+ public String getClusterId(CommandLine commandLine) {
return "dummy";
}
@Override
- public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
+ public ClusterSpecification getClusterSpecification(CommandLine commandLine) {
return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
index 135e831..00b460f 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
@@ -19,7 +19,6 @@
package org.apache.flink.client.cli.util;
import org.apache.flink.client.cli.CliFrontend;
-import org.apache.flink.client.cli.CliFrontendTestUtils;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
@@ -35,7 +34,6 @@ public class MockedCliFrontend extends CliFrontend {
public MockedCliFrontend(ClusterClient clusterClient) throws Exception {
super(
new Configuration(),
- Collections.singletonList(new DummyCustomCommandLine<>(clusterClient)),
- CliFrontendTestUtils.getConfigDir());
+ Collections.singletonList(new DummyCustomCommandLine<>(clusterClient)));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
index ec8091a..7b34d4a 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
@@ -188,6 +188,8 @@ public class ClusterClientTest extends TestLogger {
fail("Dispose operation should have failed.");
} catch (ExecutionException e) {
assertTrue(ExceptionUtils.findThrowable(e, FlinkRuntimeException.class).isPresent());
+ } finally {
+ clusterClient.shutdown();
}
}
@@ -214,6 +216,8 @@ public class ClusterClientTest extends TestLogger {
"instance, which cannot be disposed without the user code class " +
"loader. Please provide the program jar with which you have created " +
"the savepoint via -j <JAR> for disposal.").isPresent());
+ } finally {
+ clusterClient.shutdown();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index d4d0d05..9f29ce0 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -256,22 +256,15 @@ object FlinkShell {
val options = CliFrontendParser.parseRunCommand(args.toArray)
val frontend = new CliFrontend(
configuration,
- CliFrontend.loadCustomCommandLines(),
- configurationDirectory)
+ CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
val config = frontend.getConfiguration
val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
- val clusterDescriptor = customCLI.createClusterDescriptor(
- config,
- frontend.getConfigurationDirectory,
- options.getCommandLine)
+ val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
- val clusterSpecification = customCLI.getClusterSpecification(
- config,
- options.getCommandLine)
+ val clusterSpecification = customCLI.getClusterSpecification(options.getCommandLine)
- val cluster = clusterDescriptor.deploySessionCluster(
- clusterSpecification)
+ val cluster = clusterDescriptor.deploySessionCluster(clusterSpecification)
val address = cluster.getJobManagerAddress.getAddress.getHostAddress
val port = cluster.getJobManagerAddress.getPort
@@ -291,19 +284,13 @@ object FlinkShell {
val options = CliFrontendParser.parseRunCommand(args.toArray)
val frontend = new CliFrontend(
configuration,
- CliFrontend.loadCustomCommandLines(),
- configurationDirectory)
+ CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
val config = frontend.getConfiguration
val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
- val clusterDescriptor = customCLI.createClusterDescriptor(
- configuration,
- configurationDirectory,
- options.getCommandLine)
+ val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
- val clusterId = customCLI.getClusterId(
- configuration,
- options.getCommandLine)
+ val clusterId = customCLI.getClusterId(options.getCommandLine)
val cluster = clusterDescriptor.retrieve(clusterId)
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 8716f8a..f347f94 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -577,10 +577,10 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
// load the configuration
LOG.info("testDetachedPerJobYarnClusterInternal: Trying to load configuration file");
- GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
+ Configuration configuration = GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
try {
- File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(GlobalConfiguration.loadConfiguration());
+ File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(configuration.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
if (yarnPropertiesFile.exists()) {
LOG.info("testDetachedPerJobYarnClusterInternal: Cleaning up temporary Yarn address reference: {}", yarnPropertiesFile.getAbsolutePath());
yarnPropertiesFile.delete();
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index ec8ef50..d5a9883 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -134,10 +135,10 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
// load the configuration
LOG.info("testDetachedPerJobYarnClusterInternal: Trying to load configuration file");
- GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
+ Configuration configuration = GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
try {
- File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(GlobalConfiguration.loadConfiguration());
+ File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
if (yarnPropertiesFile.exists()) {
LOG.info("testDetachedPerJobYarnClusterInternal: Cleaning up temporary Yarn address reference: {}", yarnPropertiesFile.getAbsolutePath());
yarnPropertiesFile.delete();
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index 5325ae2..ed02892 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -693,6 +693,8 @@ public abstract class YarnTestBase extends TestLogger {
switch (type) {
case YARN_SESSION:
yCli = new FlinkYarnSessionCli(
+ configuration,
+ configurationDirectory,
"",
"",
false);
@@ -702,8 +704,7 @@ public abstract class YarnTestBase extends TestLogger {
try {
CliFrontend cli = new CliFrontend(
configuration,
- CliFrontend.loadCustomCommandLines(),
- configurationDirectory);
+ CliFrontend.loadCustomCommandLines(configuration, configurationDirectory));
returnValue = cli.parseParameters(args);
} catch (Exception e) {
throw new RuntimeException("Failed to execute the following args with CliFrontend: "
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index 63421f9..5fb7f90 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
@@ -330,7 +331,7 @@ public class YarnClusterClient extends ClusterClient {
}
try {
- File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig);
+ File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
if (propertiesFile.isFile()) {
if (propertiesFile.delete()) {
LOG.info("Deleted Yarn properties file at {}", propertiesFile.getAbsoluteFile().toString());
[12/14] flink git commit: [FLINK-8201] [yarn] Delete temp
configuration file after uploading it to HDFS
Posted by tr...@apache.org.
[FLINK-8201] [yarn] Delete temp configuration file after uploading it to HDFS
The Utils#createTaskExecutorContext method creates a temporary local configuration
file which it then uploads to HDFS. This fille should be removed after the upload
has completed.
This closes #5123.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d0bc3000
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d0bc3000
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d0bc3000
Branch: refs/heads/master
Commit: d0bc300087b037b156425ccd509147177dd9529e
Parents: ed0716a
Author: wenlong.lwl <we...@alibaba-inc.com>
Authored: Tue Dec 5 15:54:39 2017 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:06 2018 +0100
----------------------------------------------------------------------
.../main/java/org/apache/flink/yarn/Utils.java | 32 +++++++++++++-------
1 file changed, 21 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d0bc3000/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
index 652afec..9ae5b54 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
@@ -23,6 +23,7 @@ import org.apache.flink.configuration.ResourceManagerOptions;
import org.apache.flink.runtime.clusterframework.BootstrapTools;
import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
import org.apache.flink.runtime.util.HadoopUtils;
+import org.apache.flink.util.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -408,17 +409,26 @@ public final class Utils {
log.debug("Writing TaskManager configuration to {}", taskManagerConfigFile.getAbsolutePath());
BootstrapTools.writeConfiguration(taskManagerConfig, taskManagerConfigFile);
- Path homeDirPath = new Path(clientHomeDir);
- FileSystem fs = homeDirPath.getFileSystem(yarnConfig);
-
- flinkConf = setupLocalResource(
- fs,
- appId,
- new Path(taskManagerConfigFile.toURI()),
- homeDirPath,
- "").f1;
-
- log.info("Prepared local resource for modified yaml: {}", flinkConf);
+ try {
+ Path homeDirPath = new Path(clientHomeDir);
+ FileSystem fs = homeDirPath.getFileSystem(yarnConfig);
+
+ flinkConf = setupLocalResource(
+ fs,
+ appId,
+ new Path(taskManagerConfigFile.toURI()),
+ homeDirPath,
+ "").f1;
+
+ log.info("Prepared local resource for modified yaml: {}", flinkConf);
+ } finally {
+ try {
+ FileUtils.deleteFileOrDirectory(taskManagerConfigFile);
+ } catch (IOException e) {
+ log.info("Could not delete temporary configuration file " +
+ taskManagerConfigFile.getAbsolutePath() + '.', e);
+ }
+ }
}
Map<String, LocalResource> taskManagerLocalResources = new HashMap<>();
[05/14] flink git commit: [FLINK-8342] [flip6] Remove generic type
parameter from ClusterDescriptor
Posted by tr...@apache.org.
[FLINK-8342] [flip6] Remove generic type parameter from ClusterDescriptor
This closes #5228.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/10e900b2
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/10e900b2
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/10e900b2
Branch: refs/heads/master
Commit: 10e900b25ac03876d3f9e78f260d48efe6b9d853
Parents: d7e9dc1
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Jan 2 10:19:34 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:03 2018 +0100
----------------------------------------------------------------------
.../client/cli/AbstractCustomCommandLine.java | 4 +--
.../apache/flink/client/cli/CliFrontend.java | 34 ++++++++++----------
.../flink/client/cli/CliFrontendParser.java | 14 ++++----
.../flink/client/cli/CustomCommandLine.java | 5 ++-
.../org/apache/flink/client/cli/DefaultCLI.java | 5 ++-
.../flink/client/cli/Flip6DefaultCLI.java | 5 ++-
.../client/deployment/ClusterDescriptor.java | 8 ++---
.../Flip6StandaloneClusterDescriptor.java | 2 +-
.../deployment/StandaloneClusterDescriptor.java | 2 +-
.../apache/flink/client/cli/DefaultCLITest.java | 4 +--
.../client/cli/util/DummyClusterDescriptor.java | 14 ++++----
.../client/cli/util/DummyCustomCommandLine.java | 12 +++----
.../client/cli/util/MockedCliFrontend.java | 2 +-
.../YARNSessionCapacitySchedulerITCase.java | 2 +-
.../flink/yarn/YARNSessionFIFOITCase.java | 4 +--
.../yarn/AbstractYarnClusterDescriptor.java | 4 +--
.../flink/yarn/cli/FlinkYarnSessionCli.java | 2 +-
17 files changed, 57 insertions(+), 66 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
index c7a1672..da21556 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
@@ -19,7 +19,6 @@
package org.apache.flink.client.cli;
import org.apache.flink.client.ClientUtils;
-import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.UnmodifiableConfiguration;
@@ -38,9 +37,8 @@ import static org.apache.flink.client.cli.CliFrontend.setJobManagerAddressInConf
* Base class for {@link CustomCommandLine} implementations which specify a JobManager address and
* a ZooKeeper namespace.
*
- * @param <C> type of the ClusterClient which is returned
*/
-public abstract class AbstractCustomCommandLine<C extends ClusterClient> implements CustomCommandLine<C> {
+public abstract class AbstractCustomCommandLine implements CustomCommandLine {
protected final Option zookeeperNamespaceOption = new Option("z", "zookeeperNamespace", true,
"Namespace to create the Zookeeper sub-paths for high availability mode");
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 630154c..453d086 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -106,7 +106,7 @@ public class CliFrontend {
private final Configuration configuration;
- private final List<CustomCommandLine<?>> customCommandLines;
+ private final List<CustomCommandLine> customCommandLines;
private final Options customCommandLineOptions;
@@ -116,7 +116,7 @@ public class CliFrontend {
public CliFrontend(
Configuration configuration,
- List<CustomCommandLine<?>> customCommandLines) throws Exception {
+ List<CustomCommandLine> customCommandLines) throws Exception {
this.configuration = Preconditions.checkNotNull(configuration);
this.customCommandLines = Preconditions.checkNotNull(customCommandLines);
@@ -129,7 +129,7 @@ public class CliFrontend {
this.customCommandLineOptions = new Options();
- for (CustomCommandLine<?> customCommandLine : customCommandLines) {
+ for (CustomCommandLine customCommandLine : customCommandLines) {
customCommandLine.addGeneralOptions(customCommandLineOptions);
customCommandLine.addRunOptions(customCommandLineOptions);
}
@@ -196,9 +196,9 @@ public class CliFrontend {
throw new CliArgsException("Could not build the program from JAR file.", e);
}
- final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
+ final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
+ final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
try {
final String clusterId = customCommandLine.getClusterId(commandLine);
@@ -351,8 +351,8 @@ public class CliFrontend {
scheduled = true;
}
- final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+ final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
+ final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
final String clusterId = activeCommandLine.getClusterId(commandLine);
@@ -473,9 +473,9 @@ public class CliFrontend {
throw new CliArgsException("Missing JobID");
}
- final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
+ final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+ final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
final String clusterId = activeCommandLine.getClusterId(commandLine);
@@ -553,9 +553,9 @@ public class CliFrontend {
throw new CliArgsException("Missing JobID in the command line arguments.");
}
- final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
+ final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+ final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
final String clusterId = activeCommandLine.getClusterId(commandLine);
@@ -617,9 +617,9 @@ public class CliFrontend {
return;
}
- CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
+ CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
- final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
+ final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
final String clusterId = customCommandLine.getClusterId(commandLine);
@@ -972,7 +972,7 @@ public class CliFrontend {
final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory);
// 3. load the custom command lines
- final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines(
+ final List<CustomCommandLine> customCommandLines = loadCustomCommandLines(
configuration,
configurationDirectory);
@@ -1039,8 +1039,8 @@ public class CliFrontend {
config.setInteger(JobManagerOptions.PORT, address.getPort());
}
- public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
- List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
+ public static List<CustomCommandLine> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
+ List<CustomCommandLine> customCommandLines = new ArrayList<>(2);
// Command line interface of the YARN session, with a special initialization here
// to prefix all options with y/yarn.
@@ -1087,7 +1087,7 @@ public class CliFrontend {
* @param className The fully-qualified class name to load.
* @param params The constructor parameters
*/
- private static CustomCommandLine<?> loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
+ private static CustomCommandLine loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
Class<? extends CustomCommandLine> customCliClass =
Class.forName(className).asSubclass(CustomCommandLine.class);
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 475d854..10507d6 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -236,7 +236,7 @@ public class CliFrontendParser {
/**
* Prints the help for the client.
*/
- public static void printHelp(Collection<CustomCommandLine<?>> customCommandLines) {
+ public static void printHelp(Collection<CustomCommandLine> customCommandLines) {
System.out.println("./flink <ACTION> [OPTIONS] [ARGUMENTS]");
System.out.println();
System.out.println("The following actions are available:");
@@ -251,7 +251,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForRun(Collection<CustomCommandLine<?>> customCommandLines) {
+ public static void printHelpForRun(Collection<CustomCommandLine> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -279,7 +279,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForList(Collection<CustomCommandLine<?>> customCommandLines) {
+ public static void printHelpForList(Collection<CustomCommandLine> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -294,7 +294,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForStop(Collection<CustomCommandLine<?>> customCommandLines) {
+ public static void printHelpForStop(Collection<CustomCommandLine> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -309,7 +309,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForCancel(Collection<CustomCommandLine<?>> customCommandLines) {
+ public static void printHelpForCancel(Collection<CustomCommandLine> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -324,7 +324,7 @@ public class CliFrontendParser {
System.out.println();
}
- public static void printHelpForSavepoint(Collection<CustomCommandLine<?>> customCommandLines) {
+ public static void printHelpForSavepoint(Collection<CustomCommandLine> customCommandLines) {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
@@ -345,7 +345,7 @@ public class CliFrontendParser {
* @param runOptions True if the run options should be printed, False to print only general options
*/
private static void printCustomCliOptions(
- Collection<CustomCommandLine<?>> customCommandLines,
+ Collection<CustomCommandLine> customCommandLines,
HelpFormatter formatter,
boolean runOptions) {
// prints options from all available command-line classes
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index f642484..aabc224 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -20,7 +20,6 @@ package org.apache.flink.client.cli;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
-import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.util.FlinkException;
import org.apache.commons.cli.CommandLine;
@@ -31,7 +30,7 @@ import javax.annotation.Nullable;
/**
* Custom command-line interface to load hooks for the command-line interface.
*/
-public interface CustomCommandLine<ClusterType extends ClusterClient> {
+public interface CustomCommandLine {
/**
* Signals whether the custom command-line wants to execute or not.
@@ -67,7 +66,7 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
* @return ClusterDescriptor
* @throws FlinkException if the ClusterDescriptor could not be created
*/
- ClusterDescriptor<ClusterType> createClusterDescriptor(CommandLine commandLine) throws FlinkException;
+ ClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException;
/**
* Returns the cluster id if a cluster id was specified on the command line, otherwise it
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index c29c5b7..5660765 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -21,7 +21,6 @@ package org.apache.flink.client.cli;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
-import org.apache.flink.client.program.StandaloneClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.FlinkException;
@@ -32,7 +31,7 @@ import javax.annotation.Nullable;
/**
* The default CLI which is used for interaction with standalone clusters.
*/
-public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClient> {
+public class DefaultCLI extends AbstractCustomCommandLine {
public DefaultCLI(Configuration configuration) {
super(configuration);
@@ -50,7 +49,7 @@ public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClien
}
@Override
- public ClusterDescriptor<StandaloneClusterClient> createClusterDescriptor(
+ public ClusterDescriptor createClusterDescriptor(
CommandLine commandLine) throws FlinkException {
final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
index 3adeca6..1a75aac 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
@@ -21,7 +21,6 @@ package org.apache.flink.client.cli;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.client.deployment.ClusterSpecification;
import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor;
-import org.apache.flink.client.program.rest.RestClusterClient;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.FlinkException;
@@ -34,7 +33,7 @@ import javax.annotation.Nullable;
/**
* The default CLI which is used for interaction with standalone clusters.
*/
-public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient> {
+public class Flip6DefaultCLI extends AbstractCustomCommandLine {
public static final Option FLIP_6 = new Option("flip6", "Switches the client to Flip-6 mode.");
@@ -63,7 +62,7 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient
}
@Override
- public ClusterDescriptor<RestClusterClient> createClusterDescriptor(
+ public ClusterDescriptor createClusterDescriptor(
CommandLine commandLine) throws FlinkException {
final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
index 1603930..07eeabc 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
@@ -24,7 +24,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
/**
* A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
*/
-public interface ClusterDescriptor<ClientType extends ClusterClient> extends AutoCloseable {
+public interface ClusterDescriptor extends AutoCloseable {
/**
* Returns a String containing details about the cluster (NodeManagers, available memory, ...).
@@ -38,7 +38,7 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> extends Aut
* @return Client for the cluster
* @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
*/
- ClientType retrieve(String applicationID) throws UnsupportedOperationException;
+ ClusterClient retrieve(String applicationID) throws UnsupportedOperationException;
/**
* Triggers deployment of a cluster.
@@ -46,7 +46,7 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> extends Aut
* @return Client for the cluster
* @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
*/
- ClientType deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException;
+ ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException;
/**
* Deploys a per-job cluster with the given job on the cluster.
@@ -56,7 +56,7 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> extends Aut
* @return Cluster client to talk to the Flink cluster
* @throws ClusterDeploymentException if the cluster could not be deployed
*/
- ClientType deployJobCluster(
+ ClusterClient deployJobCluster(
final ClusterSpecification clusterSpecification,
final JobGraph jobGraph);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
index b8eb534..a35a68b 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.Preconditions;
/**
* A deployment descriptor for an existing cluster.
*/
-public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor<RestClusterClient> {
+public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
private final Configuration config;
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
index 3808efa..ae25194 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
/**
* A deployment descriptor for an existing cluster.
*/
-public class StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterClient> {
+public class StandaloneClusterDescriptor implements ClusterDescriptor {
private final Configuration config;
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
index e73b9c9..6eb005d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
@@ -62,7 +62,7 @@ public class DefaultCLITest extends TestLogger {
final InetSocketAddress expectedAddress = new InetSocketAddress(localhost, port);
- final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+ final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
@@ -89,7 +89,7 @@ public class DefaultCLITest extends TestLogger {
CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
- final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+ final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
index e46957b..d300055 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
@@ -26,14 +26,12 @@ import org.apache.flink.util.Preconditions;
/**
* Dummy {@link ClusterDescriptor} implementation for testing purposes.
- *
- * @param <C> type of the returned {@link ClusterClient}
*/
-public class DummyClusterDescriptor<C extends ClusterClient> implements ClusterDescriptor<C> {
+public class DummyClusterDescriptor implements ClusterDescriptor {
- private final C clusterClient;
+ private final ClusterClient clusterClient;
- public DummyClusterDescriptor(C clusterClient) {
+ public DummyClusterDescriptor(ClusterClient clusterClient) {
this.clusterClient = Preconditions.checkNotNull(clusterClient);
}
@@ -43,17 +41,17 @@ public class DummyClusterDescriptor<C extends ClusterClient> implements ClusterD
}
@Override
- public C retrieve(String applicationID) throws UnsupportedOperationException {
+ public ClusterClient retrieve(String applicationID) throws UnsupportedOperationException {
return clusterClient;
}
@Override
- public C deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+ public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
return clusterClient;
}
@Override
- public C deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+ public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
return clusterClient;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
index a36e8e9..5279d85 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
@@ -31,13 +31,11 @@ import javax.annotation.Nullable;
/**
* Dummy implementation of the {@link CustomCommandLine} for testing purposes.
- *
- * @param <T> type of the returned cluster client
*/
-public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCommandLine<T> {
- private final T clusterClient;
+public class DummyCustomCommandLine implements CustomCommandLine {
+ private final ClusterClient clusterClient;
- public DummyCustomCommandLine(T clusterClient) {
+ public DummyCustomCommandLine(ClusterClient clusterClient) {
this.clusterClient = Preconditions.checkNotNull(clusterClient);
}
@@ -62,8 +60,8 @@ public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCo
}
@Override
- public ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) {
- return new DummyClusterDescriptor<>(clusterClient);
+ public ClusterDescriptor createClusterDescriptor(CommandLine commandLine) {
+ return new DummyClusterDescriptor(clusterClient);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
index 00b460f..477293d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
@@ -34,6 +34,6 @@ public class MockedCliFrontend extends CliFrontend {
public MockedCliFrontend(ClusterClient clusterClient) throws Exception {
super(
new Configuration(),
- Collections.singletonList(new DummyCustomCommandLine<>(clusterClient)));
+ Collections.singletonList(new DummyCustomCommandLine(clusterClient)));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index f347f94..5bed22e 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -365,7 +365,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
@Test
public void testNonexistingQueueWARNmessage() {
LOG.info("Starting testNonexistingQueueWARNmessage()");
- addTestAppender(YarnClusterDescriptor.class, Level.WARN);
+ addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
"-t", flinkLibFolder.getAbsolutePath(),
"-n", "1",
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index d5a9883..cc26350 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -179,7 +179,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
@Ignore("The test is too resource consuming (8.5 GB of memory)")
@Test
public void testResourceComputation() {
- addTestAppender(YarnClusterDescriptor.class, Level.WARN);
+ addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
LOG.info("Starting testResourceComputation()");
runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
"-n", "5",
@@ -207,7 +207,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
@Ignore("The test is too resource consuming (8 GB of memory)")
@Test
public void testfullAlloc() {
- addTestAppender(YarnClusterDescriptor.class, Level.WARN);
+ addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
LOG.info("Starting testfullAlloc()");
runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
"-n", "2",
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 0372319..0a977df 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -99,8 +99,8 @@ import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties
/**
* The descriptor with deployment information for spawning or resuming a {@link YarnClusterClient}.
*/
-public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor<YarnClusterClient> {
- private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class);
+public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor {
+ private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class);
/**
* Minimum memory requirements, checked by the Client.
http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d4ab41f..d797f47 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -81,7 +81,7 @@ import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_
/**
* Class handling the command line interface to the YARN session.
*/
-public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterClient> {
+public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class);
//------------------------------------ Constants -------------------------
[11/14] flink git commit: [hotfix] Log failure message only if Yarn
application truly failed
Posted by tr...@apache.org.
[hotfix] Log failure message only if Yarn application truly failed
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ed0716ac
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ed0716ac
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ed0716ac
Branch: refs/heads/master
Commit: ed0716ac710387979735140d18a598240b43085d
Parents: dbe0e82
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Dec 29 17:08:45 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:06 2018 +0100
----------------------------------------------------------------------
.../main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/ed0716ac/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d4cf9ad..cc7f4c1 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -686,8 +686,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
.getYarnApplicationState() + " and final state " + appReport
.getFinalApplicationStatus() + " at " + appReport.getFinishTime());
- if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState()
- == YarnApplicationState.KILLED) {
+ if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED) {
LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics());
LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve "
+ "the full application log using this command:"