Skip to content
Merged
Show file tree
Hide file tree
Changes from 18 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
20 changes: 18 additions & 2 deletions docs/content/docs/custom-resource/reference.md
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,22 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r
| Parameter | Type | Docs |
| ----------| ---- | ---- |

### ConfigObjectNode
**Class**: org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode

**Description**: Allows parsing configurations as YAML, and adds related utility methods.

| Parameter | Type | Docs |
| ----------| ---- | ---- |

### ConfigObjectNodeDeserializer
**Class**: org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNodeDeserializer

**Description**: Allows to deserialize to ConfigObjectNode.

| Parameter | Type | Docs |
| ----------| ---- | ---- |

### FlinkDeploymentSpec
**Class**: org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec

Expand All @@ -66,7 +82,7 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r
| ----------| ---- | ---- |
| job | org.apache.flink.kubernetes.operator.api.spec.JobSpec | Job specification for application deployments/session job. Null for session clusters. |
| restartNonce | java.lang.Long | Nonce used to manually trigger restart for the cluster/session job. In order to trigger restart, change the number to a different non-null value. |
| flinkConfiguration | java.util.Map<java.lang.String,java.lang.String> | Flink configuration overrides for the Flink deployment or Flink session job. |
| flinkConfiguration | org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode | Flink configuration overrides for the Flink deployment or Flink session job. |
| image | java.lang.String | Flink docker image used to start the Job and TaskManager pods. |
| imagePullPolicy | java.lang.String | Image pull policy of the Flink docker image. |
| serviceAccount | java.lang.String | Kubernetes service used by the Flink deployment. |
Expand All @@ -87,7 +103,7 @@ This serves as a full reference for FlinkDeployment and FlinkSessionJob custom r
| ----------| ---- | ---- |
| job | org.apache.flink.kubernetes.operator.api.spec.JobSpec | Job specification for application deployments/session job. Null for session clusters. |
| restartNonce | java.lang.Long | Nonce used to manually trigger restart for the cluster/session job. In order to trigger restart, change the number to a different non-null value. |
| flinkConfiguration | java.util.Map<java.lang.String,java.lang.String> | Flink configuration overrides for the Flink deployment or Flink session job. |
| flinkConfiguration | org.apache.flink.kubernetes.operator.api.spec.ConfigObjectNode | Flink configuration overrides for the Flink deployment or Flink session job. |
| deploymentName | java.lang.String | The name of the target session cluster deployment. |

### FlinkStateSnapshotSpec
Expand Down
24 changes: 13 additions & 11 deletions e2e-tests/data/autoscaler.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -30,17 +30,19 @@ spec:
annotations:
nginx.ingress.kubernetes.io/rewrite-target: "/$2"
flinkConfiguration:
taskmanager.numberOfTaskSlots: "2"
high-availability.type: kubernetes
high-availability.storageDir: file:///opt/flink/volume/flink-ha
state.checkpoints.dir: file:///opt/flink/volume/flink-cp
state.savepoints.dir: file:///opt/flink/volume/flink-sp

job.autoscaler.enabled: "true"
job.autoscaler.scaling.enabled: "true"
job.autoscaler.stabilization.interval: "5s"
job.autoscaler.metrics.window: "1m"
job.autoscaler.scale-down.interval: "0m"
taskmanager.numberOfTaskSlots: 2
high-availability:
type: kubernetes
storageDir: file:///opt/flink/volume/flink-ha
state:
checkpoints.dir: file:///opt/flink/volume/flink-cp
savepoints.dir: file:///opt/flink/volume/flink-sp
job.autoscaler:
enabled: "true"
scaling.enabled: "true"
stabilization.interval: "5s"
metrics.window: "1m"
scale-down.interval: "0m"

