Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,212 @@
/*
* 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.kubernetes.operator.utils;

import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.configuration.KubernetesDeploymentTarget;
import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
import org.apache.flink.kubernetes.operator.crd.spec.FlinkDeploymentSpec;
import org.apache.flink.kubernetes.operator.crd.spec.Resource;
import org.apache.flink.util.StringUtils;

import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.client.internal.SerializationUtils;

import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.Files;
import java.util.Collections;

import static org.apache.flink.kubernetes.operator.utils.FlinkUtils.mergePodTemplates;

/** Builder to get effective flink config from {@link FlinkDeployment}. */
public class FlinkConfigBuilder {
private final FlinkDeployment deploy;
private final FlinkDeploymentSpec spec;
private final Configuration effectiveConfig;

public FlinkConfigBuilder(FlinkDeployment deploy) {
this.deploy = deploy;
this.spec = this.deploy.getSpec();
this.effectiveConfig =
FlinkUtils.loadConfiguration(
System.getenv().get(ConfigConstants.ENV_FLINK_CONF_DIR));
}

public FlinkConfigBuilder applyImage() {
if (!StringUtils.isNullOrWhitespaceOnly(spec.getImage())) {
effectiveConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE, spec.getImage());
}
return this;
}

public FlinkConfigBuilder applyImagePullPolicy() {
if (!StringUtils.isNullOrWhitespaceOnly(spec.getImagePullPolicy())) {
effectiveConfig.set(
KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY,
KubernetesConfigOptions.ImagePullPolicy.valueOf(spec.getImagePullPolicy()));
}
return this;
}

public FlinkConfigBuilder applyFlinkConfiguration() {
// Parse config from spec's flinkConfiguration
if (spec.getFlinkConfiguration() != null && !spec.getFlinkConfiguration().isEmpty()) {
spec.getFlinkConfiguration().forEach(effectiveConfig::setString);
}
return this;
}

public FlinkConfigBuilder applyCommonPodTemplate() throws IOException {
if (spec.getPodTemplate() != null) {
effectiveConfig.set(
KubernetesConfigOptions.KUBERNETES_POD_TEMPLATE,
createTempFile(spec.getPodTemplate()));
}
return this;
}

public FlinkConfigBuilder applyIngressDomain() {
// Web UI
if (spec.getIngressDomain() != null) {
effectiveConfig.set(
KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE,
KubernetesConfigOptions.ServiceExposedType.ClusterIP);
}
return this;
}

public FlinkConfigBuilder applyJobManagerSpec() throws IOException {
if (spec.getJobManager() != null) {
if (spec.getJobManager() != null) {
setResource(spec.getJobManager().getResource(), effectiveConfig, true);
setPodTemplate(
spec.getPodTemplate(),
spec.getJobManager().getPodTemplate(),
effectiveConfig,
true);
}
}
return this;
}

public FlinkConfigBuilder applyTaskManagerSpec() throws IOException {
if (spec.getTaskManager() != null) {
setResource(spec.getTaskManager().getResource(), effectiveConfig, false);
setPodTemplate(
spec.getPodTemplate(),
spec.getTaskManager().getPodTemplate(),
effectiveConfig,
false);
if (spec.getTaskManager().getTaskSlots() > 0) {
effectiveConfig.set(
TaskManagerOptions.NUM_TASK_SLOTS, spec.getTaskManager().getTaskSlots());
}
}
return this;
}

public FlinkConfigBuilder applyJobOrSessionSpec() throws URISyntaxException {
if (spec.getJob() != null) {
effectiveConfig.set(
DeploymentOptions.TARGET, KubernetesDeploymentTarget.APPLICATION.getName());
final URI uri = new URI(spec.getJob().getJarURI());
effectiveConfig.set(PipelineOptions.JARS, Collections.singletonList(uri.toString()));

if (spec.getJob().getParallelism() > 0) {
effectiveConfig.set(
CoreOptions.DEFAULT_PARALLELISM, spec.getJob().getParallelism());
}
} else {
effectiveConfig.set(
DeploymentOptions.TARGET, KubernetesDeploymentTarget.SESSION.getName());
}
return this;
}

