Skip to content

Commit

Permalink
[FLINK-9820] Forward dynamic properties to Flink configuration in Clu…
Browse files Browse the repository at this point in the history
…sterEntrypoint

With this commit we can use dynamic properties to overwrite configuration values in the
ClusterEntrypoint.

This closes apache#6317.
  • Loading branch information
tillrohrmann committed Jul 13, 2018
1 parent 5a4bdf2 commit 2fbbf8e
Show file tree
Hide file tree
Showing 3 changed files with 79 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import javax.annotation.Nonnull;

import java.io.File;
import java.util.Properties;
import java.util.Set;

/**
* Utility class for {@link Configuration} related helper functions.
Expand Down Expand Up @@ -54,6 +56,25 @@ public static String[] parseLocalStateDirectories(Configuration configuration) {
return splitPaths(configValue);
}

/**
* Creates a new {@link Configuration} from the given {@link Properties}.
*
* @param properties to convert into a {@link Configuration}
* @return {@link Configuration} which has been populated by the values of the given {@link Properties}
*/
@Nonnull
public static Configuration createConfiguration(Properties properties) {
final Configuration configuration = new Configuration();

final Set<String> propertyNames = properties.stringPropertyNames();

for (String propertyName : propertyNames) {
configuration.setString(propertyName, properties.getProperty(propertyName));
}

return configuration;
}

@Nonnull
private static String[] splitPaths(@Nonnull String separatedPaths) {
return separatedPaths.length() > 0 ? separatedPaths.split(",|" + File.pathSeparator) : EMPTY;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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
*
* https://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.configuration;

import org.apache.flink.util.TestLogger;

import org.junit.Test;

import java.util.Properties;

import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;

/**
* Tests for the {@link ConfigurationUtils}.
*/
public class ConfigurationUtilsTest extends TestLogger {

@Test
public void testPropertiesToConfiguration() {
final Properties properties = new Properties();
final int entries = 10;

for (int i = 0; i < entries; i++) {
properties.setProperty("key" + i, "value" + i);
}

final Configuration configuration = ConfigurationUtils.createConfiguration(properties);

for (String key : properties.stringPropertyNames()) {
assertThat(configuration.getString(key, ""), is(equalTo(properties.getProperty(key))));
}

assertThat(configuration.toMap().size(), is(properties.size()));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ConfigurationUtils;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.JobManagerOptions;
Expand Down Expand Up @@ -688,14 +689,15 @@ protected abstract ArchivedExecutionGraphStore createSerializableExecutionGraphS
Configuration configuration,
ScheduledExecutor scheduledExecutor) throws IOException;

private static EntrypointClusterConfiguration parseArguments(String[] args) throws FlinkParseException {
protected static EntrypointClusterConfiguration parseArguments(String[] args) throws FlinkParseException {
final CommandLineParser<EntrypointClusterConfiguration> clusterConfigurationParser = new CommandLineParser<>(new EntrypointClusterConfigurationParserFactory());

return clusterConfigurationParser.parse(args);
}

protected static Configuration loadConfiguration(EntrypointClusterConfiguration entrypointClusterConfiguration) {
final Configuration configuration = GlobalConfiguration.loadConfiguration(entrypointClusterConfiguration.getConfigDir());
final Configuration dynamicProperties = ConfigurationUtils.createConfiguration(entrypointClusterConfiguration.getDynamicProperties());
final Configuration configuration = GlobalConfiguration.loadConfiguration(entrypointClusterConfiguration.getConfigDir(), dynamicProperties);

final int restPort = entrypointClusterConfiguration.getRestPort();

Expand Down

0 comments on commit 2fbbf8e

Please sign in to comment.