Skip to content
Merged
Show file tree
Hide file tree
Changes from 7 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
Expand Up @@ -34,6 +34,7 @@
import java.util.Map;

import static java.util.Map.entry;
import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode;

/** client code for ../basic.yaml. */
public class Basic {
Expand All @@ -49,7 +50,7 @@ public static void main(String[] args) {
flinkDeploymentSpec.setImage("flink:1.19");
Map<String, String> flinkConfiguration =
Map.ofEntries(entry("taskmanager.numberOfTaskSlots", "2"));
flinkDeploymentSpec.setFlinkConfiguration(flinkConfiguration);
flinkDeploymentSpec.setFlinkConfiguration(toJsonNode(flinkConfiguration));
flinkDeployment.setSpec(flinkDeploymentSpec);
flinkDeploymentSpec.setServiceAccount("flink");
JobManagerSpec jobManagerSpec = new JobManagerSpec();
Expand Down
14 changes: 14 additions & 0 deletions flink-kubernetes-operator-api/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,13 @@ under the License.
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This dependency is now added twice if I see correctly

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good catch fixed

<version>${assertj.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime</artifactId>
Expand All @@ -138,6 +145,13 @@ under the License.
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>
<version>${assertj.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<profiles>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,13 @@
import org.apache.flink.kubernetes.operator.api.diff.Diffable;
import org.apache.flink.kubernetes.operator.api.diff.SpecDiff;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import lombok.AllArgsConstructor;
import lombok.Data;
import lombok.NoArgsConstructor;
import lombok.experimental.SuperBuilder;

import java.util.Map;

/** The common spec. */
@Experimental
@Data
Expand Down Expand Up @@ -57,5 +57,6 @@ public abstract class AbstractFlinkSpec implements Diffable<AbstractFlinkSpec> {
type = DiffType.SCALE,
mode = KubernetesDeploymentMode.NATIVE)
})
private Map<String, String> flinkConfiguration;
@JsonDeserialize(using = JsonNodeNullDeserializer.class)
private JsonNode flinkConfiguration;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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.api.spec;

import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer;

/** Makes sure JsonNode is properly deserialized to null not NullNode. */
public class JsonNodeNullDeserializer extends JsonNodeDeserializer {

@Override
public JsonNode getNullValue(DeserializationContext ctxt) {
return null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,30 @@

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

import org.apache.flink.configuration.Configuration;
import org.apache.flink.kubernetes.operator.api.AbstractFlinkResource;
import org.apache.flink.kubernetes.operator.api.reconciler.ReconciliationMetadata;
import org.apache.flink.kubernetes.operator.api.spec.AbstractFlinkSpec;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.NullNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;

import javax.annotation.Nullable;

import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;

/** Spec utilities. */
public class SpecUtils {
public static final String INTERNAL_METADATA_JSON_KEY = "resource_metadata";
private static final ObjectMapper objectMapper = new ObjectMapper();
private static final ObjectMapper yamlObjectMapper = new ObjectMapper(new YAMLFactory());

/**
* Deserializes the spec and custom metadata object from JSON.
Expand Down Expand Up @@ -120,4 +130,87 @@ public static <T> T clone(T object) {
throw new IllegalStateException(e);
}
}

public static JsonNode toJsonNode(Map<String, String> properties) {
ObjectNode jsonNode = yamlObjectMapper.createObjectNode();
for (Map.Entry<String, String> entry : properties.entrySet()) {
jsonNode.put(entry.getKey(), entry.getValue());
}
return jsonNode;
}

public static Map<String, String> toStringMap(JsonNode node) {
if (node == null) {
return new HashMap<>();
}
if (node instanceof NullNode) {
return new HashMap<>();
}
Map<String, String> flatMap = new HashMap<>();
flattenHelper(node, "", flatMap);
return flatMap;
}

private static void flattenHelper(
JsonNode node, String parentKey, Map<String, String> flatMap) {
if (node.isObject()) {
Iterator<Map.Entry<String, JsonNode>> fields = node.fields();
while (fields.hasNext()) {
Map.Entry<String, JsonNode> field = fields.next();
String newKey =
parentKey.isEmpty() ? field.getKey() : parentKey + "." + field.getKey();
flattenHelper(field.getValue(), newKey, flatMap);
}
} else if (node.isArray()) {
for (int i = 0; i < node.size(); i++) {
String newKey = parentKey + "[" + i + "]";
flattenHelper(node.get(i), newKey, flatMap);
}
} else {
// Store values as strings
flatMap.put(parentKey, node.asText());
}
}

public static void addConfigProperties(AbstractFlinkSpec spec, Map<String, String> properties) {
spec.setFlinkConfiguration(addProperties(spec.getFlinkConfiguration(), properties));
}

public static void addConfigProperty(AbstractFlinkSpec spec, String key, String value) {
spec.setFlinkConfiguration(addProperties(spec.getFlinkConfiguration(), Map.of(key, value)));
}

public static void removeConfigProperties(AbstractFlinkSpec spec, Set<String> keys) {
spec.setFlinkConfiguration(removeProperties(spec.getFlinkConfiguration(), keys));
}

public static void removeConfigProperties(AbstractFlinkSpec spec, String... keys) {
spec.setFlinkConfiguration(removeProperties(spec.getFlinkConfiguration(), Set.of(keys)));
}

public static JsonNode addProperties(JsonNode node, Map<String, String> properties) {
var map = toStringMap(node);
map.putAll(properties);
return mapToJsonNode(map);
}

public static JsonNode removeProperty(JsonNode node, String key) {
var map = toStringMap(node);
map.remove(key);
return mapToJsonNode(map);
}

public static JsonNode removeProperties(JsonNode node, Set<String> keys) {
var map = toStringMap(node);
map.keySet().removeAll(keys);
return mapToJsonNode(map);
}

public static JsonNode mapToJsonNode(Map<String, String> config) {
return yamlObjectMapper.valueToTree(config);
}

public static JsonNode configurationToJsonNode(Configuration configuration) {
return mapToJsonNode(configuration.toMap());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,8 @@
import java.util.Map;
import java.util.UUID;

import static org.apache.flink.kubernetes.operator.api.utils.SpecUtils.toJsonNode;

/** Base Testing utilities. */
public class BaseTestUtils {

Expand Down Expand Up @@ -156,7 +158,7 @@ public static FlinkSessionJob buildSessionJob(
.upgradeMode(UpgradeMode.STATELESS)
.state(jobState)
.build())
.flinkConfiguration(conf)
.flinkConfiguration(toJsonNode(conf))
.build());
return sessionJob;
}
Expand Down Expand Up @@ -186,7 +188,7 @@ public static FlinkDeploymentSpec getTestFlinkDeploymentSpec(FlinkVersion versio
.imagePullPolicy(IMAGE_POLICY)
.serviceAccount(SERVICE_ACCOUNT)
.flinkVersion(version)
.flinkConfiguration(conf)
.flinkConfiguration(toJsonNode(conf))
.jobManager(new JobManagerSpec(new Resource(1.0, "2048m", "2G"), 1, null))
.taskManager(new TaskManagerSpec(new Resource(1.0, "2048m", "2G"), null, null))
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,16 +24,22 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
import org.junit.jupiter.api.Test;

import java.util.Map;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;

/** Test for {@link SpecUtils}. */
public class SpecUtilsTest {
class SpecUtilsTest {

private static final ObjectMapper yamlObjectMapper = new ObjectMapper(new YAMLFactory());

@Test
public void testSpecSerializationWithVersion() throws JsonProcessingException {
void testSpecSerializationWithVersion() throws JsonProcessingException {
FlinkDeployment app = BaseTestUtils.buildApplicationCluster();
String serialized = SpecUtils.writeSpecWithMeta(app.getSpec(), app);
ObjectNode node = (ObjectNode) new ObjectMapper().readTree(serialized);
Expand All @@ -56,7 +62,7 @@ public void testSpecSerializationWithVersion() throws JsonProcessingException {
}

@Test
public void testSpecSerializationWithoutGeneration() throws JsonProcessingException {
void testSpecSerializationWithoutGeneration() throws JsonProcessingException {
// with regards to ReconcialiationMetadata & SpecWithMeta
FlinkDeployment app = BaseTestUtils.buildApplicationCluster();
app.getMetadata().setGeneration(12L);
Expand All @@ -76,4 +82,57 @@ public void testSpecSerializationWithoutGeneration() throws JsonProcessingExcept
var migrated = SpecUtils.deserializeSpecWithMeta(oldSerialized, FlinkDeploymentSpec.class);
assertNull(migrated.getMeta());
}

@Test
void convertsStringMapToJsonNode() {
var map = Map.of("k1", "v1", "k2", "v2", "k3.nested", "v3");
var node = SpecUtils.mapToJsonNode(map);

assertThat(node).hasSize(3);
assertThat(node.get("k1").asText()).isEqualTo("v1");
assertThat(node.get("k2").asText()).isEqualTo("v2");
assertThat(node.get("k3.nested").asText()).isEqualTo("v3");
}

@Test
void convertsJsonNodeToMap() throws JsonProcessingException {
var node =
yamlObjectMapper.readTree("k1: v1 \n" + "k2: v2 \n" + "k3:\n" + " nested: v3\n");

var map = SpecUtils.toStringMap(node);
assertThat(map).hasSize(3);
assertThat(map.get("k1")).isEqualTo("v1");
assertThat(map.get("k2")).isEqualTo("v2");
assertThat(map.get("k3.nested")).isEqualTo("v3");
}

@Test
void addConfigPropertyToSpec() {
var spec = new FlinkDeploymentSpec();

SpecUtils.addConfigProperty(spec, "k1", "v1");

assertThat(spec.getFlinkConfiguration().get("k1").asText()).isEqualTo("v1");
}

@Test
void addConfigPropertiesToSpec() {
var spec = new FlinkDeploymentSpec();

SpecUtils.addConfigProperties(spec, Map.of("k1", "v1", "k2", "v2"));

assertThat(spec.getFlinkConfiguration().get("k1").asText()).isEqualTo("v1");
assertThat(spec.getFlinkConfiguration().get("k2").asText()).isEqualTo("v2");
}

@Test
void removeConfigPropertiesFromSpec() {
var spec = new FlinkDeploymentSpec();
SpecUtils.addConfigProperties(spec, Map.of("k1", "v1", "k2", "v2"));

SpecUtils.removeConfigProperties(spec, "k1");

assertThat(spec.getFlinkConfiguration().get("k1")).isNull();
assertThat(spec.getFlinkConfiguration().get("k2").asText()).isEqualTo("v2");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
import org.apache.flink.kubernetes.operator.api.spec.Resource;
import org.apache.flink.kubernetes.operator.api.utils.SpecUtils;
import org.apache.flink.kubernetes.operator.config.FlinkConfigBuilder;

import io.javaoperatorsdk.operator.processing.event.ResourceID;
Expand All @@ -45,13 +46,11 @@ public class KubernetesScalingRealizer
@Override
public void realizeParallelismOverrides(
KubernetesJobAutoScalerContext context, Map<String, String> parallelismOverrides) {

context.getResource()
.getSpec()
.getFlinkConfiguration()
.put(
SpecUtils.addConfigProperties(
context.getResource().getSpec(),
Map.of(
PipelineOptions.PARALLELISM_OVERRIDES.key(),
getOverrideString(context, parallelismOverrides));
getOverrideString(context, parallelismOverrides)));
}

@Override
Expand All @@ -63,15 +62,17 @@ public void realizeConfigOverrides(
}
FlinkDeployment flinkDeployment = ((FlinkDeployment) context.getResource());
// Apply config overrides
Map<String, String> flinkConf = flinkDeployment.getSpec().getFlinkConfiguration();
for (String keyToRemove : configChanges.getRemovals()) {
flinkConf.remove(keyToRemove);
}
flinkConf.putAll(configChanges.getOverrides());

SpecUtils.removeConfigProperties(flinkDeployment.getSpec(), configChanges.getRemovals());
SpecUtils.addConfigProperties(flinkDeployment.getSpec(), configChanges.getOverrides());

// Update total memory in spec
var totalMemoryOverride =
MemoryTuning.getTotalMemory(Configuration.fromMap(flinkConf), context);
MemoryTuning.getTotalMemory(
Configuration.fromMap(
SpecUtils.toStringMap(
flinkDeployment.getSpec().getFlinkConfiguration())),
context);
if (totalMemoryOverride.compareTo(MemorySize.ZERO) <= 0) {
LOG.warn("Total memory override {} is not valid", totalMemoryOverride);
return;
Expand Down
Loading