public Configuration build() {

// Set cluster config
final String namespace = deploy.getMetadata().getNamespace();
final String clusterId = deploy.getMetadata().getName();
effectiveConfig.setString(KubernetesConfigOptions.NAMESPACE, namespace);
effectiveConfig.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);
return effectiveConfig;
}

public static Configuration buildFrom(FlinkDeployment dep)
throws IOException, URISyntaxException {
return new FlinkConfigBuilder(dep)
.applyFlinkConfiguration()
.applyImage()
.applyImagePullPolicy()
.applyCommonPodTemplate()
.applyIngressDomain()
.applyJobManagerSpec()
.applyTaskManagerSpec()
.applyJobOrSessionSpec()
.build();
}

private static void setResource(
Resource resource, Configuration effectiveConfig, boolean isJM) {
if (resource != null) {
final ConfigOption<MemorySize> memoryConfigOption =
isJM
? JobManagerOptions.TOTAL_PROCESS_MEMORY
: TaskManagerOptions.TOTAL_PROCESS_MEMORY;
final ConfigOption<Double> cpuConfigOption =
isJM
? KubernetesConfigOptions.JOB_MANAGER_CPU
: KubernetesConfigOptions.TASK_MANAGER_CPU;
effectiveConfig.setString(memoryConfigOption.key(), resource.getMemory());
effectiveConfig.setDouble(cpuConfigOption.key(), resource.getCpu());
}
}

private static void setPodTemplate(
Pod basicPod, Pod appendPod, Configuration effectiveConfig, boolean isJM)
throws IOException {
if (basicPod != null) {
final ConfigOption<String> podConfigOption =
isJM
? KubernetesConfigOptions.JOB_MANAGER_POD_TEMPLATE
: KubernetesConfigOptions.TASK_MANAGER_POD_TEMPLATE;
effectiveConfig.setString(
podConfigOption, createTempFile(mergePodTemplates(basicPod, appendPod)));
}
}

