[ https://issues.apache.org/jira/browse/FLINK-8339?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16311648#comment-16311648 ]
ASF GitHub Bot commented on FLINK-8339: --------------------------------------- Github user GJL commented on a diff in the pull request: https://github.com/apache/flink/pull/5225#discussion_r159656514 --- Diff: flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java --- @@ -59,31 +60,49 @@ void addGeneralOptions(Options baseOptions); /** - * Retrieves a client for a running cluster. - * @param commandLine The command-line parameters from the CliFrontend - * @param config The Flink config - * @param configurationDirectory Directory for configuration files - * @return Client if a cluster could be retrieved - * @throws UnsupportedOperationException if the operation is not supported + * 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 */ - ClusterType retrieveCluster( - CommandLine commandLine, - Configuration config, - String configurationDirectory) throws UnsupportedOperationException; + ClusterDescriptor<ClusterType> createClusterDescriptor( + Configuration configuration, + String configurationDirectory, + CommandLine commandLine); /** - * Creates the client for the cluster. - * @param applicationName The application name to use - * @param commandLine The command-line options parsed by the CliFrontend - * @param config The Flink config to use - * @param configurationDirectory Directory for configuration files - *@param userJarFiles User jar files to include in the classpath of the cluster. @return The client to communicate with the cluster which the CustomCommandLine brought up. - * @throws Exception if the cluster could not be created + * Returns the cluster id if a cluster id was specified on the command line, otherwise it + * returns null. + * + * <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 */ - ClusterType createCluster( - String applicationName, - CommandLine commandLine, - Configuration config, - String configurationDirectory, - List<URL> userJarFiles) throws Exception; + @Nullable + String getClusterId(Configuration configuration, 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 + */ + ClusterSpecification getClusterSpecification( + Configuration configuration, + CommandLine commandLine); + + default CommandLine parseCommandLineOptions(String[] args, boolean stopAtNonOptions) throws CliArgsException { --- End diff -- It seems that this is only needed for unit tests. > Let CustomCommandLine return a ClusterDescriptor > ------------------------------------------------ > > Key: FLINK-8339 > URL: https://issues.apache.org/jira/browse/FLINK-8339 > Project: Flink > Issue Type: Sub-task > Components: Client > Affects Versions: 1.5.0 > Reporter: Till Rohrmann > Assignee: Till Rohrmann > Labels: flip-6 > Fix For: 1.5.0 > > > The {{CustomCommandLine}} currently is able to retrieve a {{ClusterClient}} > and deploy a cluster. In order to better separate concerns it would be good > if the {{CustomCommandLine}} would simply return a {{ClusterDescriptor}} > which could then be used to retrieve a {{ClusterClient}} or to deploy a Flink > cluster. -- This message was sent by Atlassian JIRA (v6.4.14#64029)