flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8339) Let CustomCommandLine return a ClusterDescriptor
Date Thu, 04 Jan 2018 16:58:00 GMT

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

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_r159698673
  
    --- Diff: flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
---
    @@ -56,43 +51,30 @@ public String getId() {
     		return "flip6";
     	}
     
    -	@Override
    -	public void addRunOptions(Options baseOptions) {
    -	}
    -
     	@Override
     	public void addGeneralOptions(Options baseOptions) {
    +		super.addGeneralOptions(baseOptions);
     		baseOptions.addOption(FLIP_6);
     	}
     
     	@Override
    -	public RestClusterClient retrieveCluster(CommandLine commandLine, Configuration config,
String configurationDirectory) {
    -		if (commandLine.hasOption(CliFrontendParser.ADDRESS_OPTION.getOpt())) {
    -			String addressWithPort = commandLine.getOptionValue(CliFrontendParser.ADDRESS_OPTION.getOpt());
    -			InetSocketAddress jobManagerAddress = ClientUtils.parseHostPortAddress(addressWithPort);
    -			setJobManagerAddressInConfig(config, jobManagerAddress);
    -		}
    -
    -		if (commandLine.hasOption(CliFrontendParser.ZOOKEEPER_NAMESPACE_OPTION.getOpt())) {
    -			String zkNamespace = commandLine.getOptionValue(CliFrontendParser.ZOOKEEPER_NAMESPACE_OPTION.getOpt());
    -			config.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace);
    -		}
    +	public ClusterDescriptor<RestClusterClient> createClusterDescriptor(
    +			Configuration configuration,
    +			String configurationDirectory,
    +			CommandLine commandLine) {
    +		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration,
commandLine);
     
    -		Flip6StandaloneClusterDescriptor descriptor = new Flip6StandaloneClusterDescriptor(config);
    -		return descriptor.retrieve(null);
    +		return new Flip6StandaloneClusterDescriptor(effectiveConfiguration);
     	}
     
     	@Override
    -	public RestClusterClient createCluster(
    -			String applicationName,
    -			CommandLine commandLine,
    -			Configuration config,
    -			String configurationDirectory,
    -			List<URL> userJarFiles) throws UnsupportedOperationException {
    -
    -		Flip6StandaloneClusterDescriptor descriptor = new Flip6StandaloneClusterDescriptor(config);
    -		ClusterSpecification clusterSpecification = ClusterSpecification.fromConfiguration(config);
    +	@Nullable
    +	public String getClusterId(Configuration configuration, CommandLine commandLine) {
    +		return "flip6Standalone";
    +	}
     
    -		return descriptor.deploySessionCluster(clusterSpecification);
    +	@Override
    +	public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine
commandLine) {
    +		return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
    --- End diff --
    
    Same thing here. See comment on `DefaultCLI.java`.


> 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)

Mime
View raw message