Skip to content

Commit

Permalink
[FLINK-17935] Move set yarn.log-config-file to YarnClusterClientFacto…
Browse files Browse the repository at this point in the history
…ry.createClusterDescriptor()

This closes apache#12455.
  • Loading branch information
kl0u committed Jun 4, 2020
1 parent 20abff9 commit a674b5e
Show file tree
Hide file tree
Showing 15 changed files with 125 additions and 47 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1045,7 +1045,7 @@ static void setJobManagerAddressInConfig(Configuration config, InetSocketAddress

public static List<CustomCommandLine> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
List<CustomCommandLine> customCommandLines = new ArrayList<>();
customCommandLines.add(new ExecutorCLI(configuration));
customCommandLines.add(new ExecutorCLI(configuration, configurationDirectory));

// Command line interface of the YARN session, with a special initialization here
// to prefix all options with y/yarn.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.DeploymentOptionsInternal;
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.core.execution.DefaultExecutorServiceLoader;
import org.apache.flink.core.execution.PipelineExecutor;
Expand Down Expand Up @@ -72,8 +73,11 @@ public class ExecutorCLI implements CustomCommandLine {

private final Configuration baseConfiguration;

public ExecutorCLI(final Configuration configuration) {
private final String configurationDir;

public ExecutorCLI(final Configuration configuration, final String configDir) {
this.baseConfiguration = new UnmodifiableConfiguration(checkNotNull(configuration));
this.configurationDir = checkNotNull(configDir);
}

@Override
Expand Down Expand Up @@ -115,6 +119,7 @@ public Configuration applyCommandLineOptionsToConfiguration(final CommandLine co
}

encodeDynamicProperties(commandLine, effectiveConfiguration);
effectiveConfiguration.set(DeploymentOptionsInternal.CONF_DIR, configurationDir);

if (LOG.isDebugEnabled()) {
LOG.debug("Effective Configuration: {}", effectiveConfiguration);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,9 @@
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Options;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import java.util.Arrays;
import java.util.List;
Expand All @@ -41,13 +43,18 @@
*/
public class ExecutorCLITest {

@Rule
public TemporaryFolder tmp = new TemporaryFolder();

private Options testOptions;

@Before
public void initOptions() {
testOptions = new Options();

final ExecutorCLI cliUnderTest = new ExecutorCLI(new Configuration());
final ExecutorCLI cliUnderTest = new ExecutorCLI(
new Configuration(),
tmp.getRoot().getAbsolutePath());
cliUnderTest.addGeneralOptions(testOptions);
}

Expand All @@ -57,7 +64,9 @@ public void testExecutorInBaseConfigIsPickedUp() throws CliArgsException {
final Configuration loadedConfig = new Configuration();
loadedConfig.set(DeploymentOptions.TARGET, expectedExecutorName);

final ExecutorCLI cliUnderTest = new ExecutorCLI(loadedConfig);
final ExecutorCLI cliUnderTest = new ExecutorCLI(
loadedConfig,
tmp.getRoot().getAbsolutePath());
final CommandLine emptyCommandLine = CliFrontendParser.parse(testOptions, new String[0], true);

final Configuration configuration = cliUnderTest.applyCommandLineOptionsToConfiguration(emptyCommandLine);
Expand Down Expand Up @@ -87,7 +96,9 @@ public void testWithPreexistingConfigurationInConstructor() throws CliArgsExcept
"-D" + CoreOptions.DEFAULT_PARALLELISM.key() + "=5"
};

final ExecutorCLI cliUnderTest = new ExecutorCLI(loadedConfig);
final ExecutorCLI cliUnderTest = new ExecutorCLI(
loadedConfig,
tmp.getRoot().getAbsolutePath());
final CommandLine commandLine = CliFrontendParser.parse(testOptions, args, true);

final Configuration configuration = cliUnderTest.applyCommandLineOptionsToConfiguration(commandLine);
Expand All @@ -113,7 +124,9 @@ private void testIsActiveHelper(final String executorOption) throws CliArgsExcep
final ConfigOption<Integer> configOption = key("test.int").intType().noDefaultValue();
final int expectedValue = 42;

final ExecutorCLI cliUnderTest = new ExecutorCLI(new Configuration());
final ExecutorCLI cliUnderTest = new ExecutorCLI(
new Configuration(),
tmp.getRoot().getAbsolutePath());

final String[] args = {executorOption, expectedExecutorName, "-D" + configOption.key() + "=" + expectedValue};
final CommandLine commandLine = CliFrontendParser.parse(testOptions, args, true);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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: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.annotation.Internal;

import static org.apache.flink.configuration.ConfigOptions.key;

/**
* Internal options used during deployment.
*/
@Internal
public class DeploymentOptionsInternal {

public static final ConfigOption<String> CONF_DIR =
key("$internal.deployment.config-dir")
.stringType()
.noDefaultValue()
.withDescription("**DO NOT USE** The path to the configuration directory.");

}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.api.java.tuple.Tuple2;
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.ExecutorCLI;
import org.apache.flink.client.deployment.ClusterClientFactory;
import org.apache.flink.client.deployment.ClusterClientServiceLoader;
Expand Down Expand Up @@ -68,14 +69,14 @@ public class KubernetesSessionCli {
private final ExecutorCLI cli;
private final ClusterClientServiceLoader clusterClientServiceLoader;

public KubernetesSessionCli(Configuration configuration) {
this(configuration, new DefaultClusterClientServiceLoader());
public KubernetesSessionCli(Configuration configuration, String configDir) {
this(configuration, new DefaultClusterClientServiceLoader(), configDir);
}

public KubernetesSessionCli(Configuration configuration, ClusterClientServiceLoader clusterClientServiceLoader) {
public KubernetesSessionCli(Configuration configuration, ClusterClientServiceLoader clusterClientServiceLoader, String configDir) {
this.baseConfiguration = new UnmodifiableConfiguration(checkNotNull(configuration));
this.clusterClientServiceLoader = checkNotNull(clusterClientServiceLoader);
this.cli = new ExecutorCLI(baseConfiguration);
this.cli = new ExecutorCLI(baseConfiguration, configDir);
}

public Configuration getEffectiveConfiguration(String[] args) throws CliArgsException {
Expand Down Expand Up @@ -178,10 +179,12 @@ private Tuple2<Boolean, Boolean> repStep(BufferedReader in) throws IOException,
public static void main(String[] args) {
final Configuration configuration = GlobalConfiguration.loadConfiguration();

final String configDir = CliFrontend.getConfigurationDirectoryFromEnv();

int retCode;

try {
final KubernetesSessionCli cli = new KubernetesSessionCli(configuration);
final KubernetesSessionCli cli = new KubernetesSessionCli(configuration, configDir);
retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args));
} catch (CliArgsException e) {
retCode = AbstractCustomCommandLine.handleCliArgsException(e, LOG);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.kubernetes.kubeclient.decorators;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.client.cli.CliFrontend;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.kubeclient.FlinkPod;
Expand Down Expand Up @@ -167,7 +166,7 @@ String getFlinkConfData(Map<String, String> propertiesMap) throws IOException {
}

private List<File> getLocalLogConfFiles() {
final String confDir = CliFrontend.getConfigurationDirectoryFromEnv();
final String confDir = kubernetesComponentConf.getConfigDirectory();
final File logbackFile = new File(confDir, CONFIG_FILE_LOGBACK_NAME);
final File log4jFile = new File(confDir, CONFIG_FILE_LOG4J_NAME);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@

package org.apache.flink.kubernetes.kubeclient.parameters;

import org.apache.flink.client.cli.CliFrontend;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptionsInternal;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.utils.Constants;

Expand Down Expand Up @@ -54,6 +54,13 @@ public Configuration getFlinkConfiguration() {
return flinkConfig;
}

@Override
public String getConfigDirectory() {
final String configDir = flinkConfig.get(DeploymentOptionsInternal.CONF_DIR);
checkNotNull(configDir);
return configDir;
}

@Override
public String getClusterId() {
final String clusterId = flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID);
Expand Down Expand Up @@ -130,14 +137,14 @@ public String getContainerEntrypoint() {

@Override
public boolean hasLogback() {
final String confDir = CliFrontend.getConfigurationDirectoryFromEnv();
final String confDir = getConfigDirectory();
final File logbackFile = new File(confDir, CONFIG_FILE_LOGBACK_NAME);
return logbackFile.exists();
}

@Override
public boolean hasLog4j() {
final String confDir = CliFrontend.getConfigurationDirectoryFromEnv();
final String confDir = getConfigDirectory();
final File log4jFile = new File(confDir, CONFIG_FILE_LOG4J_NAME);
return log4jFile.exists();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
*/
public interface KubernetesParameters {

String getConfigDirectory();

String getClusterId();

String getNamespace();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@

package org.apache.flink.kubernetes;

import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptionsInternal;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.core.testutils.CommonTestUtils;
Expand All @@ -29,7 +29,6 @@
import org.apache.flink.kubernetes.utils.Constants;
import org.apache.flink.runtime.clusterframework.BootstrapTools;
import org.apache.flink.runtime.concurrent.Executors;
import org.apache.flink.test.util.TestBaseUtils;
import org.apache.flink.util.TestLogger;

import io.fabric8.kubernetes.client.KubernetesClient;
Expand Down Expand Up @@ -77,24 +76,20 @@ protected void setupFlinkConfig() {
flinkConfig.setString(KubernetesConfigOptions.CONTAINER_IMAGE, CONTAINER_IMAGE);
flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY);
flinkConfig.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(JOB_MANAGER_MEMORY));
flinkConfig.set(DeploymentOptionsInternal.CONF_DIR, flinkConfDir.toString());
}

protected void onSetup() throws Exception {
}

@Before
public final void setup() throws Exception {
setupFlinkConfig();

flinkConfDir = temporaryFolder.newFolder().getAbsoluteFile();
hadoopConfDir = temporaryFolder.newFolder().getAbsoluteFile();

setupFlinkConfig();
writeFlinkConfiguration();

Map<String, String> map = new HashMap<>();
map.put(ConfigConstants.ENV_FLINK_CONF_DIR, flinkConfDir.toString());
TestBaseUtils.setEnv(map);

kubeClient = server.getClient().inNamespace(NAMESPACE);
flinkKubeClient = new Fabric8FlinkKubeClient(flinkConfig, kubeClient, Executors::newDirectExecutorService);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,17 @@
import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.DeploymentOptionsInternal;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.executors.KubernetesSessionClusterExecutor;

import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import java.util.Map;

Expand All @@ -46,9 +49,14 @@
*/
public class KubernetesSessionCliTest {

@Rule
public TemporaryFolder tmp = new TemporaryFolder();

@Test
public void testKubernetesSessionCliSetsDeploymentTargetCorrectly() throws CliArgsException {
final KubernetesSessionCli cli = new KubernetesSessionCli(new Configuration());
final KubernetesSessionCli cli = new KubernetesSessionCli(
new Configuration(),
tmp.getRoot().getAbsolutePath());

final String[] args = {};
final Configuration configuration = cli.getEffectiveConfiguration(args);
Expand All @@ -59,7 +67,9 @@ public void testKubernetesSessionCliSetsDeploymentTargetCorrectly() throws CliAr
@Test
public void testDynamicProperties() throws Exception {

final KubernetesSessionCli cli = new KubernetesSessionCli(new Configuration());
final KubernetesSessionCli cli = new KubernetesSessionCli(
new Configuration(),
tmp.getRoot().getAbsolutePath());
final String[] args = new String[] {
"-e", KubernetesSessionClusterExecutor.NAME,
"-Dakka.ask.timeout=5 min",
Expand All @@ -72,9 +82,10 @@ public void testDynamicProperties() throws Exception {
Assert.assertNotNull(clientFactory);

final Map<String, String> executorConfigMap = executorConfig.toMap();
assertEquals(3, executorConfigMap.size());
assertEquals(4, executorConfigMap.size());
assertEquals("5 min", executorConfigMap.get("akka.ask.timeout"));
assertEquals("-DappName=foobar", executorConfigMap.get("env.java.opts"));
assertEquals(tmp.getRoot().getAbsolutePath(), executorConfig.get(DeploymentOptionsInternal.CONF_DIR));
assertTrue(executorConfigMap.containsKey(DeploymentOptions.TARGET.key()));
}

Expand Down Expand Up @@ -131,7 +142,9 @@ public void testCommandLineClusterSpecification() throws Exception {
"-D" + TaskManagerOptions.NUM_TASK_SLOTS.key() + "=" + slotsPerTaskManager
};

final KubernetesSessionCli cli = new KubernetesSessionCli(configuration);
final KubernetesSessionCli cli = new KubernetesSessionCli(
configuration,
tmp.getRoot().getAbsolutePath());

Configuration executorConfig = cli.getEffectiveConfiguration(args);
ClusterClientFactory<String> clientFactory = getClusterClientFactory(executorConfig);
Expand All @@ -157,7 +170,9 @@ public void testConfigurationClusterSpecification() throws Exception {
configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, slotsPerTaskManager);

final String[] args = {"-e", KubernetesSessionClusterExecutor.NAME};
final KubernetesSessionCli cli = new KubernetesSessionCli(configuration);
final KubernetesSessionCli cli = new KubernetesSessionCli(
configuration,
tmp.getRoot().getAbsolutePath());

Configuration executorConfig = cli.getEffectiveConfiguration(args);
ClusterClientFactory<String> clientFactory = getClusterClientFactory(executorConfig);
Expand Down Expand Up @@ -220,7 +235,9 @@ public void testHeapMemoryPropertyWithOldConfigKey() throws Exception {
configuration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB, 2048);
configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB, 4096);

final KubernetesSessionCli cli = new KubernetesSessionCli(configuration);
final KubernetesSessionCli cli = new KubernetesSessionCli(
configuration,
tmp.getRoot().getAbsolutePath());

final Configuration executorConfig = cli.getEffectiveConfiguration(new String[]{});
final ClusterClientFactory<String> clientFactory = getClusterClientFactory(executorConfig);
Expand Down Expand Up @@ -258,6 +275,8 @@ private KubernetesSessionCli createFlinkKubernetesCustomCliWithJmAndTmTotalMemor
Configuration configuration = new Configuration();
configuration.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(totalMemory));
configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(totalMemory));
return new KubernetesSessionCli(configuration);
return new KubernetesSessionCli(
configuration,
tmp.getRoot().getAbsolutePath());
}
}
Loading

0 comments on commit a674b5e

Please sign in to comment.