private static String createTempFile(Pod podTemplate) throws IOException {
final File tmp = File.createTempFile("podTemplate_", ".yaml");
Files.write(tmp.toPath(), SerializationUtils.dumpAsYaml(podTemplate).getBytes());
tmp.deleteOnExit();
return tmp.getAbsolutePath();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,35 +17,19 @@

package org.apache.flink.kubernetes.operator.utils;

import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
import org.apache.flink.kubernetes.configuration.KubernetesDeploymentTarget;
import org.apache.flink.kubernetes.operator.crd.FlinkDeployment;
import org.apache.flink.kubernetes.operator.crd.spec.FlinkDeploymentSpec;
import org.apache.flink.util.StringUtils;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.client.KubernetesClient;
import io.fabric8.kubernetes.client.internal.SerializationUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.nio.file.Files;
import java.util.Collections;
import java.util.Iterator;

/** Flink Utility methods used by the operator. */
Expand All @@ -55,109 +39,9 @@ public class FlinkUtils {
private static final ObjectMapper MAPPER = new ObjectMapper();

public static Configuration getEffectiveConfig(FlinkDeployment flinkApp) {
String namespace = flinkApp.getMetadata().getNamespace();
String clusterId = flinkApp.getMetadata().getName();
FlinkDeploymentSpec spec = flinkApp.getSpec();

try {
String flinkConfDir = System.getenv().get(ConfigConstants.ENV_FLINK_CONF_DIR);
Configuration effectiveConfig = loadConfiguration(flinkConfDir);

effectiveConfig.setString(KubernetesConfigOptions.NAMESPACE, namespace);
effectiveConfig.setString(KubernetesConfigOptions.CLUSTER_ID, clusterId);

if (spec.getIngressDomain() != null) {
effectiveConfig.set(
KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE,
KubernetesConfigOptions.ServiceExposedType.ClusterIP);
}

if (spec.getJob() != null) {
effectiveConfig.set(
DeploymentOptions.TARGET, KubernetesDeploymentTarget.APPLICATION.getName());
} else {
effectiveConfig.set(
DeploymentOptions.TARGET, KubernetesDeploymentTarget.SESSION.getName());
}

if (!StringUtils.isNullOrWhitespaceOnly(spec.getImage())) {
effectiveConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE, spec.getImage());
}

if (!StringUtils.isNullOrWhitespaceOnly(spec.getImagePullPolicy())) {
effectiveConfig.set(
KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY,
KubernetesConfigOptions.ImagePullPolicy.valueOf(spec.getImagePullPolicy()));
}

if (spec.getFlinkConfiguration() != null && !spec.getFlinkConfiguration().isEmpty()) {
spec.getFlinkConfiguration().forEach(effectiveConfig::setString);
}

// Pod template
if (spec.getPodTemplate() != null) {
effectiveConfig.set(
KubernetesConfigOptions.KUBERNETES_POD_TEMPLATE,
createTempFile(spec.getPodTemplate()));
}

if (spec.getJobManager() != null) {
if (spec.getJobManager().getResource() != null) {
effectiveConfig.setString(
JobManagerOptions.TOTAL_PROCESS_MEMORY.key(),
spec.getJobManager().getResource().getMemory());
effectiveConfig.set(
KubernetesConfigOptions.JOB_MANAGER_CPU,
spec.getJobManager().getResource().getCpu());
}

if (spec.getJobManager().getPodTemplate() != null) {
effectiveConfig.set(
KubernetesConfigOptions.JOB_MANAGER_POD_TEMPLATE,
createTempFile(
mergePodTemplates(
spec.getPodTemplate(),
spec.getJobManager().getPodTemplate())));
}
}

if (spec.getTaskManager() != null) {
if (spec.getTaskManager().getTaskSlots() > 0) {
effectiveConfig.set(
TaskManagerOptions.NUM_TASK_SLOTS,
spec.getTaskManager().getTaskSlots());
}

if (spec.getTaskManager().getResource() != null) {
effectiveConfig.setString(
TaskManagerOptions.TOTAL_PROCESS_MEMORY.key(),
spec.getTaskManager().getResource().getMemory());
effectiveConfig.set(
KubernetesConfigOptions.TASK_MANAGER_CPU,
spec.getTaskManager().getResource().getCpu());
}

if (spec.getTaskManager().getPodTemplate() != null) {
effectiveConfig.set(
KubernetesConfigOptions.TASK_MANAGER_POD_TEMPLATE,
createTempFile(
mergePodTemplates(
spec.getPodTemplate(),
spec.getTaskManager().getPodTemplate())));
}
}

if (spec.getJob() != null) {
final URI uri = new URI(spec.getJob().getJarURI());
effectiveConfig.set(
PipelineOptions.JARS, Collections.singletonList(uri.toString()));

if (spec.getJob().getParallelism() > 0) {
effectiveConfig.set(
CoreOptions.DEFAULT_PARALLELISM, spec.getJob().getParallelism());
}
}

final Configuration effectiveConfig = FlinkConfigBuilder.buildFrom(flinkApp);
LOG.debug("Effective config: {}", effectiveConfig);
return effectiveConfig;
} catch (Exception e) {
throw new RuntimeException("Failed to load configuration", e);
Expand All @@ -172,13 +56,6 @@ public static Configuration loadConfiguration(String confDir) {
return configuration;
}

private static String createTempFile(Pod podTemplate) throws IOException {
File tmp = File.createTempFile("podTemplate_", ".yaml");
Files.write(tmp.toPath(), SerializationUtils.dumpAsYaml(podTemplate).getBytes());
tmp.deleteOnExit();
return tmp.getAbsolutePath();
}

public static Pod mergePodTemplates(Pod toPod, Pod fromPod) {
if (fromPod == null) {
return toPod;
Expand Down
Loading