# Invalid Validations for testing autoscaler configurations
# kubernetes.operator.job.autoscaler.scale-down.max-factor: "-0.6"
Expand Down
4 changes: 2 additions & 2 deletions e2e-tests/data/flinkdep-batch-cr.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@ spec:
annotations:
nginx.ingress.kubernetes.io/rewrite-target: "/$2"
flinkConfiguration:
taskmanager.numberOfTaskSlots: "2"
kubernetes.operator.snapshot.resource.enabled: "false"
taskmanager.numberOfTaskSlots: 2
kubernetes.operator.snapshot.resource.enabled: false
serviceAccount: flink
podTemplate:
spec:
Expand Down
14 changes: 8 additions & 6 deletions e2e-tests/data/flinkdep-cr.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,14 @@ spec:
annotations:
nginx.ingress.kubernetes.io/rewrite-target: "/$2"
flinkConfiguration:
taskmanager.numberOfTaskSlots: "2"
high-availability.type: kubernetes
high-availability.storageDir: file:///opt/flink/volume/flink-ha
state.checkpoints.dir: file:///opt/flink/volume/flink-cp
state.savepoints.dir: file:///opt/flink/volume/flink-sp
kubernetes.operator.snapshot.resource.enabled: "false"
taskmanager.numberOfTaskSlots: 2
high-availability:
type: kubernetes
storageDir: file:///opt/flink/volume/flink-ha
state:
checkpoints.dir: file:///opt/flink/volume/flink-cp
savepoints.dir: file:///opt/flink/volume/flink-sp
kubernetes.operator.snapshot.resource.enabled: false
serviceAccount: flink
podTemplate:
spec:
Expand Down
12 changes: 7 additions & 5 deletions e2e-tests/data/multi-sessionjob.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,13 @@ spec:
annotations:
nginx.ingress.kubernetes.io/rewrite-target: "/$2"
flinkConfiguration:
taskmanager.numberOfTaskSlots: "2"
high-availability.type: kubernetes
high-availability.storageDir: file:///opt/flink/volume/flink-ha
state.checkpoints.dir: file:///opt/flink/volume/flink-cp
state.savepoints.dir: file:///opt/flink/volume/flink-sp
taskmanager.numberOfTaskSlots: 2
high-availability:
type: kubernetes
storageDir: file:///opt/flink/volume/flink-ha
state:
checkpoints.dir: file:///opt/flink/volume/flink-cp
savepoints.dir: file:///opt/flink/volume/flink-sp
serviceAccount: flink
podTemplate:
spec:
Expand Down
14 changes: 8 additions & 6 deletions e2e-tests/data/sessionjob-cr.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,14 @@ spec:
annotations:
nginx.ingress.kubernetes.io/rewrite-target: "/$2"
flinkConfiguration:
taskmanager.numberOfTaskSlots: "2"
high-availability.type: kubernetes
high-availability.storageDir: file:///opt/flink/volume/flink-ha
state.checkpoints.dir: file:///opt/flink/volume/flink-cp
state.savepoints.dir: file:///opt/flink/volume/flink-sp
kubernetes.operator.snapshot.resource.enabled: "false"
taskmanager.numberOfTaskSlots: 2
high-availability:
type: kubernetes
storageDir: file:///opt/flink/volume/flink-ha
state:
checkpoints.dir: file:///opt/flink/volume/flink-cp
savepoints.dir: file:///opt/flink/volume/flink-sp
kubernetes.operator.snapshot.resource.enabled: false
serviceAccount: flink
podTemplate:
spec:
Expand Down
34 changes: 34 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,33 @@ 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>

<dependency>
<groupId>io.fabric8</groupId>
<artifactId>kube-api-test-client-inject</artifactId>
<version>${fabric8.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>io.fabric8</groupId>
<artifactId>kubernetes-httpclient-${fabric8.httpclient.impl}</artifactId>
<version>${fabric8.version}</version>
<exclusions>
<exclusion>
<groupId>com.squareup.okhttp3</groupId>
<artifactId>okhttp</artifactId>
</exclusion>
</exclusions>
<scope>test</scope>
</dependency>
</dependencies>

<profiles>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,18 @@
package org.apache.flink.kubernetes.operator.api.spec;

import org.apache.flink.annotation.Experimental;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.kubernetes.operator.api.diff.DiffType;
import org.apache.flink.kubernetes.operator.api.diff.Diffable;
import org.apache.flink.kubernetes.operator.api.diff.SpecDiff;

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

import java.util.Map;

Expand Down Expand Up @@ -57,5 +61,18 @@ public abstract class AbstractFlinkSpec implements Diffable<AbstractFlinkSpec> {
type = DiffType.SCALE,
mode = KubernetesDeploymentMode.NATIVE)
})
private Map<String, String> flinkConfiguration;
@JsonDeserialize(using = ConfigObjectNodeDeserializer.class)
private ConfigObjectNode flinkConfiguration = new ConfigObjectNode();

@Tolerate
@JsonIgnore
public void setFlinkConfiguration(Map<String, String> config) {
flinkConfiguration.setAllFrom(config);
}

@Tolerate
@JsonIgnore
public void setFlinkConfiguration(Configuration config) {
setFlinkConfiguration(config.toMap());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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 org.apache.flink.configuration.Configuration;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;

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

/** Allows parsing configurations as YAML, and adds related utility methods. */
public class ConfigObjectNode extends ObjectNode {

public ConfigObjectNode() {
this(JsonNodeFactory.instance);
}

public ConfigObjectNode(JsonNodeFactory nc, Map<String, JsonNode> kids) {
super(nc, kids);
}

public ConfigObjectNode(JsonNodeFactory nc) {
super(nc);
}

public void remove(String... names) {
remove(Arrays.asList(names));
}

public void putAllFrom(Map<String, String> value) {
value.forEach(this::put);
}

public void setAllFrom(Map<String, String> value) {
removeAll();
putAllFrom(value);
}

public Map<String, String> asFlatMap() {
Map<String, String> flatMap = new HashMap<>();
flattenHelper(this, "", flatMap);
return flatMap;
}

public Configuration asConfiguration() {
return Configuration.fromMap(asFlatMap());
}

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());
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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.core.JsonParser;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonDeserializer;
import com.fasterxml.jackson.databind.node.ObjectNode;

import java.io.IOException;

/** Allows to deserialize to ConfigObjectNode. */
public class ConfigObjectNodeDeserializer extends JsonDeserializer<ConfigObjectNode> {

@Override
public ConfigObjectNode deserialize(
JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException {
ObjectNode tree = jsonParser.readValueAsTree();
var res = new ConfigObjectNode();
tree.fields().forEachRemaining(entry -> res.set(entry.getKey(), entry.getValue()));
return res;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ public static String writeSpecWithMeta(
}
}

// We do not have access to Flink's Preconditions from here
// We do not have access to Flink's Preconditions from here
private static <T> T checkNotNull(T object) {
if (object == null) {
throw new NullPointerException();
Expand Down
Loading