extends State {
* of values. The next time {@link #get()} is called (for the same state partition) the returned
* state will represent the updated list.
*
- * If null is passed in, the state value will remain unchanged.
+ *
If null is passed in, the behaviour is undefined (implementation related). TODO: An
+ * unified behaviour across all sub-classes.
*
* @param value The new value for the state.
* @throws Exception Thrown if the system cannot access the state.
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java
index 7508054f5c021..3ea5b16fb7f96 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java
@@ -48,10 +48,13 @@ public interface ListState extends MergingState> {
* given list of values. The next time {@link #get()} is called (for the same state partition)
* the returned state will represent the updated list.
*
- * If null or an empty list is passed in, the state value will be null.
+ *
If an empty list is passed in, the state value will be null.
+ *
+ *
Null value passed in or any null value in list is not allowed.
*
* @param values The new values for the state.
- * @throws Exception The method may forward exception thrown internally (by I/O or functions).
+ * @throws Exception The method may forward exception thrown internally (by I/O or functions, or
+ * sanity check for null value).
*/
void update(List values) throws Exception;
@@ -60,10 +63,13 @@ public interface ListState extends MergingState> {
* existing list of values. The next time {@link #get()} is called (for the same state
* partition) the returned state will represent the updated list.
*
- * If null or an empty list is passed in, the state value remains unchanged.
+ *
If an empty list is passed in, the state value remains unchanged.
+ *
+ *
Null value passed in or any null value in list is not allowed.
*
* @param values The new values to be added to the state.
- * @throws Exception The method may forward exception thrown internally (by I/O or functions).
+ * @throws Exception The method may forward exception thrown internally (by I/O or functions, or
+ * sanity check for null value).
*/
void addAll(List values) throws Exception;
}
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
index b95a1d97d883a..d17d12592d1a6 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -109,6 +109,19 @@ public class CheckpointingOptions {
.defaultValue(1)
.withDescription("The maximum number of completed checkpoints to retain.");
+ /* Option whether to clean individual checkpoint's operatorstates in parallel. If enabled,
+ * operator states are discarded in parallel using the ExecutorService passed to the cleaner.
+ * This speeds up checkpoints cleaning, but adds load to the IO.
+ */
+ @Documentation.Section(Documentation.Sections.COMMON_STATE_BACKENDS)
+ public static final ConfigOption CLEANER_PARALLEL_MODE =
+ ConfigOptions.key("state.checkpoint.cleaner.parallel-mode")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription(
+ "Option whether to discard a checkpoint's states in parallel using"
+ + " the ExecutorService passed into the cleaner");
+
/** @deprecated Checkpoints are always asynchronous. */
@Deprecated
public static final ConfigOption ASYNC_SNAPSHOTS =
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
index 58150beb4b7f5..c0b9ca8459ef1 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java
@@ -84,6 +84,13 @@ public class ClusterOptions {
+ "By default it will use 4 * the number of CPU cores (hardware contexts) that the cluster process has access to. "
+ "Increasing the pool size allows to run more IO operations concurrently.");
+ /**
+ * @deprecated Please use {@link TaskManagerOptions#TASK_MANAGER_LOAD_BALANCE_MODE} instead.
+ * Note: The 'taskmanager.load-balance.mode: SLOTS' is equal to
+ * 'cluster.evenly-spread-out-slots: true'. The 'taskmanager.load-balance.mode: NONE' is
+ * equal to 'cluster.evenly-spread-out-slots: false'.
+ */
+ @Deprecated
@Documentation.Section(Documentation.Sections.EXPERT_SCHEDULING)
public static final ConfigOption EVENLY_SPREAD_OUT_SLOTS_STRATEGY =
ConfigOptions.key("cluster.evenly-spread-out-slots")
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
index 1da99be996670..0378e380923f2 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java
@@ -507,6 +507,32 @@ public enum SchedulerType {
.withDescription(
"Configure the minimum increase in parallelism for a job to scale up.");
+ @Documentation.Section({
+ Documentation.Sections.EXPERT_SCHEDULING,
+ Documentation.Sections.ALL_JOB_MANAGER
+ })
+ public static final ConfigOption SCHEDULER_SCALING_INTERVAL_MIN =
+ key("jobmanager.adaptive-scheduler.scaling-interval.min")
+ .durationType()
+ .defaultValue(Duration.ofSeconds(30))
+ // rescaling and let the user increase the value for high workloads
+ .withDescription("Determines the minimum time between scaling operations.");
+
+ @Documentation.Section({
+ Documentation.Sections.EXPERT_SCHEDULING,
+ Documentation.Sections.ALL_JOB_MANAGER
+ })
+ public static final ConfigOption SCHEDULER_SCALING_INTERVAL_MAX =
+ key("jobmanager.adaptive-scheduler.scaling-interval.max")
+ .durationType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "Determines the maximum interval time after which a scaling operation is forced even if the %s aren't met. The scaling operation will be ignored when the resource hasn't changed. This option is disabled by default.",
+ code(MIN_PARALLELISM_INCREASE.key()))
+ .build());
+
@Documentation.Section({
Documentation.Sections.EXPERT_SCHEDULING,
Documentation.Sections.ALL_JOB_MANAGER
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
index 8b2099ee8e3df..aeeb4b4725793 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java
@@ -21,8 +21,12 @@
import org.apache.flink.annotation.Experimental;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
import static org.apache.flink.configuration.ConfigOptions.key;
+import static org.apache.flink.configuration.description.TextElement.code;
/** The set of configuration options relating to network stack. */
@PublicEvolving
@@ -585,7 +589,7 @@ public class NettyShuffleEnvironmentOptions {
.defaultValue(100)
.withDeprecatedKeys("taskmanager.net.request-backoff.initial")
.withDescription(
- "Minimum backoff in milliseconds for partition requests of input channels.");
+ "Minimum backoff in milliseconds for partition requests of local input channels.");
/** Maximum backoff for partition requests of input channels. */
@Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
@@ -595,7 +599,22 @@ public class NettyShuffleEnvironmentOptions {
.defaultValue(10000)
.withDeprecatedKeys("taskmanager.net.request-backoff.max")
.withDescription(
- "Maximum backoff in milliseconds for partition requests of input channels.");
+ "Maximum backoff in milliseconds for partition requests of local input channels.");
+
+ /** The timeout for partition request listener in result partition manager. */
+ @Documentation.Section(Documentation.Sections.ALL_TASK_MANAGER_NETWORK)
+ public static final ConfigOption NETWORK_PARTITION_REQUEST_TIMEOUT =
+ key("taskmanager.network.partition-request-timeout")
+ .durationType()
+ .defaultValue(Duration.ofSeconds(10))
+ .withDescription(
+ Description.builder()
+ .text(
+ "Timeout for an individual partition request of remote input channels. "
+ + "The partition request will finally fail if the total wait time exceeds "
+ + "twice the value of %s.",
+ code(NETWORK_REQUEST_BACKOFF_MAX.key()))
+ .build());
// ------------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
index 8f4a836d4f338..0e4bc7f7d15f3 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java
@@ -25,9 +25,12 @@
import org.apache.flink.configuration.description.Description;
import org.apache.flink.util.TimeUtils;
+import javax.annotation.Nonnull;
+
import java.time.Duration;
import java.util.HashMap;
import java.util.Map;
+import java.util.Optional;
import static org.apache.flink.configuration.ConfigOptions.key;
import static org.apache.flink.configuration.description.TextElement.code;
@@ -708,6 +711,55 @@ public class TaskManagerOptions {
"Time we wait for the timers in milliseconds to finish all pending timer threads"
+ " when the stream task is cancelled.");
+ @Documentation.Section({
+ Documentation.Sections.EXPERT_SCHEDULING,
+ Documentation.Sections.ALL_TASK_MANAGER
+ })
+ public static final ConfigOption TASK_MANAGER_LOAD_BALANCE_MODE =
+ ConfigOptions.key("taskmanager.load-balance.mode")
+ .enumType(TaskManagerLoadBalanceMode.class)
+ .defaultValue(TaskManagerLoadBalanceMode.NONE)
+ .withDescription(
+ Description.builder()
+ .text(
+ "Mode for the load-balance allocation strategy across all available %s.",
+ code("TaskManagers"))
+ .list(
+ text(
+ "The %s mode tries to spread out the slots evenly across all available %s.",
+ code(TaskManagerLoadBalanceMode.SLOTS.name()),
+ code("TaskManagers")),
+ text(
+ "The %s mode is the default mode without any specified strategy.",
+ code(TaskManagerLoadBalanceMode.NONE.name())))
+ .build());
+
+ /** Type of {@link TaskManagerOptions#TASK_MANAGER_LOAD_BALANCE_MODE}. */
+ public enum TaskManagerLoadBalanceMode {
+ NONE,
+ SLOTS;
+
+ /**
+ * The method is mainly to load the {@link
+ * TaskManagerOptions#TASK_MANAGER_LOAD_BALANCE_MODE} from {@link Configuration}, which is
+ * compatible with {@link ClusterOptions#EVENLY_SPREAD_OUT_SLOTS_STRATEGY}.
+ */
+ public static TaskManagerLoadBalanceMode loadFromConfiguration(
+ @Nonnull Configuration configuration) {
+ Optional taskManagerLoadBalanceModeOptional =
+ configuration.getOptional(TaskManagerOptions.TASK_MANAGER_LOAD_BALANCE_MODE);
+ if (taskManagerLoadBalanceModeOptional.isPresent()) {
+ return taskManagerLoadBalanceModeOptional.get();
+ }
+ boolean evenlySpreadOutSlots =
+ configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY);
+
+ return evenlySpreadOutSlots
+ ? TaskManagerLoadBalanceMode.SLOTS
+ : TaskManagerOptions.TASK_MANAGER_LOAD_BALANCE_MODE.defaultValue();
+ }
+ }
+
// ------------------------------------------------------------------------
/** Not intended to be instantiated. */
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java b/flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java
new file mode 100644
index 0000000000000..4ffae33b0808b
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/configuration/TaskManagerLoadBalanceModeTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.configuration;
+
+import org.junit.jupiter.api.Test;
+
+import static org.apache.flink.configuration.ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY;
+import static org.apache.flink.configuration.TaskManagerOptions.TASK_MANAGER_LOAD_BALANCE_MODE;
+import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test for {@link TaskManagerLoadBalanceMode}. */
+class TaskManagerLoadBalanceModeTest {
+
+ @Test
+ void testReadTaskManagerLoadBalanceMode() {
+ // Check for non-set 'taskmanager.load-balance.mode' and
+ // 'cluster.evenly-spread-out-slots: false'
+ Configuration conf1 = new Configuration();
+ assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf1))
+ .isEqualTo(TASK_MANAGER_LOAD_BALANCE_MODE.defaultValue());
+
+ // Check for non-set 'taskmanager.load-balance.mode' and
+ // 'cluster.evenly-spread-out-slots: true'
+ Configuration conf2 = new Configuration();
+ conf2.set(EVENLY_SPREAD_OUT_SLOTS_STRATEGY, true);
+ assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf2))
+ .isEqualTo(TaskManagerLoadBalanceMode.SLOTS);
+
+ // Check for setting manually 'taskmanager.load-balance.mode: NONE' and
+ // 'cluster.evenly-spread-out-slots: false'
+ Configuration conf3 = new Configuration();
+ conf3.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.NONE);
+ assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf3))
+ .isEqualTo(TaskManagerLoadBalanceMode.NONE);
+
+ // Check for setting manually 'taskmanager.load-balance.mode: NONE' and
+ // 'cluster.evenly-spread-out-slots: true'
+ Configuration conf4 = new Configuration();
+ conf4.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.NONE);
+ conf4.set(EVENLY_SPREAD_OUT_SLOTS_STRATEGY, true);
+ assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf4))
+ .isEqualTo(TaskManagerLoadBalanceMode.NONE);
+
+ // Check for setting manually 'taskmanager.load-balance.mode: SLOTS' and
+ // 'cluster.evenly-spread-out-slots: false'
+ Configuration conf5 = new Configuration();
+ conf5.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.SLOTS);
+ assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf5))
+ .isEqualTo(TaskManagerLoadBalanceMode.SLOTS);
+
+ // Check for setting manually 'taskmanager.load-balance.mode: SLOTS' and
+ // 'cluster.evenly-spread-out-slots: true'
+ Configuration conf6 = new Configuration();
+ conf6.set(TASK_MANAGER_LOAD_BALANCE_MODE, TaskManagerLoadBalanceMode.SLOTS);
+ conf6.set(EVENLY_SPREAD_OUT_SLOTS_STRATEGY, true);
+ assertThat(TaskManagerLoadBalanceMode.loadFromConfiguration(conf6))
+ .isEqualTo(TaskManagerLoadBalanceMode.SLOTS);
+ }
+}
diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh
index f09c68d493fb9..dcd48a256f72a 100755
--- a/flink-dist/src/main/flink-bin/bin/config.sh
+++ b/flink-dist/src/main/flink-bin/bin/config.sh
@@ -334,6 +334,12 @@ if [ -z "${FLINK_ENV_JAVA_OPTS}" ]; then
# Remove leading and ending double quotes (if present) of value
FLINK_ENV_JAVA_OPTS="-XX:+IgnoreUnrecognizedVMOptions $( echo "${FLINK_ENV_JAVA_OPTS}" | sed -e 's/^"//' -e 's/"$//' )"
+
+ JAVA_SPEC_VERSION=`${JAVA_HOME}/bin/java -XshowSettings:properties 2>&1 | grep "java.specification.version" | cut -d "=" -f 2 | tr -d '[:space:]' | rev | cut -d "." -f 1 | rev`
+ if [[ $(( $JAVA_SPEC_VERSION > 17 )) == 1 ]]; then
+ # set security manager property to allow calls to System.setSecurityManager() at runtime
+ FLINK_ENV_JAVA_OPTS="$FLINK_ENV_JAVA_OPTS -Djava.security.manager=allow"
+ fi
fi
if [ -z "${FLINK_ENV_JAVA_OPTS_JM}" ]; then
diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE
index 8eb3dbc5dc798..a5358c47c3048 100644
--- a/flink-dist/src/main/resources/META-INF/NOTICE
+++ b/flink-dist/src/main/resources/META-INF/NOTICE
@@ -12,7 +12,7 @@ This project bundles the following dependencies under the Apache Software Licens
- commons-cli:commons-cli:1.5.0
- commons-collections:commons-collections:3.2.2
- commons-io:commons-io:2.11.0
-- org.apache.commons:commons-compress:1.21
+- org.apache.commons:commons-compress:1.24.0
- org.apache.commons:commons-lang3:3.12.0
- org.apache.commons:commons-math3:3.6.1
- org.apache.commons:commons-text:1.10.0
diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh
index 43c387c1fd8cd..e38f20359babf 100644
--- a/flink-end-to-end-tests/test-scripts/common_docker.sh
+++ b/flink-end-to-end-tests/test-scripts/common_docker.sh
@@ -54,6 +54,9 @@ function build_image() {
if [[ ${PROFILE} == *"jdk11"* ]]; then
java_version=11
fi
+ if [[ ${PROFILE} == *"jdk17"* ]]; then
+ java_version=17
+ fi
cd flink-docker
./add-custom.sh -u ${file_server_address}:9999/flink.tgz -n ${image_name} -j ${java_version}
diff --git a/flink-end-to-end-tests/test-scripts/common_ha.sh b/flink-end-to-end-tests/test-scripts/common_ha.sh
index 23f89eea0bb42..a18bd15da46f8 100644
--- a/flink-end-to-end-tests/test-scripts/common_ha.sh
+++ b/flink-end-to-end-tests/test-scripts/common_ha.sh
@@ -49,7 +49,7 @@ function verify_num_occurences_in_logs() {
local text="$2"
local expected_no="$3"
- local actual_no=$(grep -r --include "*${log_pattern}*.log*" -e "${text}" "$FLINK_LOG_DIR/" | cut -d ":" -f 1 | sed "s/\.[0-9]\{1,\}$//g" | uniq | wc -l)
+ local actual_no=$(grep -r --include "*${log_pattern}*.log*" -e "${text}" "$FLINK_LOG_DIR/" | cut -d ":" -f 1 | sed "s/\.[0-9]\{1,\}$//g" | sort -u | wc -l)
[[ "${expected_no}" -eq "${actual_no}" ]]
}
diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
index 59b4ea70f3472..c12fa071fe42d 100755
--- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
+++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
@@ -21,10 +21,12 @@ source "$(dirname "$0")"/common.sh
source "$(dirname "$0")"/common_docker.sh
CONTAINER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts
-MINIKUBE_START_RETRIES=3
-MINIKUBE_START_BACKOFF=5
+RETRY_COUNT=3
+RETRY_BACKOFF_TIME=5
RESULT_HASH="e682ec6622b5e83f2eb614617d5ab2cf"
MINIKUBE_VERSION="v1.28.0"
+CRICTL_VERSION="v1.24.2"
+CRI_DOCKERD_VERSION="0.2.3"
NON_LINUX_ENV_NOTE="****** Please start/stop minikube manually in non-linux environment. ******"
@@ -39,8 +41,9 @@ function setup_kubernetes_for_linux {
if ! [ -x "$(command -v kubectl)" ]; then
echo "Installing kubectl ..."
local version=$(curl -s https://storage.googleapis.com/kubernetes-release/release/stable.txt)
- curl -Lo kubectl https://storage.googleapis.com/kubernetes-release/release/$version/bin/linux/$arch/kubectl && \
- chmod +x kubectl && sudo mv kubectl /usr/local/bin/
+ download_kubectl_url="https://storage.googleapis.com/kubernetes-release/release/$version/bin/linux/$arch/kubectl"
+ retry_download "${download_kubectl_url}"
+ chmod +x kubectl && sudo mv kubectl /usr/local/bin/
fi
# Download minikube when it is not installed beforehand.
if [ -x "$(command -v minikube)" ] && [[ "$(minikube version | grep -c $MINIKUBE_VERSION)" == "0" ]]; then
@@ -50,32 +53,32 @@ function setup_kubernetes_for_linux {
if ! [ -x "$(command -v minikube)" ]; then
echo "Installing minikube $MINIKUBE_VERSION ..."
- curl -Lo minikube https://storage.googleapis.com/minikube/releases/$MINIKUBE_VERSION/minikube-linux-$arch && \
- chmod +x minikube && sudo mv minikube /usr/bin/minikube
+ download_minikube_url="https://storage.googleapis.com/minikube/releases/$MINIKUBE_VERSION/minikube-linux-$arch"
+ retry_download "${download_minikube_url}"
+ chmod +x "minikube-linux-$arch" && sudo mv "minikube-linux-$arch" /usr/bin/minikube
fi
# conntrack is required for minikube 1.9 and later
sudo apt-get install conntrack
# crictl is required for cri-dockerd
- local crictl_version crictl_archive
- crictl_version="v1.24.2"
- crictl_archive="crictl-$crictl_version-linux-${arch}.tar.gz"
- wget -nv "https://github.com/kubernetes-sigs/cri-tools/releases/download/${crictl_version}/${crictl_archive}"
+ local crictl_archive
+ crictl_archive="crictl-$CRICTL_VERSION-linux-${arch}.tar.gz"
+ download_crictl_url="https://github.com/kubernetes-sigs/cri-tools/releases/download/${CRICTL_VERSION}/${crictl_archive}"
+ retry_download "${download_crictl_url}"
sudo tar zxvf ${crictl_archive} -C /usr/local/bin
rm -f ${crictl_archive}
# cri-dockerd is required to use Kubernetes 1.24+ and the none driver
- local cri_dockerd_version cri_dockerd_archive cri_dockerd_binary
- cri_dockerd_version="0.2.3"
- cri_dockerd_archive="cri-dockerd-${cri_dockerd_version}.${arch}.tgz"
+ local cri_dockerd_archive cri_dockerd_binary
+ cri_dockerd_archive="cri-dockerd-${CRI_DOCKERD_VERSION}.${arch}.tgz"
cri_dockerd_binary="cri-dockerd"
- wget -nv "https://github.com/Mirantis/cri-dockerd/releases/download/v${cri_dockerd_version}/${cri_dockerd_archive}"
+ retry_download "https://github.com/Mirantis/cri-dockerd/releases/download/v${CRI_DOCKERD_VERSION}/${cri_dockerd_archive}"
tar xzvf $cri_dockerd_archive "cri-dockerd/${cri_dockerd_binary}" --strip-components=1
sudo install -o root -g root -m 0755 "${cri_dockerd_binary}" "/usr/local/bin/${cri_dockerd_binary}"
rm ${cri_dockerd_binary}
- wget -nv https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${cri_dockerd_version}/packaging/systemd/cri-docker.service
- wget -nv https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${cri_dockerd_version}/packaging/systemd/cri-docker.socket
+ retry_download "https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${CRI_DOCKERD_VERSION}/packaging/systemd/cri-docker.service"
+ retry_download "https://raw.githubusercontent.com/Mirantis/cri-dockerd/v${CRI_DOCKERD_VERSION}/packaging/systemd/cri-docker.socket"
sudo mv cri-docker.socket cri-docker.service /etc/systemd/system/
sudo sed -i -e "s,/usr/bin/${cri_dockerd_binary},/usr/local/bin/${cri_dockerd_binary}," /etc/systemd/system/cri-docker.service
@@ -87,6 +90,21 @@ function setup_kubernetes_for_linux {
sudo sysctl fs.protected_regular=0
}
+function retry_download {
+ if [[ "$#" != 1 ]]; then
+ echo "Fatal error: No parameter or too many parameters passed: $@"
+ exit 1;
+ fi
+
+ local download_url download_command
+ download_url="$1"
+ download_command="wget -nv ${download_url}"
+ if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} "${download_command}"; then
+ echo "ERROR: Download failed repeatedly after ${RETRY_COUNT} tries. Aborting..."
+ exit 1
+ fi
+}
+
function check_kubernetes_status {
minikube status
return $?
@@ -138,7 +156,7 @@ function start_kubernetes {
echo "The mounting process is running with pid $minikube_mount_pid"
else
setup_kubernetes_for_linux
- if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} start_kubernetes_if_not_running; then
+ if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} start_kubernetes_if_not_running; then
echo "Could not start minikube. Aborting..."
exit 1
fi
@@ -153,7 +171,7 @@ function stop_kubernetes {
else
echo "Stopping minikube ..."
stop_command="minikube stop"
- if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} "${stop_command}"; then
+ if ! retry_times ${RETRY_COUNT} ${RETRY_BACKOFF_TIME} "${stop_command}"; then
echo "Could not stop minikube. Aborting..."
exit 1
fi
diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE
index 0236725e0a47f..c5afbb58e3690 100644
--- a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE
+++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE
@@ -18,7 +18,7 @@ This project bundles the following dependencies under the Apache Software Licens
- commons-collections:commons-collections:3.2.2
- commons-io:commons-io:2.11.0
- commons-logging:commons-logging:1.1.3
-- org.apache.commons:commons-compress:1.21
+- org.apache.commons:commons-compress:1.24.0
- org.apache.commons:commons-configuration2:2.1.1
- org.apache.commons:commons-lang3:3.12.0
- org.apache.commons:commons-text:1.10.0
diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE
index c16ab1adc986d..69137435b28a7 100644
--- a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE
+++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE
@@ -24,7 +24,7 @@ This project bundles the following dependencies under the Apache Software Licens
- commons-io:commons-io:2.11.0
- commons-logging:commons-logging:1.1.3
- joda-time:joda-time:2.5
-- org.apache.commons:commons-compress:1.21
+- org.apache.commons:commons-compress:1.24.0
- org.apache.commons:commons-configuration2:2.1.1
- org.apache.commons:commons-lang3:3.12.0
- org.apache.commons:commons-text:1.10.0
diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE
index 3356afa220538..fc834f1e4d467 100644
--- a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE
+++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE
@@ -36,7 +36,7 @@ This project bundles the following dependencies under the Apache Software Licens
- io.airlift:units:1.3
- joda-time:joda-time:2.5
- org.alluxio:alluxio-shaded-client:2.7.3
-- org.apache.commons:commons-compress:1.21
+- org.apache.commons:commons-compress:1.24.0
- org.apache.commons:commons-configuration2:2.1.1
- org.apache.commons:commons-lang3:3.12.0
- org.apache.commons:commons-text:1.10.0
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java
index c11c4bfb6b6f4..424e44817ee67 100644
--- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/RegistryAvroDeserializationSchemaTest.java
@@ -85,7 +85,7 @@ void testSpecificRecordReadMoreFieldsThanWereWritten() throws IOException {
+ " \"fields\": [\n"
+ " {\"name\": \"name\", \"type\": \"string\"}"
+ " ]\n"
- + "}]");
+ + "}");
RegistryAvroDeserializationSchema deserializer =
new RegistryAvroDeserializationSchema<>(
SimpleRecord.class,
diff --git a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE
index dd601b9119f9e..2231d190de11d 100644
--- a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE
+++ b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE
@@ -13,8 +13,8 @@ This project bundles the following dependencies under the Apache Software Licens
- io.confluent:common-config:7.2.2
- io.confluent:common-utils:7.2.2
- io.confluent:kafka-schema-registry-client:7.2.2
-- org.apache.avro:avro:1.11.1
-- org.apache.commons:commons-compress:1.21
+- org.apache.avro:avro:1.11.3
+- org.apache.commons:commons-compress:1.24.0
- org.apache.kafka:kafka-clients:7.2.2-ccs
- org.glassfish.jersey.core:jersey-common:2.30
- org.xerial.snappy:snappy-java:1.1.10.4
diff --git a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE
index 21f85619d82fa..c0ca160784f8d 100644
--- a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE
+++ b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE
@@ -6,8 +6,8 @@ The Apache Software Foundation (http://www.apache.org/).
This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
-- org.apache.avro:avro:1.11.1
+- org.apache.avro:avro:1.11.3
- com.fasterxml.jackson.core:jackson-core:2.14.3
- com.fasterxml.jackson.core:jackson-databind:2.14.3
- com.fasterxml.jackson.core:jackson-annotations:2.14.3
-- org.apache.commons:commons-compress:1.21
+- org.apache.commons:commons-compress:1.24.0
diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java
index ed219e69db592..d4ffbce264e7b 100644
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/MultiStateKeyIterator.java
@@ -31,6 +31,8 @@
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.stream.Stream;
/**
* An iterator for reading all keys in a state backend across multiple partitioned states.
@@ -46,7 +48,10 @@ public final class MultiStateKeyIterator implements CloseableIterator {
private final KeyedStateBackend backend;
- private final Iterator internal;
+ /** Avoids using Stream#flatMap due to a known flaw, see FLINK-26585 for more details. */
+ private final Iterator extends StateDescriptor, ?>> outerIter;
+
+ private Iterator innerIter;
private final CloseableRegistry registry;
@@ -54,39 +59,43 @@ public final class MultiStateKeyIterator implements CloseableIterator {
public MultiStateKeyIterator(
List extends StateDescriptor, ?>> descriptors, KeyedStateBackend backend) {
+
this.descriptors = Preconditions.checkNotNull(descriptors);
this.backend = Preconditions.checkNotNull(backend);
+ outerIter = this.descriptors.iterator();
+ innerIter = null;
+
this.registry = new CloseableRegistry();
- this.internal =
- descriptors.stream()
- .map(
- descriptor ->
- backend.getKeys(
- descriptor.getName(), VoidNamespace.INSTANCE))
- .peek(
- stream -> {
- try {
- registry.registerCloseable(stream::close);
- } catch (IOException e) {
- throw new RuntimeException(
- "Failed to read keys from configured StateBackend",
- e);
- }
- })
- .flatMap(stream -> stream)
- .iterator();
}
@Override
public boolean hasNext() {
- return internal.hasNext();
+ while (innerIter == null || !innerIter.hasNext()) {
+ if (!outerIter.hasNext()) {
+ return false;
+ }
+
+ StateDescriptor, ?> descriptor = outerIter.next();
+ Stream stream = backend.getKeys(descriptor.getName(), VoidNamespace.INSTANCE);
+ innerIter = stream.iterator();
+ try {
+ registry.registerCloseable(stream::close);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to read keys from configured StateBackend", e);
+ }
+ }
+ return true;
}
@Override
public K next() {
- currentKey = internal.next();
- return currentKey;
+ if (!this.hasNext()) {
+ throw new NoSuchElementException();
+ } else {
+ currentKey = this.innerIter.next();
+ return currentKey;
+ }
}
/** Removes the current key from ALL known states in the state backend. */
diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java
index 33fcf2cea816d..e891c5583ed69 100644
--- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java
+++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/MultiStateKeyIteratorTest.java
@@ -18,28 +18,57 @@
package org.apache.flink.state.api.input;
+import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.State;
import org.apache.flink.api.common.state.StateDescriptor;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.Keyed;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.PriorityComparable;
+import org.apache.flink.runtime.state.SavepointResources;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.StateSerializerProvider;
+import org.apache.flink.runtime.state.StateSnapshotTransformer;
import org.apache.flink.runtime.state.VoidNamespace;
import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+import org.apache.flink.runtime.state.heap.InternalKeyContext;
+import org.apache.flink.runtime.state.heap.InternalKeyContextImpl;
+import org.apache.flink.runtime.state.metrics.LatencyTrackingStateConfig;
import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
+import org.apache.flink.runtime.state.ttl.mock.MockRestoreOperation;
import org.apache.flink.runtime.state.ttl.mock.MockStateBackend;
import org.junit.Assert;
import org.junit.Test;
+import javax.annotation.Nonnull;
+
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
+import java.util.concurrent.RunnableFuture;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
/** Test for the multi-state key iterator. */
public class MultiStateKeyIteratorTest {
@@ -68,6 +97,37 @@ private static AbstractKeyedStateBackend createKeyedStateBackend() {
new CloseableRegistry());
}
+ private static CountingKeysKeyedStateBackend createCountingKeysKeyedStateBackend(
+ Integer numKeys) {
+ Environment env = new DummyEnvironment();
+ TypeSerializer keySerializer = IntSerializer.INSTANCE;
+ int numberOfKeyGroups = 129;
+ KeyGroupRange keyGroupRange = KeyGroupRange.of(0, 128);
+ TaskKvStateRegistry kvStateRegistry = null;
+ TtlTimeProvider ttlTimeProvider = TtlTimeProvider.DEFAULT;
+ @Nonnull Collection stateHandles = Collections.emptyList();
+ CloseableRegistry cancelStreamRegistry = new CloseableRegistry();
+
+ Map>> stateValues = new HashMap<>();
+ MockRestoreOperation restoreOperation =
+ new MockRestoreOperation<>(stateHandles, stateValues);
+ restoreOperation.restore();
+
+ StateSerializerProvider keySerializerProvider =
+ StateSerializerProvider.fromNewRegisteredSerializer(keySerializer);
+
+ return new CountingKeysKeyedStateBackend(
+ numKeys,
+ kvStateRegistry,
+ keySerializerProvider.currentSchemaSerializer(),
+ env.getUserCodeClassLoader().asClassLoader(),
+ env.getExecutionConfig(),
+ ttlTimeProvider,
+ LatencyTrackingStateConfig.disabled(),
+ cancelStreamRegistry,
+ new InternalKeyContextImpl<>(keyGroupRange, numberOfKeyGroups));
+ }
+
private static void setKey(
AbstractKeyedStateBackend backend,
ValueStateDescriptor descriptor,
@@ -79,6 +139,17 @@ private static void setKey(
.update(0);
}
+ private static void clearKey(
+ AbstractKeyedStateBackend backend,
+ ValueStateDescriptor descriptor,
+ int key)
+ throws Exception {
+ backend.setCurrentKey(key);
+ backend.getPartitionedState(
+ VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, descriptor)
+ .clear();
+ }
+
@Test
public void testIteratorPullsKeyFromAllDescriptors() throws Exception {
AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend();
@@ -99,6 +170,36 @@ public void testIteratorPullsKeyFromAllDescriptors() throws Exception {
Assert.assertEquals("Unexpected keys found", Arrays.asList(1, 2), keys);
}
+ @Test
+ public void testIteratorSkipsEmptyDescriptors() throws Exception {
+ AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend();
+
+ List> threeDescriptors = new ArrayList<>(3);
+ threeDescriptors.add(new ValueStateDescriptor<>("state-1", Types.INT));
+ threeDescriptors.add(new ValueStateDescriptor<>("state-2", Types.INT));
+ threeDescriptors.add(new ValueStateDescriptor<>("state-3", Types.INT));
+
+ setKey(keyedStateBackend, threeDescriptors.get(0), 1);
+
+ // initializes descriptor 1, but empties it immediately after
+ setKey(keyedStateBackend, threeDescriptors.get(1), 12);
+ clearKey(keyedStateBackend, threeDescriptors.get(1), 12);
+
+ setKey(keyedStateBackend, threeDescriptors.get(2), 2);
+
+ MultiStateKeyIterator iterator =
+ new MultiStateKeyIterator<>(threeDescriptors, keyedStateBackend);
+
+ List keys = new ArrayList<>();
+
+ while (iterator.hasNext()) {
+ keys.add(iterator.next());
+ }
+
+ Assert.assertEquals("Unexpected number of keys", 2, keys.size());
+ Assert.assertEquals("Unexpected keys found", Arrays.asList(1, 2), keys);
+ }
+
@Test
public void testIteratorRemovesFromAllDescriptors() throws Exception {
AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend();
@@ -125,4 +226,117 @@ public void testIteratorRemovesFromAllDescriptors() throws Exception {
.count());
}
}
+
+ /** Test for lazy enumeration of inner iterators. */
+ @Test
+ public void testIteratorPullsSingleKeyFromAllDescriptors() throws AssertionError {
+ CountingKeysKeyedStateBackend keyedStateBackend =
+ createCountingKeysKeyedStateBackend(100_000_000);
+ MultiStateKeyIterator testedIterator =
+ new MultiStateKeyIterator<>(descriptors, keyedStateBackend);
+
+ testedIterator.hasNext();
+
+ Assert.assertEquals(
+ "Unexpected number of keys enumerated",
+ 1,
+ keyedStateBackend.numberOfKeysEnumerated);
+ }
+
+ /**
+ * Mockup {@link AbstractKeyedStateBackend} that counts how many keys are enumerated.
+ *
+ * Generates a configured number of integer keys, only method actually implemented is {@link
+ * CountingKeysKeyedStateBackend#getKeys(java.lang.String, java.lang.Object)}
+ */
+ static class CountingKeysKeyedStateBackend extends AbstractKeyedStateBackend {
+ int numberOfKeysGenerated;
+ public long numberOfKeysEnumerated;
+
+ public CountingKeysKeyedStateBackend(
+ int numberOfKeysGenerated,
+ TaskKvStateRegistry kvStateRegistry,
+ TypeSerializer keySerializer,
+ ClassLoader userCodeClassLoader,
+ ExecutionConfig executionConfig,
+ TtlTimeProvider ttlTimeProvider,
+ LatencyTrackingStateConfig latencyTrackingStateConfig,
+ CloseableRegistry cancelStreamRegistry,
+ InternalKeyContext keyContext) {
+ super(
+ kvStateRegistry,
+ keySerializer,
+ userCodeClassLoader,
+ executionConfig,
+ ttlTimeProvider,
+ latencyTrackingStateConfig,
+ cancelStreamRegistry,
+ keyContext);
+ this.numberOfKeysGenerated = numberOfKeysGenerated;
+ numberOfKeysEnumerated = 0;
+ }
+
+ @Override
+ public Stream getKeys(String state, N namespace) {
+ return IntStream.range(0, this.numberOfKeysGenerated)
+ .boxed()
+ .peek(i -> numberOfKeysEnumerated++);
+ }
+
+ @Override
+ public int numKeyValueStateEntries() {
+ return numberOfKeysGenerated;
+ }
+
+ @Nonnull
+ @Override
+ public & Keyed>>
+ KeyGroupedInternalPriorityQueue create(
+ @Nonnull String stateName,
+ @Nonnull TypeSerializer byteOrderedElementSerializer) {
+ throw new UnsupportedOperationException(
+ "Operations other than getKeys() are not supported on this testing StateBackend.");
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {}
+
+ @Nonnull
+ @Override
+ public RunnableFuture> snapshot(
+ long checkpointId,
+ long timestamp,
+ @Nonnull CheckpointStreamFactory streamFactory,
+ @Nonnull CheckpointOptions checkpointOptions)
+ throws UnsupportedOperationException {
+ throw new UnsupportedOperationException(
+ "Operations other than getKeys() are not supported on this testing StateBackend.");
+ }
+
+ @Nonnull
+ @Override
+ public IS createOrUpdateInternalState(
+ @Nonnull TypeSerializer namespaceSerializer,
+ @Nonnull StateDescriptor stateDesc,
+ @Nonnull
+ StateSnapshotTransformer.StateSnapshotTransformFactory
+ snapshotTransformFactory)
+ throws UnsupportedOperationException {
+ throw new UnsupportedOperationException(
+ "Operations other than getKeys() are not supported on this testing StateBackend.");
+ }
+
+ @Override
+ public Stream> getKeysAndNamespaces(String state) {
+ throw new UnsupportedOperationException(
+ "Operations other than getKeys() are not supported on this testing StateBackend.");
+ }
+
+ @Nonnull
+ @Override
+ public SavepointResources savepoint() throws UnsupportedOperationException {
+ throw new UnsupportedOperationException(
+ "Operations other than getKeys() are not supported on this testing StateBackend.");
+ }
+ }
}
diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java
index ddb7e3c4fd30f..61a087cfde907 100644
--- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java
+++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java
@@ -71,6 +71,10 @@ public MeterView(Counter counter, int timeSpanInSeconds) {
this.values = new long[this.timeSpanInSeconds / UPDATE_INTERVAL_SECONDS + 1];
}
+ public MeterView(Gauge extends Number> numberGauge) {
+ this(new GaugeWrapper(numberGauge));
+ }
+
@Override
public void markEvent() {
this.counter.inc();
@@ -98,4 +102,39 @@ public void update() {
currentRate =
((double) (values[time] - values[(time + 1) % values.length]) / timeSpanInSeconds);
}
+
+ /** Simple wrapper to expose number gauges as timers. */
+ static class GaugeWrapper implements Counter {
+
+ final Gauge extends Number> numberGauge;
+
+ GaugeWrapper(Gauge extends Number> numberGauge) {
+ this.numberGauge = numberGauge;
+ }
+
+ @Override
+ public void inc() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void inc(long n) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void dec() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void dec(long n) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long getCount() {
+ return numberGauge.getValue().longValue();
+ }
+ }
}
diff --git a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py
index 4dbb243fcf984..39c134a8ed336 100644
--- a/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py
+++ b/flink-python/pyflink/examples/datastream/connectors/kafka_csv_format.py
@@ -21,8 +21,7 @@
from pyflink.common import Types
from pyflink.datastream import StreamExecutionEnvironment
from pyflink.datastream.connectors.kafka import FlinkKafkaProducer, FlinkKafkaConsumer
-from pyflink.datastream.formats.csv import CsvRowSerializationSchema
-from pyflink.datastream.formats.json import JsonRowDeserializationSchema
+from pyflink.datastream.formats.csv import CsvRowSerializationSchema, CsvRowDeserializationSchema
# Make sure that the Kafka cluster is started and the topic 'test_csv_topic' is
@@ -46,9 +45,8 @@ def write_to_kafka(env):
def read_from_kafka(env):
- deserialization_schema = JsonRowDeserializationSchema.Builder() \
- .type_info(Types.ROW([Types.INT(), Types.STRING()])) \
- .build()
+ type_info = Types.ROW([Types.INT(), Types.STRING()])
+ deserialization_schema = CsvRowDeserializationSchema.Builder(type_info).build()
kafka_consumer = FlinkKafkaConsumer(
topics='test_csv_topic',
diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
index 62f8166917c1f..d6ba3eb2f5cd1 100644
--- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
+++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
@@ -1971,6 +1971,9 @@
"location" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"timestamp" : {
"type" : "integer"
},
@@ -2014,6 +2017,9 @@
"location" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"taskManagerId" : {
"type" : "string"
},
@@ -2044,6 +2050,9 @@
"location" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"taskManagerId" : {
"type" : "string"
}
@@ -2598,6 +2607,9 @@
"host" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"start-time" : {
"type" : "integer"
},
@@ -2941,6 +2953,9 @@
"host" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"user-accumulators" : {
"type" : "array",
"items" : {
@@ -3033,6 +3048,9 @@
"host" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"start-time" : {
"type" : "integer"
},
@@ -3142,6 +3160,9 @@
"host" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"start-time" : {
"type" : "integer"
},
@@ -3338,6 +3359,9 @@
"host" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"duration" : {
"type" : "integer"
},
@@ -3393,6 +3417,9 @@
"host" : {
"type" : "string"
},
+ "endpoint" : {
+ "type" : "string"
+ },
"status" : {
"type" : "string",
"enum" : [ "CREATED", "SCHEDULED", "DEPLOYING", "RUNNING", "FINISHED", "CANCELING", "CANCELED", "FAILED", "RECONCILING", "INITIALIZING" ]
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts
index 8888fa314fe29..bb759e2694041 100644
--- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/metrics/job-manager-metrics.component.ts
@@ -112,7 +112,11 @@ export class JobManagerMetricsComponent implements OnInit, OnDestroy {
this.listOfGCMetric = Array.from(
new Set(
this.listOfGCName.map(item =>
- item.replace('Status.JVM.GarbageCollector.', '').replace('.Count', '').replace('.Time', '')
+ item
+ .replace('Status.JVM.GarbageCollector.', '')
+ .replace('.Count', '')
+ .replace('.TimeMsPerSecond', '')
+ .replace('.Time', '')
)
)
).map(name => {
diff --git a/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts
index cfa74457a5455..8e14bcf82c1d4 100644
--- a/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts
+++ b/flink-runtime-web/web-dashboard/src/app/pages/submit/submit.component.ts
@@ -78,7 +78,7 @@ export class SubmitComponent implements OnInit, OnDestroy {
public validateForm: UntypedFormGroup;
public planVisible = false;
- @ViewChild(DagreComponent, { static: true }) private readonly dagreComponent: DagreComponent;
+ @ViewChild(DagreComponent) private readonly dagreComponent: DagreComponent;
private readonly destroy$ = new Subject();
@@ -177,6 +177,7 @@ export class SubmitComponent implements OnInit, OnDestroy {
)
.subscribe(data => {
this.planVisible = true;
+ this.cdr.detectChanges();
this.dagreComponent.flush(data.nodes, data.links, true);
});
}
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 359a0e181cb7e..c6bd2e909644e 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -34,7 +34,14 @@ under the License.
jar
-
+
+ ${surefire.module.config.jdk21} --add-opens=java.base/java.util=ALL-UNNAMED
+ -Djava.security.manager=allow
+
+
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java
index b8972875e2f4f..5fa6b08ff0f30 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoint.java
@@ -17,6 +17,11 @@
package org.apache.flink.runtime.checkpoint;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
/** A checkpoint, pending or completed. */
public interface Checkpoint {
DiscardObject NOOP_DISCARD_OBJECT = () -> {};
@@ -33,5 +38,9 @@ public interface Checkpoint {
/** Extra interface for discarding the checkpoint. */
interface DiscardObject {
void discard() throws Exception;
+
+ default CompletableFuture discardAsync(Executor ioExecutor) {
+ return FutureUtils.runAsync(this::discard, ioExecutor);
+ }
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
index b67dfefac74f5..6d1bacaa83a66 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointsCleaner.java
@@ -18,8 +18,9 @@
package org.apache.flink.runtime.checkpoint;
+import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.util.AutoCloseableAsync;
-import org.apache.flink.util.function.RunnableWithException;
+import org.apache.flink.util.concurrent.FutureUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,6 +48,7 @@ public class CheckpointsCleaner implements Serializable, AutoCloseableAsync {
private static final Logger LOG = LoggerFactory.getLogger(CheckpointsCleaner.class);
private static final long serialVersionUID = 2545865801947537790L;
+ private final boolean parallelMode;
private final Object lock = new Object();
@GuardedBy("lock")
@@ -60,6 +62,14 @@ public class CheckpointsCleaner implements Serializable, AutoCloseableAsync {
@GuardedBy("lock")
private final List subsumedCheckpoints = new ArrayList<>();
+ public CheckpointsCleaner() {
+ this.parallelMode = CheckpointingOptions.CLEANER_PARALLEL_MODE.defaultValue();
+ }
+
+ public CheckpointsCleaner(boolean parallelMode) {
+ this.parallelMode = parallelMode;
+ }
+
int getNumberOfCheckpointsToClean() {
synchronized (lock) {
return numberOfCheckpointsToClean;
@@ -71,10 +81,35 @@ public void cleanCheckpoint(
boolean shouldDiscard,
Runnable postCleanAction,
Executor executor) {
- Checkpoint.DiscardObject discardObject =
- shouldDiscard ? checkpoint.markAsDiscarded() : Checkpoint.NOOP_DISCARD_OBJECT;
+ LOG.debug(
+ "Clean checkpoint {} parallel-mode={} shouldDiscard={}",
+ checkpoint.getCheckpointID(),
+ parallelMode,
+ shouldDiscard);
+ if (shouldDiscard) {
+ incrementNumberOfCheckpointsToClean();
+
+ Checkpoint.DiscardObject discardObject = checkpoint.markAsDiscarded();
+ CompletableFuture discardFuture =
+ parallelMode
+ ? discardObject.discardAsync(executor)
+ : FutureUtils.runAsync(discardObject::discard, executor);
+ discardFuture.handle(
+ (Object outerIgnored, Throwable outerThrowable) -> {
+ if (outerThrowable != null) {
+ LOG.warn(
+ "Could not properly discard completed checkpoint {}.",
+ checkpoint.getCheckpointID(),
+ outerThrowable);
+ }
- cleanup(checkpoint, discardObject::discard, postCleanAction, executor);
+ decrementNumberOfCheckpointsToClean();
+ postCleanAction.run();
+ return null;
+ });
+ } else {
+ executor.execute(postCleanAction);
+ }
}
/**
@@ -123,30 +158,7 @@ public void cleanSubsumedCheckpoints(
public void cleanCheckpointOnFailedStoring(
CompletedCheckpoint completedCheckpoint, Executor executor) {
- Checkpoint.DiscardObject discardObject = completedCheckpoint.markAsDiscarded();
- cleanup(completedCheckpoint, discardObject::discard, () -> {}, executor);
- }
-
- private void cleanup(
- Checkpoint checkpoint,
- RunnableWithException cleanupAction,
- Runnable postCleanupAction,
- Executor executor) {
- incrementNumberOfCheckpointsToClean();
- executor.execute(
- () -> {
- try {
- cleanupAction.run();
- } catch (Exception e) {
- LOG.warn(
- "Could not properly discard completed checkpoint {}.",
- checkpoint.getCheckpointID(),
- e);
- } finally {
- decrementNumberOfCheckpointsToClean();
- postCleanupAction.run();
- }
- });
+ cleanCheckpoint(completedCheckpoint, true, () -> {}, executor);
}
private void incrementNumberOfCheckpointsToClean() {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
index daee8986f10f6..3925f3ca2fe3a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
@@ -25,9 +25,12 @@
import org.apache.flink.runtime.jobgraph.RestoreMode;
import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateObject;
import org.apache.flink.runtime.state.StateUtil;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+import org.apache.flink.util.concurrent.FutureUtils.ConjunctFuture;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -43,6 +46,9 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -327,7 +333,6 @@ public String toString() {
/** Implementation of {@link org.apache.flink.runtime.checkpoint.Checkpoint.DiscardObject}. */
@NotThreadSafe
public class CompletedCheckpointDiscardObject implements DiscardObject {
-
@Override
public void discard() throws Exception {
LOG.trace("Executing discard procedure for {}.", this);
@@ -371,5 +376,34 @@ public void discard() throws Exception {
private boolean isMarkedAsDiscarded() {
return completedCheckpointStats == null || completedCheckpointStats.isDiscarded();
}
+
+ @Override
+ public CompletableFuture discardAsync(Executor ioExecutor) {
+ checkState(
+ isMarkedAsDiscarded(),
+ "Checkpoint should be marked as discarded before discard.");
+
+ List discardables =
+ operatorStates.values().stream()
+ .flatMap(op -> op.getDiscardables().stream())
+ .collect(Collectors.toList());
+ discardables.add(metadataHandle);
+
+ ConjunctFuture discardStates =
+ FutureUtils.completeAll(
+ discardables.stream()
+ .map(
+ item ->
+ FutureUtils.runAsync(
+ item::discardState, ioExecutor))
+ .collect(Collectors.toList()));
+
+ return FutureUtils.runAfterwards(
+ discardStates,
+ () -> {
+ operatorStates.clear();
+ storageLocation.disposeStorageLocation();
+ });
+ }
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
index 462968fe160e9..2b6c4e06197c2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
@@ -21,6 +21,7 @@
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.state.CompositeStateHandle;
import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.runtime.state.StateObject;
import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import org.apache.flink.util.CollectionUtil;
import org.apache.flink.util.Preconditions;
@@ -29,8 +30,10 @@
import java.util.Collection;
import java.util.Collections;
+import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkState;
@@ -165,6 +168,18 @@ public OperatorState copyAndDiscardInFlightData() {
return newState;
}
+ public List getDiscardables() {
+ List toDispose =
+ operatorSubtaskStates.values().stream()
+ .flatMap(op -> op.getDiscardables().stream())
+ .collect(Collectors.toList());
+
+ if (coordinatorState != null) {
+ toDispose.add(coordinatorState);
+ }
+ return toDispose;
+ }
+
@Override
public void discardState() throws Exception {
for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
index 8b0e2846e60d6..4fede9683e5bb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
@@ -194,22 +194,27 @@ public InflightDataRescalingDescriptor getOutputRescalingDescriptor() {
return outputRescalingDescriptor;
}
+ public List getDiscardables() {
+ List toDispose =
+ new ArrayList<>(
+ managedOperatorState.size()
+ + rawOperatorState.size()
+ + managedKeyedState.size()
+ + rawKeyedState.size()
+ + inputChannelState.size()
+ + resultSubpartitionState.size());
+ toDispose.addAll(managedOperatorState);
+ toDispose.addAll(rawOperatorState);
+ toDispose.addAll(managedKeyedState);
+ toDispose.addAll(rawKeyedState);
+ toDispose.addAll(collectUniqueDelegates(inputChannelState, resultSubpartitionState));
+ return toDispose;
+ }
+
@Override
public void discardState() {
try {
- List toDispose =
- new ArrayList<>(
- managedOperatorState.size()
- + rawOperatorState.size()
- + managedKeyedState.size()
- + rawKeyedState.size()
- + inputChannelState.size()
- + resultSubpartitionState.size());
- toDispose.addAll(managedOperatorState);
- toDispose.addAll(rawOperatorState);
- toDispose.addAll(managedKeyedState);
- toDispose.addAll(rawKeyedState);
- toDispose.addAll(collectUniqueDelegates(inputChannelState, resultSubpartitionState));
+ List toDispose = getDiscardables();
StateUtil.bestEffortDiscardAllStateObjects(toDispose);
} catch (Exception e) {
LOG.warn("Error while discarding operator states.", e);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
index 9ec66bb95fe62..2f2cc6f6f6736 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
@@ -29,11 +29,14 @@
import org.apache.flink.runtime.state.CheckpointMetadataOutputStream;
import org.apache.flink.runtime.state.CheckpointStorageLocation;
import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
+import org.apache.flink.runtime.state.StateObject;
import org.apache.flink.runtime.state.StateUtil;
import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import org.apache.flink.util.CollectionUtil;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.FutureUtils;
+import org.apache.flink.util.concurrent.FutureUtils.ConjunctFuture;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -53,6 +56,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledFuture;
+import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -652,5 +656,39 @@ public void discard() {
operatorStates.clear();
}
}
+
+ @Override
+ public CompletableFuture discardAsync(Executor ioExecutor) {
+ synchronized (lock) {
+ if (discarded) {
+ Preconditions.checkState(
+ disposed, "Checkpoint should be disposed before being discarded");
+ } else {
+ discarded = true;
+ }
+ }
+ List discardables =
+ operatorStates.values().stream()
+ .flatMap(op -> op.getDiscardables().stream())
+ .collect(Collectors.toList());
+
+ ConjunctFuture discardStates =
+ FutureUtils.completeAll(
+ discardables.stream()
+ .map(
+ item ->
+ FutureUtils.runAsync(
+ item::discardState, ioExecutor))
+ .collect(Collectors.toList()));
+
+ return FutureUtils.runAfterwards(
+ discardStates,
+ () -> {
+ operatorStates.clear();
+ if (targetLocation != null) {
+ targetLocation.disposeOnFailure();
+ }
+ });
+ }
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
index ef9bcd0440b12..e41bcfe73385b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
@@ -31,10 +31,14 @@
import javax.annotation.Nullable;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.Map;
+import java.util.Optional;
import java.util.function.BiFunction;
import java.util.function.Function;
+import java.util.stream.Collectors;
/**
* This class is a wrapper over multiple alternative {@link OperatorSubtaskState} that are (partial)
@@ -286,14 +290,14 @@ public PrioritizedOperatorSubtaskState build() {
}
return new PrioritizedOperatorSubtaskState(
- resolvePrioritizedAlternatives(
+ computePrioritizedAlternatives(
jobManagerState.getManagedKeyedState(),
managedKeyedAlternatives,
- eqStateApprover(KeyedStateHandle::getKeyGroupRange)),
- resolvePrioritizedAlternatives(
+ KeyedStateHandle::getKeyGroupRange),
+ computePrioritizedAlternatives(
jobManagerState.getRawKeyedState(),
rawKeyedAlternatives,
- eqStateApprover(KeyedStateHandle::getKeyGroupRange)),
+ KeyedStateHandle::getKeyGroupRange),
resolvePrioritizedAlternatives(
jobManagerState.getManagedOperatorState(),
managedOperatorAlternatives,
@@ -313,22 +317,121 @@ public PrioritizedOperatorSubtaskState build() {
restoredCheckpointId);
}
+ /**
+ * This method creates an alternative recovery option by replacing as much job manager state
+ * with higher prioritized (=local) alternatives as possible.
+ *
+ * @param jobManagerState the state that the task got assigned from the job manager (this
+ * state lives in remote storage).
+ * @param alternativesByPriority local alternatives to the job manager state, ordered by
+ * priority.
+ * @param identityExtractor function to extract an identifier from a state object.
+ * @return prioritized state alternatives.
+ * @param the type of the state objects we process.
+ * @param the type of object that represents the id the state object type.
+ */
+
+ List> computePrioritizedAlternatives(
+ StateObjectCollection jobManagerState,
+ List> alternativesByPriority,
+ Function identityExtractor) {
+
+ if (alternativesByPriority != null
+ && !alternativesByPriority.isEmpty()
+ && jobManagerState.hasState()) {
+
+ Optional> mergedAlternative =
+ tryComputeMixedLocalAndRemoteAlternative(
+ jobManagerState, alternativesByPriority, identityExtractor);
+
+ // Return the mix of local/remote state as first and pure remote state as second
+ // alternative (in case that we fail to recover from the local state, e.g. because
+ // of corruption).
+ if (mergedAlternative.isPresent()) {
+ return Arrays.asList(mergedAlternative.get(), jobManagerState);
+ }
+ }
+
+ return Collections.singletonList(jobManagerState);
+ }
+
+ /**
+ * This method creates an alternative recovery option by replacing as much job manager state
+ * with higher prioritized (=local) alternatives as possible. Returns empty Optional if the
+ * JM state is empty or nothing could be replaced.
+ *
+ * @param jobManagerState the state that the task got assigned from the job manager (this
+ * state lives in remote storage).
+ * @param alternativesByPriority local alternatives to the job manager state, ordered by
+ * priority.
+ * @param identityExtractor function to extract an identifier from a state object.
+ * @return A state collection where all JM state handles for which we could find local *
+ * alternatives are replaced by the alternative with the highest priority. Empty
+ * optional if no state could be replaced.
+ * @param the type of the state objects we process.
+ * @param the type of object that represents the id the state object type.
+ */
+ static
+ Optional>
+ tryComputeMixedLocalAndRemoteAlternative(
+ StateObjectCollection jobManagerState,
+ List> alternativesByPriority,
+ Function identityExtractor) {
+
+ List result = Collections.emptyList();
+
+ // Build hash index over ids of the JM state
+ Map indexById =
+ jobManagerState.stream()
+ .collect(Collectors.toMap(identityExtractor, Function.identity()));
+
+ // Move through all alternative in order from high to low priority
+ for (StateObjectCollection alternative : alternativesByPriority) {
+ // Check all the state objects in the alternative if they can replace JM state
+ for (STATE_OBJ_TYPE stateHandle : alternative) {
+ // Remove the current state object's id from the index to check for a match
+ if (indexById.remove(identityExtractor.apply(stateHandle)) != null) {
+ if (result.isEmpty()) {
+ // Lazy init result collection
+ result = new ArrayList<>(jobManagerState.size());
+ }
+ // If the id was still in the index, replace with higher prio alternative
+ result.add(stateHandle);
+
+ // If the index is empty we are already done, all JM state was replaces with
+ // the best alternative.
+ if (indexById.isEmpty()) {
+ return Optional.of(new StateObjectCollection<>(result));
+ }
+ }
+ }
+ }
+
+ // Nothing useful to return
+ if (result.isEmpty()) {
+ return Optional.empty();
+ }
+
+ // Add all remaining JM state objects that we could not replace from the index to the
+ // final result
+ result.addAll(indexById.values());
+ return Optional.of(new StateObjectCollection<>(result));
+ }
+
/**
* This helper method resolves the dependencies between the ground truth of the operator
* state obtained from the job manager and potential alternatives for recovery, e.g. from a
* task-local source.
*/
- protected
- List> resolvePrioritizedAlternatives(
- StateObjectCollection jobManagerState,
- List> alternativesByPriority,
- BiFunction approveFun) {
+ List> resolvePrioritizedAlternatives(
+ StateObjectCollection jobManagerState,
+ List> alternativesByPriority,
+ BiFunction approveFun) {
// Nothing to resolve if there are no alternatives, or the ground truth has already no
- // state, or if we can
- // assume that a rescaling happened because we find more than one handle in the JM state
- // (this is more a sanity
- // check).
+ // state, or if we can assume that a rescaling happened because we find more than one
+ // handle in the JM state
+ // (this is more a sanity check).
if (alternativesByPriority == null
|| alternativesByPriority.isEmpty()
|| !jobManagerState.hasState()
@@ -347,8 +450,7 @@ List> resolvePrioritizedAlternatives(
for (StateObjectCollection alternative : alternativesByPriority) {
// We found an alternative to the JM state if it has state, we have a 1:1
- // relationship, and the
- // approve-function signaled true.
+ // relationship, and the approve-function signaled true.
if (alternative != null
&& alternative.hasState()
&& alternative.size() == 1
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
index db3b5dd22d0e7..bd91e8e1fbb4d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
@@ -341,7 +341,7 @@ static void serializeKeyedStateHandle(KeyedStateHandle stateHandle, DataOutputSt
dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
dos.writeLong(incrementalKeyedStateHandle.getCheckpointedSize());
- serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos);
+ serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaDataStateHandle(), dos);
serializeHandleAndLocalPathList(incrementalKeyedStateHandle.getSharedState(), dos);
serializeHandleAndLocalPathList(incrementalKeyedStateHandle.getPrivateState(), dos);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
index 50ed98ed1dcf7..333a91e0a7320 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputGateDeploymentDescriptor.java
@@ -33,7 +33,7 @@
import org.apache.flink.runtime.jobgraph.DistributionPattern;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
-import org.apache.flink.runtime.taskexecutor.ShuffleDescriptorsCache;
+import org.apache.flink.runtime.util.GroupCache;
import org.apache.flink.util.CompressedSerializedValue;
import org.apache.flink.util.Preconditions;
@@ -167,7 +167,7 @@ public ShuffleDescriptor[] getShuffleDescriptors() {
public void tryLoadAndDeserializeShuffleDescriptors(
@Nullable PermanentBlobService blobService,
JobID jobId,
- ShuffleDescriptorsCache shuffleDescriptorsCache)
+ GroupCache shuffleDescriptorsCache)
throws IOException {
if (inputChannels != null) {
return;
@@ -190,13 +190,14 @@ private void tryLoadAndDeserializeShuffleDescriptorGroup(
@Nullable PermanentBlobService blobService,
JobID jobId,
MaybeOffloaded serializedShuffleDescriptors,
- ShuffleDescriptorsCache shuffleDescriptorsCache)
+ GroupCache shuffleDescriptorsCache)
throws IOException, ClassNotFoundException {
if (serializedShuffleDescriptors instanceof Offloaded) {
PermanentBlobKey blobKey =
((Offloaded) serializedShuffleDescriptors)
.serializedValueKey;
- ShuffleDescriptorGroup shuffleDescriptorGroup = shuffleDescriptorsCache.get(blobKey);
+ ShuffleDescriptorGroup shuffleDescriptorGroup =
+ shuffleDescriptorsCache.get(jobId, blobKey);
if (shuffleDescriptorGroup == null) {
Preconditions.checkNotNull(blobService);
// NOTE: Do not delete the ShuffleDescriptor BLOBs since it may be needed again
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index bd3b770142c01..5684066735f03 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -23,19 +23,22 @@
import org.apache.flink.runtime.blob.PermanentBlobService;
import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory.ShuffleDescriptorGroup;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.JobInformation;
import org.apache.flink.runtime.executiongraph.TaskInformation;
-import org.apache.flink.runtime.taskexecutor.ShuffleDescriptorsCache;
-import org.apache.flink.util.FileUtils;
+import org.apache.flink.runtime.util.GroupCache;
+import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;
import javax.annotation.Nullable;
+import java.io.BufferedInputStream;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
+import java.nio.file.Files;
import java.util.List;
/**
@@ -97,11 +100,23 @@ public Offloaded(PermanentBlobKey serializedValueKey) {
}
}
- /** Serialized job information or null if offloaded. */
- private MaybeOffloaded serializedJobInformation;
+ /** Serialized job information if non-offloaded or PermanentBlobKey if offloaded. */
+ private final MaybeOffloaded serializedJobInformation;
- /** Serialized task information or null if offloaded. */
- private MaybeOffloaded serializedTaskInformation;
+ /** Serialized task information if non-offloaded or PermanentBlobKey if offloaded. */
+ private final MaybeOffloaded serializedTaskInformation;
+
+ /**
+ * The job information, it isn't null when serializedJobInformation is offloaded and after
+ * {@link #loadBigData}.
+ */
+ private transient JobInformation jobInformation;
+
+ /**
+ * The task information, it isn't null when serializedTaskInformation is offloaded and after
+ * {@link #loadBigData}.
+ */
+ private transient TaskInformation taskInformation;
/**
* The ID referencing the job this task belongs to.
@@ -151,39 +166,43 @@ public TaskDeploymentDescriptor(
}
/**
- * Return the sub task's serialized job information.
+ * Return the sub task's job information.
*
- * @return serialized job information (may throw {@link IllegalStateException} if {@link
- * #loadBigData} is not called beforehand).
+ * @return job information (may throw {@link IllegalStateException} if {@link #loadBigData} is
+ * not called beforehand).
* @throws IllegalStateException If job information is offloaded to BLOB store.
*/
- public SerializedValue getSerializedJobInformation() {
+ public JobInformation getJobInformation() throws IOException, ClassNotFoundException {
+ if (jobInformation != null) {
+ return jobInformation;
+ }
if (serializedJobInformation instanceof NonOffloaded) {
NonOffloaded jobInformation =
(NonOffloaded) serializedJobInformation;
- return jobInformation.serializedValue;
- } else {
- throw new IllegalStateException(
- "Trying to work with offloaded serialized job information.");
+ return jobInformation.serializedValue.deserializeValue(getClass().getClassLoader());
}
+ throw new IllegalStateException(
+ "Trying to work with offloaded serialized job information.");
}
/**
- * Return the sub task's serialized task information.
+ * Return the sub task's task information.
*
- * @return serialized task information (may throw {@link IllegalStateException} if {@link
- * #loadBigData} is not called beforehand)).
+ * @return task information (may throw {@link IllegalStateException} if {@link #loadBigData} is
+ * not called beforehand)).
* @throws IllegalStateException If job information is offloaded to BLOB store.
*/
- public SerializedValue getSerializedTaskInformation() {
+ public TaskInformation getTaskInformation() throws IOException, ClassNotFoundException {
+ if (taskInformation != null) {
+ return taskInformation;
+ }
if (serializedTaskInformation instanceof NonOffloaded) {
NonOffloaded taskInformation =
(NonOffloaded) serializedTaskInformation;
- return taskInformation.serializedValue;
- } else {
- throw new IllegalStateException(
- "Trying to work with offloaded serialized job information.");
+ return taskInformation.serializedValue.deserializeValue(getClass().getClassLoader());
}
+ throw new IllegalStateException(
+ "Trying to work with offloaded serialized task information.");
}
/**
@@ -242,7 +261,9 @@ public AllocationID getAllocationId() {
*/
public void loadBigData(
@Nullable PermanentBlobService blobService,
- ShuffleDescriptorsCache shuffleDescriptorsCache)
+ GroupCache jobInformationCache,
+ GroupCache taskInformationCache,
+ GroupCache shuffleDescriptorsCache)
throws IOException, ClassNotFoundException {
// re-integrate offloaded job info from blob
@@ -253,13 +274,19 @@ public void loadBigData(
Preconditions.checkNotNull(blobService);
- final File dataFile = blobService.getFile(jobId, jobInfoKey);
- // NOTE: Do not delete the job info BLOB since it may be needed again during recovery.
- // (it is deleted automatically on the BLOB server and cache when the job
- // enters a terminal state)
- SerializedValue serializedValue =
- SerializedValue.fromBytes(FileUtils.readAllBytes(dataFile.toPath()));
- serializedJobInformation = new NonOffloaded<>(serializedValue);
+ JobInformation jobInformation = jobInformationCache.get(jobId, jobInfoKey);
+ if (jobInformation == null) {
+ final File dataFile = blobService.getFile(jobId, jobInfoKey);
+ // NOTE: Do not delete the job info BLOB since it may be needed again during
+ // recovery. (it is deleted automatically on the BLOB server and cache when the job
+ // enters a terminal state)
+ jobInformation =
+ InstantiationUtil.deserializeObject(
+ new BufferedInputStream(Files.newInputStream(dataFile.toPath())),
+ getClass().getClassLoader());
+ jobInformationCache.put(jobId, jobInfoKey, jobInformation);
+ }
+ this.jobInformation = jobInformation.deepCopy();
}
// re-integrate offloaded task info from blob
@@ -269,23 +296,25 @@ public void loadBigData(
Preconditions.checkNotNull(blobService);
- final File dataFile = blobService.getFile(jobId, taskInfoKey);
- // NOTE: Do not delete the task info BLOB since it may be needed again during recovery.
- // (it is deleted automatically on the BLOB server and cache when the job
- // enters a terminal state)
- SerializedValue serializedValue =
- SerializedValue.fromBytes(FileUtils.readAllBytes(dataFile.toPath()));
- serializedTaskInformation = new NonOffloaded<>(serializedValue);
+ TaskInformation taskInformation = taskInformationCache.get(jobId, taskInfoKey);
+ if (taskInformation == null) {
+ final File dataFile = blobService.getFile(jobId, taskInfoKey);
+ // NOTE: Do not delete the task info BLOB since it may be needed again during
+ // recovery. (it is deleted automatically on the BLOB server and cache when the job
+ // enters a terminal state)
+ taskInformation =
+ InstantiationUtil.deserializeObject(
+ new BufferedInputStream(Files.newInputStream(dataFile.toPath())),
+ getClass().getClassLoader());
+ taskInformationCache.put(jobId, taskInfoKey, taskInformation);
+ }
+ this.taskInformation = taskInformation.deepCopy();
}
for (InputGateDeploymentDescriptor inputGate : inputGates) {
inputGate.tryLoadAndDeserializeShuffleDescriptors(
blobService, jobId, shuffleDescriptorsCache);
}
-
- // make sure that the serialized job and task information fields are filled
- Preconditions.checkNotNull(serializedJobInformation);
- Preconditions.checkNotNull(serializedTaskInformation);
}
@Override
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java
index e657c8d740012..a734294ad9dee 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java
@@ -21,6 +21,7 @@
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
@@ -88,7 +89,10 @@ public CheckpointResourcesCleanupRunner(
this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor);
this.initializationTimestamp = initializationTimestamp;
- this.checkpointsCleaner = new CheckpointsCleaner();
+ this.checkpointsCleaner =
+ new CheckpointsCleaner(
+ jobManagerConfiguration.getBoolean(
+ CheckpointingOptions.CLEANER_PARALLEL_MODE));
this.resultFuture = new CompletableFuture<>();
this.cleanupFuture = resultFuture.thenCompose(ignored -> runCleanupAsync());
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
index f30bf6f32d214..5792caa271153 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobInformation.java
@@ -21,13 +21,18 @@
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.SerializedValue;
+import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableCollection;
+import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList;
+
import java.io.Serializable;
import java.net.URL;
import java.util.Collection;
+import java.util.Objects;
/** Container class for job information which is stored in the {@link ExecutionGraph}. */
public class JobInformation implements Serializable {
@@ -44,13 +49,13 @@ public class JobInformation implements Serializable {
private final SerializedValue serializedExecutionConfig;
/** Configuration of the job. */
- private final Configuration jobConfiguration;
+ private final UnmodifiableConfiguration jobConfiguration;
/** Blob keys for the required jar files. */
- private final Collection requiredJarFileBlobKeys;
+ private final ImmutableCollection requiredJarFileBlobKeys;
/** URLs specifying the classpath to add to the class loader. */
- private final Collection requiredClasspathURLs;
+ private final ImmutableCollection requiredClasspathURLs;
public JobInformation(
JobID jobId,
@@ -62,9 +67,12 @@ public JobInformation(
this.jobId = Preconditions.checkNotNull(jobId);
this.jobName = Preconditions.checkNotNull(jobName);
this.serializedExecutionConfig = Preconditions.checkNotNull(serializedExecutionConfig);
- this.jobConfiguration = Preconditions.checkNotNull(jobConfiguration);
- this.requiredJarFileBlobKeys = Preconditions.checkNotNull(requiredJarFileBlobKeys);
- this.requiredClasspathURLs = Preconditions.checkNotNull(requiredClasspathURLs);
+ this.jobConfiguration =
+ new UnmodifiableConfiguration(Preconditions.checkNotNull(jobConfiguration));
+ this.requiredJarFileBlobKeys =
+ ImmutableList.copyOf(Preconditions.checkNotNull(requiredJarFileBlobKeys));
+ this.requiredClasspathURLs =
+ ImmutableList.copyOf(Preconditions.checkNotNull(requiredClasspathURLs));
}
public JobID getJobId() {
@@ -79,18 +87,51 @@ public SerializedValue getSerializedExecutionConfig() {
return serializedExecutionConfig;
}
- public Configuration getJobConfiguration() {
+ public UnmodifiableConfiguration getJobConfiguration() {
return jobConfiguration;
}
- public Collection getRequiredJarFileBlobKeys() {
+ public ImmutableCollection getRequiredJarFileBlobKeys() {
return requiredJarFileBlobKeys;
}
- public Collection getRequiredClasspathURLs() {
+ public ImmutableCollection getRequiredClasspathURLs() {
return requiredClasspathURLs;
}
+ // All fields are immutable, so return this directly.
+ public JobInformation deepCopy() {
+ return this;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ JobInformation that = (JobInformation) o;
+ return Objects.equals(jobId, that.jobId)
+ && Objects.equals(jobName, that.jobName)
+ && Objects.equals(serializedExecutionConfig, that.serializedExecutionConfig)
+ && Objects.equals(jobConfiguration, that.jobConfiguration)
+ && Objects.equals(requiredJarFileBlobKeys, that.requiredJarFileBlobKeys)
+ && Objects.equals(requiredClasspathURLs, that.requiredClasspathURLs);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ jobId,
+ jobName,
+ serializedExecutionConfig,
+ jobConfiguration,
+ requiredJarFileBlobKeys,
+ requiredClasspathURLs);
+ }
+
// ------------------------------------------------------------------------
@Override
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java
index 6a3b1a26e7f0b..e1b59d4bfb12f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/TaskInformation.java
@@ -23,6 +23,7 @@
import org.apache.flink.util.Preconditions;
import java.io.Serializable;
+import java.util.Objects;
/**
* Container class for operator/task specific information which are stored at the {@link
@@ -88,4 +89,43 @@ public String getInvokableClassName() {
public Configuration getTaskConfiguration() {
return taskConfiguration;
}
+
+ public TaskInformation deepCopy() {
+ return new TaskInformation(
+ getJobVertexId(),
+ getTaskName(),
+ getNumberOfSubtasks(),
+ getMaxNumberOfSubtasks(),
+ getInvokableClassName(),
+ // Return the new Configuration to avoid shared conf being changed.
+ new Configuration(getTaskConfiguration()));
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ TaskInformation that = (TaskInformation) o;
+ return numberOfSubtasks == that.numberOfSubtasks
+ && maxNumberOfSubtasks == that.maxNumberOfSubtasks
+ && Objects.equals(jobVertexId, that.jobVertexId)
+ && Objects.equals(taskName, that.taskName)
+ && Objects.equals(invokableClassName, that.invokableClassName)
+ && Objects.equals(taskConfiguration, that.taskConfiguration);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ jobVertexId,
+ taskName,
+ numberOfSubtasks,
+ maxNumberOfSubtasks,
+ invokableClassName,
+ taskConfiguration);
+ }
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java
index 2faff9650d3e9..f3ae8ff7d2b42 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java
@@ -61,6 +61,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.METRIC_GROUP_INPUT;
import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.METRIC_GROUP_OUTPUT;
@@ -68,6 +69,7 @@
import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.registerDebloatingTaskMetrics;
import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.registerInputMetrics;
import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.registerOutputMetrics;
+import static org.apache.flink.util.ExecutorUtils.gracefulShutdown;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -409,15 +411,15 @@ public void close() {
}
try {
- batchShuffleReadBufferPool.destroy();
+ gracefulShutdown(10, TimeUnit.SECONDS, batchShuffleReadIOExecutor);
} catch (Throwable t) {
- LOG.warn("Cannot shut down batch shuffle read buffer pool properly.", t);
+ LOG.warn("Cannot shut down batch shuffle read IO executor properly.", t);
}
try {
- batchShuffleReadIOExecutor.shutdown();
+ batchShuffleReadBufferPool.destroy();
} catch (Throwable t) {
- LOG.warn("Cannot shut down batch shuffle read IO executor properly.", t);
+ LOG.warn("Cannot shut down batch shuffle read buffer pool properly.", t);
}
isClosed = true;
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java
index fe71f47762b08..7f708b45188cc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleServiceFactory.java
@@ -43,6 +43,7 @@
import org.apache.flink.runtime.shuffle.ShuffleServiceFactory;
import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -86,6 +87,7 @@ public NettyShuffleEnvironment createShuffleEnvironment(
shuffleEnvironmentContext.getEventPublisher(),
shuffleEnvironmentContext.getParentMetricGroup(),
shuffleEnvironmentContext.getIoExecutor(),
+ shuffleEnvironmentContext.getScheduledExecutor(),
shuffleEnvironmentContext.getNumberOfSlots(),
shuffleEnvironmentContext.getTmpDirPaths());
}
@@ -97,13 +99,15 @@ static NettyShuffleEnvironment createNettyShuffleEnvironment(
TaskEventPublisher taskEventPublisher,
MetricGroup metricGroup,
Executor ioExecutor,
+ ScheduledExecutor scheduledExecutor,
int numberOfSlots,
String[] tmpDirPaths) {
return createNettyShuffleEnvironment(
config,
taskExecutorResourceId,
taskEventPublisher,
- new ResultPartitionManager(),
+ new ResultPartitionManager(
+ config.getPartitionRequestListenerTimeout(), scheduledExecutor),
metricGroup,
ioExecutor,
numberOfSlots,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java
index fc317ef2b023a..665dc1b975009 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkSequenceViewReader.java
@@ -18,7 +18,11 @@
package org.apache.flink.runtime.io.network;
+import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
+import org.apache.flink.runtime.io.network.partition.PartitionRequestListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability;
@@ -34,12 +38,36 @@
*/
public interface NetworkSequenceViewReader {
- void requestSubpartitionView(
+ /**
+ * When the netty server receives the downstream task's partition request and the upstream task
+ * has registered its partition, it will process the partition request immediately, otherwise it
+ * will create a {@link PartitionRequestListener} for given {@link ResultPartitionID} in {@link
+ * ResultPartitionManager} and notify the listener when the upstream task registers its
+ * partition.
+ *
+ * @param partitionProvider the result partition provider
+ * @param resultPartitionId the result partition id
+ * @param subPartitionIndex the sub partition index
+ * @throws IOException the thrown exception
+ */
+ void requestSubpartitionViewOrRegisterListener(
ResultPartitionProvider partitionProvider,
ResultPartitionID resultPartitionId,
int subPartitionIndex)
throws IOException;
+ /**
+ * When the {@link ResultPartitionManager} registers {@link ResultPartition}, it will get the
+ * {@link PartitionRequestListener} via given {@link ResultPartitionID}, and create subpartition
+ * view reader for downstream task.
+ *
+ * @param partition the result partition
+ * @param subPartitionIndex the sub partition index
+ * @throws IOException the thrown exception
+ */
+ void notifySubpartitionCreated(ResultPartition partition, int subPartitionIndex)
+ throws IOException;
+
@Nullable
BufferAndAvailability getNextBuffer() throws IOException;
@@ -91,4 +119,12 @@ void requestSubpartitionView(
InputChannelID getReceiverId();
void notifyNewBufferSize(int newBufferSize);
+
+ /**
+ * When the partition request from the given downstream task is timeout, it should notify the
+ * reader in netty server and send {@link PartitionNotFoundException} to the task.
+ *
+ * @param partitionRequestListener the timeout message of given {@link PartitionRequestListener}
+ */
+ void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener);
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
index 4aed5e138df2d..c12db284c13c5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java
@@ -22,6 +22,8 @@
import org.apache.flink.runtime.io.network.NetworkSequenceViewReader;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.PartitionRequestListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
@@ -33,8 +35,10 @@
import javax.annotation.Nullable;
import java.io.IOException;
+import java.util.Optional;
import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkState;
/**
* Simple wrapper for the subpartition view used in the new network credit-based mode.
@@ -55,6 +59,8 @@ class CreditBasedSequenceNumberingViewReader
private volatile ResultSubpartitionView subpartitionView;
+ private volatile PartitionRequestListener partitionRequestListener;
+
/**
* The status indicating whether this reader is already enqueued in the pipeline for
* transferring data or not.
@@ -78,27 +84,47 @@ class CreditBasedSequenceNumberingViewReader
}
@Override
- public void requestSubpartitionView(
+ public void requestSubpartitionViewOrRegisterListener(
ResultPartitionProvider partitionProvider,
ResultPartitionID resultPartitionId,
int subPartitionIndex)
throws IOException {
-
synchronized (requestLock) {
- if (subpartitionView == null) {
- // This call can trigger a notification we have to
- // schedule a separate task at the event loop that will
- // start consuming this. Otherwise the reference to the
- // view cannot be available in getNextBuffer().
- this.subpartitionView =
- partitionProvider.createSubpartitionView(
- resultPartitionId, subPartitionIndex, this);
+ checkState(subpartitionView == null, "Subpartition already requested");
+ checkState(
+ partitionRequestListener == null, "Partition request listener already created");
+ partitionRequestListener =
+ new NettyPartitionRequestListener(
+ partitionProvider, this, subPartitionIndex, resultPartitionId);
+ // The partition provider will create subpartitionView if resultPartition is
+ // registered, otherwise it will register a listener of partition request to the result
+ // partition manager.
+ Optional subpartitionViewOptional =
+ partitionProvider.createSubpartitionViewOrRegisterListener(
+ resultPartitionId, subPartitionIndex, this, partitionRequestListener);
+ if (subpartitionViewOptional.isPresent()) {
+ this.subpartitionView = subpartitionViewOptional.get();
} else {
- throw new IllegalStateException("Subpartition already requested");
+ // If the subpartitionView is not exist, it means that the requested partition is
+ // not registered.
+ return;
}
}
notifyDataAvailable();
+ requestQueue.notifyReaderCreated(this);
+ }
+
+ @Override
+ public void notifySubpartitionCreated(ResultPartition partition, int subPartitionIndex)
+ throws IOException {
+ synchronized (requestLock) {
+ checkState(subpartitionView == null, "Subpartition already requested");
+ subpartitionView = partition.createSubpartitionView(subPartitionIndex, this);
+ }
+
+ notifyDataAvailable();
+ requestQueue.notifyReaderCreated(this);
}
@Override
@@ -182,6 +208,12 @@ public void notifyNewBufferSize(int newBufferSize) {
subpartitionView.notifyNewBufferSize(newBufferSize);
}
+ @Override
+ public void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener) {
+ requestQueue.notifyPartitionRequestTimeout(partitionRequestListener);
+ this.partitionRequestListener = null;
+ }
+
@VisibleForTesting
int getNumCreditsAvailable() {
return numCreditsAvailable;
@@ -226,6 +258,9 @@ public Throwable getFailureCause() {
@Override
public void releaseAllResources() throws IOException {
+ if (partitionRequestListener != null) {
+ partitionRequestListener.releaseListener();
+ }
subpartitionView.releaseAllResources();
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java
new file mode 100644
index 0000000000000..b09756bf22d64
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestListener.java
@@ -0,0 +1,103 @@
+/*
+ * 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.runtime.io.network.netty;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.io.network.NetworkSequenceViewReader;
+import org.apache.flink.runtime.io.network.partition.PartitionRequestListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Implementation of {@link PartitionRequestListener} for netty partition request. */
+public class NettyPartitionRequestListener implements PartitionRequestListener {
+ private final ResultPartitionProvider resultPartitionProvider;
+ private final NetworkSequenceViewReader reader;
+ private final int subPartitionIndex;
+ private final ResultPartitionID resultPartitionId;
+ private final long createTimestamp;
+
+ public NettyPartitionRequestListener(
+ ResultPartitionProvider resultPartitionProvider,
+ NetworkSequenceViewReader reader,
+ int subPartitionIndex,
+ ResultPartitionID resultPartitionId) {
+ this(
+ resultPartitionProvider,
+ reader,
+ subPartitionIndex,
+ resultPartitionId,
+ System.currentTimeMillis());
+ }
+
+ @VisibleForTesting
+ public NettyPartitionRequestListener(
+ ResultPartitionProvider resultPartitionProvider,
+ NetworkSequenceViewReader reader,
+ int subPartitionIndex,
+ ResultPartitionID resultPartitionId,
+ long createTimestamp) {
+ this.resultPartitionProvider = resultPartitionProvider;
+ this.reader = reader;
+ this.subPartitionIndex = subPartitionIndex;
+ this.resultPartitionId = resultPartitionId;
+ this.createTimestamp = createTimestamp;
+ }
+
+ @Override
+ public long getCreateTimestamp() {
+ return createTimestamp;
+ }
+
+ @Override
+ public ResultPartitionID getResultPartitionId() {
+ return resultPartitionId;
+ }
+
+ @Override
+ public NetworkSequenceViewReader getViewReader() {
+ return reader;
+ }
+
+ @Override
+ public InputChannelID getReceiverId() {
+ return reader.getReceiverId();
+ }
+
+ @Override
+ public void notifyPartitionCreated(ResultPartition partition) throws IOException {
+ checkNotNull(partition);
+ reader.notifySubpartitionCreated(partition, subPartitionIndex);
+ }
+
+ @Override
+ public void notifyPartitionCreatedTimeout() {
+ reader.notifyPartitionRequestTimeout(this);
+ }
+
+ @Override
+ public void releaseListener() {
+ resultPartitionProvider.releasePartitionRequestListener(this);
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
index 57337fdc4f7de..933e122899adf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java
@@ -22,6 +22,9 @@
import org.apache.flink.runtime.io.network.NetworkSequenceViewReader;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse;
+import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
+import org.apache.flink.runtime.io.network.partition.PartitionRequestListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
@@ -259,6 +262,26 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exc
if (toRelease != null) {
releaseViewReader(toRelease);
}
+ } else if (msg instanceof PartitionRequestListener) {
+ PartitionRequestListener partitionRequestListener = (PartitionRequestListener) msg;
+
+ // Send partition not found message to the downstream task when the listener is timeout.
+ final ResultPartitionID resultPartitionId =
+ partitionRequestListener.getResultPartitionId();
+ final InputChannelID inputChannelId = partitionRequestListener.getReceiverId();
+ availableReaders.remove(partitionRequestListener.getViewReader());
+ allReaders.remove(inputChannelId);
+ try {
+ ctx.writeAndFlush(
+ new NettyMessage.ErrorResponse(
+ new PartitionNotFoundException(resultPartitionId), inputChannelId));
+ } catch (Exception e) {
+ LOG.warn(
+ "Write partition not found exception to {} for result partition {} fail",
+ inputChannelId,
+ resultPartitionId,
+ e);
+ }
} else {
ctx.fireUserEventTriggered(msg);
}
@@ -358,7 +381,10 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E
}
private void handleException(Channel channel, Throwable cause) throws IOException {
- LOG.error("Encountered error while consuming partitions", cause);
+ LOG.error(
+ "Encountered error while consuming partitions (connection to {})",
+ channel.remoteAddress(),
+ cause);
fatalError = true;
releaseAllResources();
@@ -393,6 +419,10 @@ private void onChannelFutureFailure(ChannelFuture future) throws Exception {
}
}
+ public void notifyPartitionRequestTimeout(PartitionRequestListener partitionRequestListener) {
+ ctx.pipeline().fireUserEventTriggered(partitionRequestListener);
+ }
+
// This listener is called after an element of the current nonEmptyReader has been
// flushed. If successful, the listener triggers further processing of the
// queues.
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
index ccd9b1c53a872..a0d6e25aec29f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java
@@ -29,7 +29,6 @@
import org.apache.flink.runtime.io.network.netty.NettyMessage.ResumeConsumption;
import org.apache.flink.runtime.io.network.netty.NettyMessage.SegmentId;
import org.apache.flink.runtime.io.network.netty.NettyMessage.TaskEventRequest;
-import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
@@ -83,19 +82,14 @@ protected void channelRead0(ChannelHandlerContext ctx, NettyMessage msg) throws
LOG.debug("Read channel on {}: {}.", ctx.channel().localAddress(), request);
- try {
- NetworkSequenceViewReader reader;
- reader =
- new CreditBasedSequenceNumberingViewReader(
- request.receiverId, request.credit, outboundQueue);
+ NetworkSequenceViewReader reader;
+ reader =
+ new CreditBasedSequenceNumberingViewReader(
+ request.receiverId, request.credit, outboundQueue);
- reader.requestSubpartitionView(
- partitionProvider, request.partitionId, request.queueIndex);
+ reader.requestSubpartitionViewOrRegisterListener(
+ partitionProvider, request.partitionId, request.queueIndex);
- outboundQueue.notifyReaderCreated(reader);
- } catch (PartitionNotFoundException notFound) {
- respondWithError(ctx, notFound, request.receiverId);
- }
}
// ----------------------------------------------------------------
// Task events
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java
new file mode 100644
index 0000000000000..82253be5256ab
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListener.java
@@ -0,0 +1,77 @@
+/*
+ * 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.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.NetworkSequenceViewReader;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+
+import java.io.IOException;
+
+/**
+ * When the netty server receives a downstream task's partition request event and finds its upstream
+ * task doesn't register its partition yet, the netty server will construct a {@link
+ * PartitionRequestListener} and notify the listener when the task deploys itself and registers its
+ * partition to {@link ResultPartitionManager}.
+ */
+public interface PartitionRequestListener {
+
+ /**
+ * The creation timestamp of this notifier, it's used to check whether the notifier is timeout.
+ *
+ * @return the creation timestamp
+ */
+ long getCreateTimestamp();
+
+ /**
+ * Get the result partition id of the notifier.
+ *
+ * @return the result partition id
+ */
+ ResultPartitionID getResultPartitionId();
+
+ /**
+ * Get the view reader of the notifier.
+ *
+ * @return the view reader
+ */
+ NetworkSequenceViewReader getViewReader();
+
+ /**
+ * Get the input channel id of the notifier.
+ *
+ * @return the input channel id
+ */
+ InputChannelID getReceiverId();
+
+ /**
+ * Notify the partition request listener when the given partition is registered.
+ *
+ * @param partition The registered partition.
+ */
+ void notifyPartitionCreated(ResultPartition partition) throws IOException;
+
+ /**
+ * When the partition request listener is timeout, it will be notified to send {@link
+ * PartitionNotFoundException}.
+ */
+ void notifyPartitionCreatedTimeout();
+
+ /** Release this listener. */
+ void releaseListener();
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java
new file mode 100644
index 0000000000000..e296d767fb890
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionRequestListenerManager.java
@@ -0,0 +1,78 @@
+/*
+ * 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.runtime.io.network.partition;
+
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/** Manages partition request listener with input channel id. */
+public class PartitionRequestListenerManager {
+ private final Map listeners;
+
+ public PartitionRequestListenerManager() {
+ this.listeners = new HashMap<>();
+ }
+
+ public Collection getPartitionRequestListeners() {
+ return listeners.values();
+ }
+
+ public void remove(InputChannelID receiverId) {
+ listeners.remove(receiverId);
+ }
+
+ public boolean isEmpty() {
+ return listeners.isEmpty();
+ }
+
+ public void registerListener(PartitionRequestListener listener) {
+ PartitionRequestListener previous = listeners.put(listener.getReceiverId(), listener);
+ if (previous != null) {
+ throw new IllegalStateException(
+ "Partition request listener with receiver "
+ + listener.getReceiverId()
+ + " has been registered.");
+ }
+ }
+
+ /**
+ * Remove the expire partition request listener and add it to the given timeoutListeners.
+ *
+ * @param now the timestamp
+ * @param timeout the timeout mills
+ * @param timeoutListeners the expire partition request listeners
+ */
+ public void removeExpiration(
+ long now, long timeout, Collection timeoutListeners) {
+ Iterator> iterator =
+ listeners.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry entry = iterator.next();
+ PartitionRequestListener partitionRequestListener = entry.getValue();
+ if ((now - partitionRequestListener.getCreateTimestamp()) > timeout) {
+ timeoutListeners.add(partitionRequestListener);
+ iterator.remove();
+ }
+ }
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
index 223052681c024..7657bf538b883 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
@@ -18,14 +18,26 @@
package org.apache.flink.runtime.io.network.partition;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.util.CollectionUtil;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
import java.io.IOException;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
import static org.apache.flink.util.Preconditions.checkState;
@@ -40,9 +52,36 @@ public class ResultPartitionManager implements ResultPartitionProvider {
private final Map registeredPartitions =
CollectionUtil.newHashMapWithExpectedSize(16);
+ @GuardedBy("registeredPartitions")
+ private final Map listenerManagers =
+ new HashMap<>();
+
+ @Nullable private ScheduledFuture> partitionListenerTimeoutChecker;
+
+ private final int partitionListenerTimeout;
+
private boolean isShutdown;
- public void registerResultPartition(ResultPartition partition) {
+ @VisibleForTesting
+ public ResultPartitionManager() {
+ this(0, null);
+ }
+
+ public ResultPartitionManager(
+ int partitionListenerTimeout, ScheduledExecutor scheduledExecutor) {
+ this.partitionListenerTimeout = partitionListenerTimeout;
+ if (partitionListenerTimeout > 0 && scheduledExecutor != null) {
+ this.partitionListenerTimeoutChecker =
+ scheduledExecutor.scheduleWithFixedDelay(
+ this::checkRequestPartitionListeners,
+ partitionListenerTimeout,
+ partitionListenerTimeout,
+ TimeUnit.MILLISECONDS);
+ }
+ }
+
+ public void registerResultPartition(ResultPartition partition) throws IOException {
+ PartitionRequestListenerManager listenerManager;
synchronized (registeredPartitions) {
checkState(!isShutdown, "Result partition manager already shut down.");
@@ -53,8 +92,16 @@ public void registerResultPartition(ResultPartition partition) {
throw new IllegalStateException("Result partition already registered.");
}
- LOG.debug("Registered {}.", partition);
+ listenerManager = listenerManagers.remove(partition.getPartitionId());
+ }
+ if (listenerManager != null) {
+ for (PartitionRequestListener listener :
+ listenerManager.getPartitionRequestListeners()) {
+ listener.notifyPartitionCreated(partition);
+ }
}
+
+ LOG.debug("Registered {}.", partition);
}
@Override
@@ -81,7 +128,51 @@ public ResultSubpartitionView createSubpartitionView(
return subpartitionView;
}
+ @Override
+ public Optional createSubpartitionViewOrRegisterListener(
+ ResultPartitionID partitionId,
+ int subpartitionIndex,
+ BufferAvailabilityListener availabilityListener,
+ PartitionRequestListener partitionRequestListener)
+ throws IOException {
+
+ final ResultSubpartitionView subpartitionView;
+ synchronized (registeredPartitions) {
+ final ResultPartition partition = registeredPartitions.get(partitionId);
+
+ if (partition == null) {
+ listenerManagers
+ .computeIfAbsent(partitionId, key -> new PartitionRequestListenerManager())
+ .registerListener(partitionRequestListener);
+ subpartitionView = null;
+ } else {
+
+ LOG.debug("Requesting subpartition {} of {}.", subpartitionIndex, partition);
+
+ subpartitionView =
+ partition.createSubpartitionView(subpartitionIndex, availabilityListener);
+ }
+ }
+
+ return subpartitionView == null ? Optional.empty() : Optional.of(subpartitionView);
+ }
+
+ @Override
+ public void releasePartitionRequestListener(PartitionRequestListener listener) {
+ synchronized (registeredPartitions) {
+ PartitionRequestListenerManager listenerManager =
+ listenerManagers.get(listener.getResultPartitionId());
+ if (listenerManager != null) {
+ listenerManager.remove(listener.getReceiverId());
+ if (listenerManager.isEmpty()) {
+ listenerManagers.remove(listener.getResultPartitionId());
+ }
+ }
+ }
+ }
+
public void releasePartition(ResultPartitionID partitionId, Throwable cause) {
+ PartitionRequestListenerManager listenerManager;
synchronized (registeredPartitions) {
ResultPartition resultPartition = registeredPartitions.remove(partitionId);
if (resultPartition != null) {
@@ -91,6 +182,13 @@ public void releasePartition(ResultPartitionID partitionId, Throwable cause) {
partitionId.getPartitionId(),
partitionId.getProducerId());
}
+ listenerManager = listenerManagers.remove(partitionId);
+ }
+ if (listenerManager != null && !listenerManager.isEmpty()) {
+ for (PartitionRequestListener listener :
+ listenerManager.getPartitionRequestListeners()) {
+ listener.notifyPartitionCreatedTimeout();
+ }
}
}
@@ -106,12 +204,61 @@ public void shutdown() {
registeredPartitions.clear();
+ releaseListenerManagers();
+
+ // stop the timeout checks for the TaskManagers
+ if (partitionListenerTimeoutChecker != null) {
+ partitionListenerTimeoutChecker.cancel(false);
+ partitionListenerTimeoutChecker = null;
+ }
+
isShutdown = true;
LOG.debug("Successful shutdown.");
}
}
+ private void releaseListenerManagers() {
+ for (PartitionRequestListenerManager listenerManager : listenerManagers.values()) {
+ for (PartitionRequestListener listener :
+ listenerManager.getPartitionRequestListeners()) {
+ listener.notifyPartitionCreatedTimeout();
+ }
+ }
+ listenerManagers.clear();
+ }
+
+ /** Check whether the partition request listener is timeout. */
+ private void checkRequestPartitionListeners() {
+ List timeoutPartitionRequestListeners = new LinkedList<>();
+ synchronized (registeredPartitions) {
+ if (isShutdown) {
+ return;
+ }
+ long now = System.currentTimeMillis();
+ Iterator> iterator =
+ listenerManagers.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry entry =
+ iterator.next();
+ PartitionRequestListenerManager partitionRequestListeners = entry.getValue();
+ partitionRequestListeners.removeExpiration(
+ now, partitionListenerTimeout, timeoutPartitionRequestListeners);
+ if (partitionRequestListeners.isEmpty()) {
+ iterator.remove();
+ }
+ }
+ }
+ for (PartitionRequestListener partitionRequestListener : timeoutPartitionRequestListeners) {
+ partitionRequestListener.notifyPartitionCreatedTimeout();
+ }
+ }
+
+ @VisibleForTesting
+ public Map getListenerManagers() {
+ return listenerManagers;
+ }
+
// ------------------------------------------------------------------------
// Notifications
// ------------------------------------------------------------------------
@@ -131,6 +278,12 @@ void onConsumedPartition(ResultPartition partition) {
partitionId.getPartitionId(),
partitionId.getProducerId());
}
+ PartitionRequestListenerManager listenerManager =
+ listenerManagers.remove(partition.getPartitionId());
+ checkState(
+ listenerManager == null || listenerManager.isEmpty(),
+ "The partition request listeners is not empty for "
+ + partition.getPartitionId());
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java
index 0cd323a3c66d4..b95cc6f8c968d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.network.partition;
import java.io.IOException;
+import java.util.Optional;
/** Interface for creating result partitions. */
public interface ResultPartitionProvider {
@@ -29,4 +30,29 @@ ResultSubpartitionView createSubpartitionView(
int index,
BufferAvailabilityListener availabilityListener)
throws IOException;
+
+ /**
+ * If the upstream task's partition has been registered, returns the result subpartition input
+ * view immediately, otherwise register the listener and return empty.
+ *
+ * @param partitionId the result partition id
+ * @param index the index
+ * @param availabilityListener the buffer availability listener
+ * @param partitionRequestListener the partition request listener
+ * @return the result subpartition view
+ * @throws IOException the thrown exception
+ */
+ Optional createSubpartitionViewOrRegisterListener(
+ ResultPartitionID partitionId,
+ int index,
+ BufferAvailabilityListener availabilityListener,
+ PartitionRequestListener partitionRequestListener)
+ throws IOException;
+
+ /**
+ * Release the given listener in this result partition provider.
+ *
+ * @param listener the given listener
+ */
+ void releasePartitionRequestListener(PartitionRequestListener listener);
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
index 1d5a444e2c4e7..f36c5d95aec56 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
@@ -77,7 +77,7 @@ public abstract class InputChannel {
protected final Counter numBuffersIn;
/** The current backoff (in ms). */
- private int currentBackoff;
+ protected int currentBackoff;
protected InputChannel(
SingleInputGate inputGate,
@@ -105,7 +105,7 @@ protected InputChannel(
this.initialBackoff = initial;
this.maxBackoff = max;
- this.currentBackoff = initial == 0 ? -1 : 0;
+ this.currentBackoff = 0;
this.numBytesIn = numBytesIn;
this.numBuffersIn = numBuffersIn;
@@ -277,12 +277,12 @@ protected int getCurrentBackoff() {
*/
protected boolean increaseBackoff() {
// Backoff is disabled
- if (currentBackoff < 0) {
+ if (initialBackoff == 0) {
return false;
}
- // This is the first time backing off
if (currentBackoff == 0) {
+ // This is the first time backing off
currentBackoff = initialBackoff;
return true;
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
index b4fdc3e3c108c..2559e02ea8c92 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
@@ -100,6 +100,9 @@ public class RemoteInputChannel extends InputChannel {
/** The initial number of exclusive buffers assigned to this channel. */
private final int initialCredit;
+ /** The milliseconds timeout for partition request listener in result partition manager. */
+ private final int partitionRequestListenerTimeout;
+
/** The number of available buffers that have not been announced to the producer yet. */
private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
@@ -124,6 +127,7 @@ public RemoteInputChannel(
ConnectionManager connectionManager,
int initialBackOff,
int maxBackoff,
+ int partitionRequestListenerTimeout,
int networkBuffersPerChannel,
Counter numBytesIn,
Counter numBuffersIn,
@@ -140,6 +144,7 @@ public RemoteInputChannel(
numBuffersIn);
checkArgument(networkBuffersPerChannel >= 0, "Must be non-negative.");
+ this.partitionRequestListenerTimeout = partitionRequestListenerTimeout;
this.initialCredit = networkBuffersPerChannel;
this.connectionId = checkNotNull(connectionId);
this.connectionManager = checkNotNull(connectionManager);
@@ -201,12 +206,30 @@ void retriggerSubpartitionRequest() throws IOException {
if (increaseBackoff()) {
partitionRequestClient.requestSubpartition(
- partitionId, consumedSubpartitionIndex, this, getCurrentBackoff());
+ partitionId, consumedSubpartitionIndex, this, 0);
} else {
failPartitionRequest();
}
}
+ /**
+ * The remote task manager creates partition request listener and returns {@link
+ * PartitionNotFoundException} until the listener is timeout, so the backoff should add the
+ * timeout milliseconds if it exists.
+ *
+ * @return true
, iff the operation was successful. Otherwise, false
.
+ */
+ @Override
+ protected boolean increaseBackoff() {
+ if (partitionRequestListenerTimeout > 0) {
+ currentBackoff += partitionRequestListenerTimeout;
+ return currentBackoff < 2 * maxBackoff;
+ }
+
+ // Backoff is disabled
+ return false;
+ }
+
@Override
public Optional getNextBuffer() throws IOException {
checkPartitionRequestQueueInitialized();
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java
index 2c0efa01592ab..f63ab7564f6a2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java
@@ -34,6 +34,7 @@
public class RemoteRecoveredInputChannel extends RecoveredInputChannel {
private final ConnectionID connectionId;
private final ConnectionManager connectionManager;
+ private final int partitionRequestListenerTimeout;
RemoteRecoveredInputChannel(
SingleInputGate inputGate,
@@ -44,6 +45,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel {
ConnectionManager connectionManager,
int initialBackOff,
int maxBackoff,
+ int partitionRequestListenerTimeout,
int networkBuffersPerChannel,
InputChannelMetrics metrics) {
super(
@@ -59,6 +61,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel {
this.connectionId = checkNotNull(connectionId);
this.connectionManager = checkNotNull(connectionManager);
+ this.partitionRequestListenerTimeout = partitionRequestListenerTimeout;
}
@Override
@@ -73,6 +76,7 @@ protected InputChannel toInputChannelInternal() throws IOException {
connectionManager,
initialBackoff,
maxBackoff,
+ partitionRequestListenerTimeout,
networkBuffersPerChannel,
numBytesIn,
numBuffersIn,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java
index de35b31bff1f8..38f6d13ebfa6b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java
@@ -79,6 +79,8 @@ public class SingleInputGateFactory {
protected final int partitionRequestMaxBackoff;
+ protected final int partitionRequestListenerTimeout;
+
@Nonnull protected final ConnectionManager connectionManager;
@Nonnull protected final ResultPartitionManager partitionManager;
@@ -118,6 +120,7 @@ public SingleInputGateFactory(
this.taskExecutorResourceId = taskExecutorResourceId;
this.partitionRequestInitialBackoff = networkConfig.partitionRequestInitialBackoff();
this.partitionRequestMaxBackoff = networkConfig.partitionRequestMaxBackoff();
+ this.partitionRequestListenerTimeout = networkConfig.getPartitionRequestListenerTimeout();
this.maxRequiredBuffersPerGate = networkConfig.maxRequiredBuffersPerGate();
this.configuredNetworkBuffersPerChannel =
NettyShuffleUtils.getNetworkBuffersPerInputChannel(
@@ -307,6 +310,7 @@ private InputChannel createInputChannel(
connectionManager,
partitionRequestInitialBackoff,
partitionRequestMaxBackoff,
+ partitionRequestListenerTimeout,
buffersPerChannel,
metrics);
},
@@ -363,6 +367,7 @@ protected InputChannel createKnownInputChannel(
connectionManager,
partitionRequestInitialBackoff,
partitionRequestMaxBackoff,
+ partitionRequestListenerTimeout,
buffersPerChannel,
metrics);
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
index 3be987b770797..af3043f0f2d5f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java
@@ -56,6 +56,8 @@ class UnknownInputChannel extends InputChannel implements ChannelStateHolder {
private final int maxBackoff;
+ private final int partitionRequestListenerTimeout;
+
private final int networkBuffersPerChannel;
private final InputChannelMetrics metrics;
@@ -72,6 +74,7 @@ public UnknownInputChannel(
ConnectionManager connectionManager,
int initialBackoff,
int maxBackoff,
+ int partitionRequestListenerTimeout,
int networkBuffersPerChannel,
InputChannelMetrics metrics) {
@@ -91,6 +94,7 @@ public UnknownInputChannel(
this.metrics = checkNotNull(metrics);
this.initialBackoff = initialBackoff;
this.maxBackoff = maxBackoff;
+ this.partitionRequestListenerTimeout = partitionRequestListenerTimeout;
this.networkBuffersPerChannel = networkBuffersPerChannel;
}
@@ -168,6 +172,7 @@ public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) {
connectionManager,
initialBackoff,
maxBackoff,
+ partitionRequestListenerTimeout,
networkBuffersPerChannel,
metrics.getNumBytesInRemoteCounter(),
metrics.getNumBuffersInRemoteCounter(),
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java
index 25012ffa8f1cd..d4c0fed0a0b62 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImpl.java
@@ -160,7 +160,7 @@ private static void addInternalRegionToMap(
checkArgument(firstBufferInRegion.subpartitionId == lastBufferInRegion.subpartitionId);
checkArgument(firstBufferInRegion.bufferIndex <= lastBufferInRegion.bufferIndex);
internalRegionsBySubpartition
- .computeIfAbsent(firstBufferInRegion.subpartitionId, ArrayList::new)
+ .computeIfAbsent(firstBufferInRegion.subpartitionId, k -> new ArrayList<>())
.add(
new InternalRegion(
firstBufferInRegion.bufferIndex,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java
index 6d3a15d427a63..70a1c0b985e05 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategy.java
@@ -125,36 +125,48 @@ static class Builder {
private Builder() {}
public Builder addBufferToSpill(BufferIndexAndChannel buffer) {
- bufferToSpill.computeIfAbsent(buffer.getChannel(), ArrayList::new).add(buffer);
+ bufferToSpill
+ .computeIfAbsent(buffer.getChannel(), k -> new ArrayList<>())
+ .add(buffer);
return this;
}
public Builder addBufferToSpill(
int subpartitionId, List buffers) {
- bufferToSpill.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers);
+ bufferToSpill
+ .computeIfAbsent(subpartitionId, k -> new ArrayList<>())
+ .addAll(buffers);
return this;
}
public Builder addBufferToSpill(
int subpartitionId, Deque buffers) {
- bufferToSpill.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers);
+ bufferToSpill
+ .computeIfAbsent(subpartitionId, k -> new ArrayList<>())
+ .addAll(buffers);
return this;
}
public Builder addBufferToRelease(BufferIndexAndChannel buffer) {
- bufferToRelease.computeIfAbsent(buffer.getChannel(), ArrayList::new).add(buffer);
+ bufferToRelease
+ .computeIfAbsent(buffer.getChannel(), k -> new ArrayList<>())
+ .add(buffer);
return this;
}
public Builder addBufferToRelease(
int subpartitionId, List buffers) {
- bufferToRelease.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers);
+ bufferToRelease
+ .computeIfAbsent(subpartitionId, k -> new ArrayList<>())
+ .addAll(buffers);
return this;
}
public Builder addBufferToRelease(
int subpartitionId, Deque buffers) {
- bufferToRelease.computeIfAbsent(subpartitionId, ArrayList::new).addAll(buffers);
+ bufferToRelease
+ .computeIfAbsent(subpartitionId, k -> new ArrayList<>())
+ .addAll(buffers);
return this;
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java
index c166f4a1c4b46..5e3786daf0c9e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSpillingStrategyUtils.java
@@ -73,7 +73,7 @@ public class HsSpillingStrategyUtils {
BufferConsumptionPriorityIterator bufferConsumptionPriorityIterator = heap.poll();
BufferIndexAndChannel bufferIndexAndChannel = bufferConsumptionPriorityIterator.next();
subpartitionToHighPriorityBuffers
- .computeIfAbsent(bufferIndexAndChannel.getChannel(), ArrayList::new)
+ .computeIfAbsent(bufferIndexAndChannel.getChannel(), k -> new ArrayList<>())
.add(bufferIndexAndChannel);
// if this iterator has next, re-added it.
if (bufferConsumptionPriorityIterator.hasNext()) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java
index d6a08da5fb772..3046683f47535 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileIndex.java
@@ -168,7 +168,7 @@ private static void addRegionToMap(
checkArgument(firstBufferInRegion.getBufferIndex() <= lastBufferInRegion.getBufferIndex());
subpartitionRegionMap
- .computeIfAbsent(firstBufferInRegion.getSubpartitionId(), ArrayList::new)
+ .computeIfAbsent(firstBufferInRegion.getSubpartitionId(), k -> new ArrayList<>())
.add(
new FixedSizeRegion(
firstBufferInRegion.getBufferIndex(),
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java
index 81aa8d78ce3e3..27d0c72ed5c7b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSinkWriterMetricGroup.java
@@ -26,7 +26,6 @@
import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
import org.apache.flink.metrics.groups.OperatorMetricGroup;
import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
-import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.metrics.MetricNames;
/** Special {@link org.apache.flink.metrics.MetricGroup} representing an Operator. */
@@ -40,7 +39,8 @@ public class InternalSinkWriterMetricGroup extends ProxyMetricGroup
private final Counter numBytesWritten;
private final OperatorIOMetricGroup operatorIOMetricGroup;
- private InternalSinkWriterMetricGroup(
+ @VisibleForTesting
+ InternalSinkWriterMetricGroup(
MetricGroup parentMetricGroup, OperatorIOMetricGroup operatorIOMetricGroup) {
super(parentMetricGroup);
numRecordsOutErrors = parentMetricGroup.counter(MetricNames.NUM_RECORDS_OUT_ERRORS);
@@ -61,18 +61,6 @@ public static InternalSinkWriterMetricGroup wrap(OperatorMetricGroup operatorMet
operatorMetricGroup, operatorMetricGroup.getIOMetricGroup());
}
- @VisibleForTesting
- public static InternalSinkWriterMetricGroup mock(MetricGroup metricGroup) {
- return new InternalSinkWriterMetricGroup(
- metricGroup, UnregisteredMetricsGroup.createOperatorIOMetricGroup());
- }
-
- @VisibleForTesting
- public static InternalSinkWriterMetricGroup mock(
- MetricGroup metricGroup, OperatorIOMetricGroup operatorIOMetricGroup) {
- return new InternalSinkWriterMetricGroup(metricGroup, operatorIOMetricGroup);
- }
-
@Override
public OperatorIOMetricGroup getIOMetricGroup() {
return operatorIOMetricGroup;
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java
index 520a54ec6035b..0074dfbe07e58 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java
@@ -24,6 +24,7 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.MetricOptions;
import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MeterView;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
@@ -129,7 +130,8 @@ public static void instantiateStatusMetrics(MetricGroup metricGroup) {
MetricGroup jvm = metricGroup.addGroup("JVM");
instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader"));
- instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector"));
+ instantiateGarbageCollectorMetrics(
+ jvm.addGroup("GarbageCollector"), ManagementFactory.getGarbageCollectorMXBeans());
instantiateMemoryMetrics(jvm.addGroup(METRIC_GROUP_MEMORY));
instantiateThreadMetrics(jvm.addGroup("Threads"));
instantiateCPUMetrics(jvm.addGroup("CPU"));
@@ -222,16 +224,32 @@ private static void instantiateClassLoaderMetrics(MetricGroup metrics) {
metrics.>gauge("ClassesUnloaded", mxBean::getUnloadedClassCount);
}
- private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) {
- List garbageCollectors =
- ManagementFactory.getGarbageCollectorMXBeans();
-
+ @VisibleForTesting
+ static void instantiateGarbageCollectorMetrics(
+ MetricGroup metrics, List garbageCollectors) {
for (final GarbageCollectorMXBean garbageCollector : garbageCollectors) {
MetricGroup gcGroup = metrics.addGroup(garbageCollector.getName());
- gcGroup.>gauge("Count", garbageCollector::getCollectionCount);
- gcGroup.>gauge("Time", garbageCollector::getCollectionTime);
+ gcGroup.gauge("Count", garbageCollector::getCollectionCount);
+ Gauge timeGauge = gcGroup.gauge("Time", garbageCollector::getCollectionTime);
+ gcGroup.meter("TimeMsPerSecond", new MeterView(timeGauge));
}
+ Gauge totalGcTime =
+ () ->
+ garbageCollectors.stream()
+ .mapToLong(GarbageCollectorMXBean::getCollectionTime)
+ .sum();
+
+ Gauge totalGcCount =
+ () ->
+ garbageCollectors.stream()
+ .mapToLong(GarbageCollectorMXBean::getCollectionCount)
+ .sum();
+
+ MetricGroup allGroup = metrics.addGroup("All");
+ allGroup.gauge("Count", totalGcCount);
+ Gauge totalTime = allGroup.gauge("Time", totalGcTime);
+ allGroup.meter("TimeMsPerSecond", new MeterView(totalTime));
}
private static void instantiateMemoryMetrics(MetricGroup metrics) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java
index 1a18364c2077a..1368748fb9c6f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java
@@ -90,7 +90,7 @@ private static SlotManager createSlotManager(
SlotManagerUtils.generateTaskManagerTotalResourceProfile(
slotManagerConfiguration.getDefaultWorkerResourceSpec()),
slotManagerConfiguration.getNumSlotsPerWorker(),
- slotManagerConfiguration.isEvenlySpreadOutSlots(),
+ slotManagerConfiguration.getTaskManagerLoadBalanceMode(),
slotManagerConfiguration.getTaskManagerTimeout(),
slotManagerConfiguration.getRedundantTaskManagerNum(),
slotManagerConfiguration.getMinTotalCpu(),
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
index 9f12c46972ce5..62f31480b4abc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultResourceAllocationStrategy.java
@@ -40,6 +40,7 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode;
import static org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerUtils.getEffectiveResourceProfile;
/**
@@ -84,7 +85,7 @@ public class DefaultResourceAllocationStrategy implements ResourceAllocationStra
public DefaultResourceAllocationStrategy(
ResourceProfile totalResourceProfile,
int numSlotsPerWorker,
- boolean evenlySpreadOutSlots,
+ TaskManagerLoadBalanceMode taskManagerLoadBalanceMode,
Time taskManagerTimeout,
int redundantTaskManagerNum,
CPUResource minTotalCPU,
@@ -95,7 +96,7 @@ public DefaultResourceAllocationStrategy(
SlotManagerUtils.generateDefaultSlotResourceProfile(
totalResourceProfile, numSlotsPerWorker);
this.availableResourceMatchingStrategy =
- evenlySpreadOutSlots
+ taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS
? LeastUtilizationResourceMatchingStrategy.INSTANCE
: AnyMatchingResourceMatchingStrategy.INSTANCE;
this.taskManagerTimeout = taskManagerTimeout;
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
index 330a65b783470..a41b0c16ef528 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java
@@ -21,7 +21,6 @@
import org.apache.flink.api.common.resources.CPUResource;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.AkkaOptions;
-import org.apache.flink.configuration.ClusterOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.configuration.ResourceManagerOptions;
@@ -33,6 +32,8 @@
import java.math.RoundingMode;
import java.time.Duration;
+import static org.apache.flink.configuration.TaskManagerOptions.TaskManagerLoadBalanceMode;
+
/** Configuration for the {@link SlotManager}. */
public class SlotManagerConfiguration {
private final Time taskManagerRequestTimeout;
@@ -41,7 +42,7 @@ public class SlotManagerConfiguration {
private final Duration declareNeededResourceDelay;
private final boolean waitResultConsumedBeforeRelease;
private final SlotMatchingStrategy slotMatchingStrategy;
- private final boolean evenlySpreadOutSlots;
+ private final TaskManagerLoadBalanceMode taskManagerLoadBalanceMode;
private final WorkerResourceSpec defaultWorkerResourceSpec;
private final int numSlotsPerWorker;
private final int minSlotNum;
@@ -59,7 +60,7 @@ public SlotManagerConfiguration(
Duration declareNeededResourceDelay,
boolean waitResultConsumedBeforeRelease,
SlotMatchingStrategy slotMatchingStrategy,
- boolean evenlySpreadOutSlots,
+ TaskManagerLoadBalanceMode taskManagerLoadBalanceMode,
WorkerResourceSpec defaultWorkerResourceSpec,
int numSlotsPerWorker,
int minSlotNum,
@@ -76,7 +77,7 @@ public SlotManagerConfiguration(
this.declareNeededResourceDelay = Preconditions.checkNotNull(declareNeededResourceDelay);
this.waitResultConsumedBeforeRelease = waitResultConsumedBeforeRelease;
this.slotMatchingStrategy = Preconditions.checkNotNull(slotMatchingStrategy);
- this.evenlySpreadOutSlots = evenlySpreadOutSlots;
+ this.taskManagerLoadBalanceMode = taskManagerLoadBalanceMode;
this.defaultWorkerResourceSpec = Preconditions.checkNotNull(defaultWorkerResourceSpec);
Preconditions.checkState(numSlotsPerWorker > 0);
this.numSlotsPerWorker = numSlotsPerWorker;
@@ -199,8 +200,8 @@ public SlotMatchingStrategy getSlotMatchingStrategy() {
return slotMatchingStrategy;
}
- public boolean isEvenlySpreadOutSlots() {
- return evenlySpreadOutSlots;
+ public TaskManagerLoadBalanceMode getTaskManagerLoadBalanceMode() {
+ return taskManagerLoadBalanceMode;
}
public WorkerResourceSpec getDefaultWorkerResourceSpec() {
@@ -260,10 +261,10 @@ public static SlotManagerConfiguration fromConfiguration(
configuration.getBoolean(
ResourceManagerOptions.TASK_MANAGER_RELEASE_WHEN_RESULT_CONSUMED);
- boolean evenlySpreadOutSlots =
- configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY);
+ TaskManagerLoadBalanceMode taskManagerLoadBalanceMode =
+ TaskManagerLoadBalanceMode.loadFromConfiguration(configuration);
final SlotMatchingStrategy slotMatchingStrategy =
- evenlySpreadOutSlots
+ taskManagerLoadBalanceMode == TaskManagerLoadBalanceMode.SLOTS
? LeastUtilizationSlotMatchingStrategy.INSTANCE
: AnyMatchingSlotMatchingStrategy.INSTANCE;
@@ -282,7 +283,7 @@ public static SlotManagerConfiguration fromConfiguration(
declareNeededResourceDelay,
waitResultConsumedBeforeRelease,
slotMatchingStrategy,
- evenlySpreadOutSlots,
+ taskManagerLoadBalanceMode,
defaultWorkerResourceSpec,
numSlotsPerWorker,
minSlotNum,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java
index 5ece82a267173..55c7875e85cf4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java
@@ -242,6 +242,7 @@ private static JobExceptionsInfoWithHistory.RootExceptionInfo createRootExceptio
historyEntry.getFailureLabels(),
historyEntry.getFailingTaskName(),
toString(historyEntry.getTaskManagerLocation()),
+ toString(historyEntry.getTaskManagerLocation()),
toTaskManagerId(historyEntry.getTaskManagerLocation()),
concurrentExceptions);
}
@@ -257,6 +258,7 @@ private static JobExceptionsInfoWithHistory.ExceptionInfo createExceptionInfo(
exceptionHistoryEntry.getFailureLabels(),
exceptionHistoryEntry.getFailingTaskName(),
toString(exceptionHistoryEntry.getTaskManagerLocation()),
+ toString(exceptionHistoryEntry.getTaskManagerLocation()),
toTaskManagerId(exceptionHistoryEntry.getTaskManagerLocation()));
}
@@ -270,9 +272,7 @@ private static void assertLocalExceptionInfo(ExceptionHistoryEntry exceptionHist
static String toString(@Nullable TaskManagerLocation location) {
// '(unassigned)' being the default value is added to support backward-compatibility for the
// deprecated fields
- return location != null
- ? taskManagerLocationToString(location.getFQDNHostname(), location.dataPort())
- : "(unassigned)";
+ return location != null ? location.getEndpoint() : "(unassigned)";
}
@VisibleForTesting
@@ -285,9 +285,7 @@ static String toTaskManagerId(@Nullable TaskManagerLocation location) {
@VisibleForTesting
@Nullable
static String toString(@Nullable ExceptionHistoryEntry.ArchivedTaskManagerLocation location) {
- return location != null
- ? taskManagerLocationToString(location.getFQDNHostname(), location.getPort())
- : null;
+ return location != null ? location.getEndpoint() : null;
}
@VisibleForTesting
@@ -295,8 +293,4 @@ static String toTaskManagerId(
@Nullable ExceptionHistoryEntry.ArchivedTaskManagerLocation location) {
return location != null ? String.format("%s", location.getResourceID()) : null;
}
-
- private static String taskManagerLocationToString(String fqdnHostname, int port) {
- return String.format("%s:%d", fqdnHostname, port);
- }
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java
index 0ed6565ac2452..948bdcb1de8f5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java
@@ -132,8 +132,7 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo(
JobID jobID,
@Nullable MetricFetcher metricFetcher) {
// Build a map that groups task executions by TaskManager
- Map taskManagerId2Host = new HashMap<>();
- Map> taskManagerExecutions = new HashMap<>();
+ Map> taskManagerExecutions = new HashMap<>();
Set representativeExecutions = new HashSet<>();
for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
AccessExecution representativeAttempt = vertex.getCurrentExecutionAttempt();
@@ -141,16 +140,9 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo(
for (AccessExecution execution : vertex.getCurrentExecutions()) {
TaskManagerLocation location = execution.getAssignedResourceLocation();
- String taskManagerHost =
- location == null
- ? "(unassigned)"
- : location.getHostname() + ':' + location.dataPort();
- String taskmanagerId =
- location == null ? "(unassigned)" : location.getResourceID().toString();
- taskManagerId2Host.put(taskmanagerId, taskManagerHost);
List executions =
taskManagerExecutions.computeIfAbsent(
- taskmanagerId, ignored -> new ArrayList<>());
+ location, ignored -> new ArrayList<>());
executions.add(execution);
}
}
@@ -158,9 +150,17 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo(
final long now = System.currentTimeMillis();
List taskManagersInfoList = new ArrayList<>(4);
- for (Map.Entry> entry : taskManagerExecutions.entrySet()) {
- String taskmanagerId = entry.getKey();
- String host = taskManagerId2Host.get(taskmanagerId);
+ for (Map.Entry> entry :
+ taskManagerExecutions.entrySet()) {
+ TaskManagerLocation location = entry.getKey();
+ // Port information is included in the host field for backward-compatibility
+ String host =
+ location == null
+ ? "(unassigned)"
+ : location.getHostname() + ':' + location.dataPort();
+ String endpoint = location == null ? "(unassigned)" : location.getEndpoint();
+ String taskmanagerId =
+ location == null ? "(unassigned)" : location.getResourceID().toString();
List executions = entry.getValue();
List ioMetricsInfos = new ArrayList<>();
@@ -266,6 +266,7 @@ private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo(
taskManagersInfoList.add(
new JobVertexTaskManagersInfo.TaskManagersInfo(
host,
+ endpoint,
jobVertexState,
startTime,
endTime,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java
index a47ad6a46be44..4db45a2f3e418 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksAllAccumulatorsHandler.java
@@ -78,7 +78,8 @@ protected SubtasksAllAccumulatorsInfo handleRequest(
for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) {
for (AccessExecution execution : vertex.getCurrentExecutions()) {
TaskManagerLocation location = execution.getAssignedResourceLocation();
- String locationString = location == null ? "(unassigned)" : location.getHostname();
+ String host = location == null ? "(unassigned)" : location.getHostname();
+ String endpoint = location == null ? "(unassigned)" : location.getEndpoint();
StringifiedAccumulatorResult[] accs = execution.getUserAccumulatorsStringified();
List userAccumulators = new ArrayList<>(accs.length);
@@ -91,7 +92,8 @@ protected SubtasksAllAccumulatorsInfo handleRequest(
new SubtasksAllAccumulatorsInfo.SubtaskAccumulatorsInfo(
execution.getParallelSubtaskIndex(),
execution.getAttemptNumber(),
- locationString,
+ host,
+ endpoint,
userAccumulators));
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java
index f097bbfbb4530..34e3fed0673ac 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/SubtasksTimesHandler.java
@@ -113,7 +113,8 @@ private static SubtasksTimesInfo createSubtaskTimesInfo(AccessExecutionJobVertex
long duration = start >= 0 ? end - start : -1L;
TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation();
- String locationString = location == null ? "(unassigned)" : location.getHostname();
+ String host = location == null ? "(unassigned)" : location.getHostname();
+ String endpoint = location == null ? "(unassigned)" : location.getEndpoint();
Map timestampMap =
CollectionUtil.newHashMapWithExpectedSize(ExecutionState.values().length);
@@ -123,7 +124,7 @@ private static SubtasksTimesInfo createSubtaskTimesInfo(AccessExecutionJobVertex
subtasks.add(
new SubtasksTimesInfo.SubtaskTimeInfo(
- num++, locationString, duration, timestampMap));
+ num++, host, endpoint, duration, timestampMap));
}
return new SubtasksTimesInfo(id, name, now, subtasks);
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java
index 2bd3fd46bfc39..0acc64d4b47dc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java
@@ -144,7 +144,8 @@ public boolean isTruncated() {
public static final class ExecutionExceptionInfo {
public static final String FIELD_NAME_EXCEPTION = "exception";
public static final String FIELD_NAME_TASK = "task";
- public static final String FIELD_NAME_LOCATION = "location";
+ @Deprecated public static final String FIELD_NAME_LOCATION = "location";
+ public static final String FIELD_NAME_ENDPOINT = "endpoint";
public static final String FIELD_NAME_TIMESTAMP = "timestamp";
public static final String FIELD_NAME_TASK_MANAGER_ID = "taskManagerId";
@@ -157,22 +158,36 @@ public static final class ExecutionExceptionInfo {
@JsonProperty(FIELD_NAME_LOCATION)
private final String location;
+ @JsonProperty(FIELD_NAME_ENDPOINT)
+ private final String endpoint;
+
@JsonProperty(FIELD_NAME_TIMESTAMP)
private final long timestamp;
@JsonProperty(FIELD_NAME_TASK_MANAGER_ID)
private final String taskManagerId;
+ public ExecutionExceptionInfo(
+ String exception,
+ String task,
+ String endpoint,
+ long timestamp,
+ String taskManagerId) {
+ this(exception, task, endpoint, endpoint, timestamp, taskManagerId);
+ }
+
@JsonCreator
public ExecutionExceptionInfo(
@JsonProperty(FIELD_NAME_EXCEPTION) String exception,
@JsonProperty(FIELD_NAME_TASK) String task,
@JsonProperty(FIELD_NAME_LOCATION) String location,
+ @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint,
@JsonProperty(FIELD_NAME_TIMESTAMP) long timestamp,
@JsonProperty(FIELD_NAME_TASK_MANAGER_ID) String taskManagerId) {
this.exception = Preconditions.checkNotNull(exception);
this.task = Preconditions.checkNotNull(task);
this.location = Preconditions.checkNotNull(location);
+ this.endpoint = Preconditions.checkNotNull(endpoint);
this.timestamp = timestamp;
this.taskManagerId = taskManagerId;
}
@@ -191,12 +206,13 @@ public boolean equals(Object o) {
&& Objects.equals(exception, that.exception)
&& Objects.equals(task, that.task)
&& Objects.equals(location, that.location)
+ && Objects.equals(endpoint, that.endpoint)
&& Objects.equals(taskManagerId, that.taskManagerId);
}
@Override
public int hashCode() {
- return Objects.hash(timestamp, exception, task, location, taskManagerId);
+ return Objects.hash(timestamp, exception, task, location, endpoint, taskManagerId);
}
@Override
@@ -205,6 +221,7 @@ public String toString() {
.add("exception='" + exception + "'")
.add("task='" + task + "'")
.add("location='" + location + "'")
+ .add("endpoint='" + endpoint + "'")
.add("timestamp=" + timestamp)
.add("taskManagerId=" + taskManagerId)
.toString();
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java
index dd6b264386e9b..cb15825cc6bd6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfoWithHistory.java
@@ -176,7 +176,8 @@ public static class ExceptionInfo {
public static final String FIELD_NAME_EXCEPTION_STACKTRACE = "stacktrace";
public static final String FIELD_NAME_EXCEPTION_TIMESTAMP = "timestamp";
public static final String FIELD_NAME_TASK_NAME = "taskName";
- public static final String FIELD_NAME_LOCATION = "location";
+ @Deprecated public static final String FIELD_NAME_LOCATION = "location";
+ public static final String FIELD_NAME_ENDPOINT = "endpoint";
public static final String FIELD_NAME_TASK_MANAGER_ID = "taskManagerId";
public static final String FIELD_NAME_FAILURE_LABELS = "failureLabels";
@@ -194,11 +195,18 @@ public static class ExceptionInfo {
@Nullable
private final String taskName;
+ /** @deprecated Use {@link ExceptionInfo#endpoint} instead. */
+ @Deprecated
@JsonInclude(NON_NULL)
@JsonProperty(FIELD_NAME_LOCATION)
@Nullable
private final String location;
+ @JsonInclude(NON_NULL)
+ @JsonProperty(FIELD_NAME_ENDPOINT)
+ @Nullable
+ private final String endpoint;
+
@JsonInclude(NON_NULL)
@JsonProperty(FIELD_NAME_TASK_MANAGER_ID)
@Nullable
@@ -208,7 +216,15 @@ public static class ExceptionInfo {
private final Map failureLabels;
public ExceptionInfo(String exceptionName, String stacktrace, long timestamp) {
- this(exceptionName, stacktrace, timestamp, Collections.emptyMap(), null, null, null);
+ this(
+ exceptionName,
+ stacktrace,
+ timestamp,
+ Collections.emptyMap(),
+ null,
+ null,
+ null,
+ null);
}
@JsonCreator
@@ -219,6 +235,7 @@ public ExceptionInfo(
@JsonProperty(FIELD_NAME_FAILURE_LABELS) Map failureLabels,
@JsonProperty(FIELD_NAME_TASK_NAME) @Nullable String taskName,
@JsonProperty(FIELD_NAME_LOCATION) @Nullable String location,
+ @JsonProperty(FIELD_NAME_ENDPOINT) @Nullable String endpoint,
@JsonProperty(FIELD_NAME_TASK_MANAGER_ID) @Nullable String taskManagerId) {
this.exceptionName = checkNotNull(exceptionName);
this.stacktrace = checkNotNull(stacktrace);
@@ -226,6 +243,7 @@ public ExceptionInfo(
this.failureLabels = checkNotNull(failureLabels);
this.taskName = taskName;
this.location = location;
+ this.endpoint = endpoint;
this.taskManagerId = taskManagerId;
}
@@ -250,12 +268,19 @@ public String getTaskName() {
return taskName;
}
+ @Deprecated
@JsonIgnore
@Nullable
public String getLocation() {
return location;
}
+ @JsonIgnore
+ @Nullable
+ public String getEndpoint() {
+ return endpoint;
+ }
+
@JsonIgnore
@Nullable
public String getTaskManagerId() {
@@ -283,13 +308,20 @@ public boolean equals(Object o) {
&& Objects.equals(timestamp, that.timestamp)
&& Objects.equals(failureLabels, that.failureLabels)
&& Objects.equals(taskName, that.taskName)
- && Objects.equals(location, that.location);
+ && Objects.equals(location, that.location)
+ && Objects.equals(location, that.endpoint);
}
@Override
public int hashCode() {
return Objects.hash(
- exceptionName, stacktrace, timestamp, failureLabels, taskName, location);
+ exceptionName,
+ stacktrace,
+ timestamp,
+ failureLabels,
+ taskName,
+ location,
+ endpoint);
}
@Override
@@ -300,7 +332,7 @@ public String toString() {
.add("timestamp=" + timestamp)
.add("failureLabels=" + failureLabels)
.add("taskName='" + taskName + "'")
- .add("location='" + location + "'")
+ .add("endpoint='" + endpoint + "'")
.toString();
}
}
@@ -330,6 +362,7 @@ public RootExceptionInfo(
null,
null,
null,
+ null,
concurrentExceptions);
}
@@ -341,6 +374,7 @@ public RootExceptionInfo(
@JsonProperty(FIELD_NAME_FAILURE_LABELS) Map failureLabels,
@JsonProperty(FIELD_NAME_TASK_NAME) @Nullable String taskName,
@JsonProperty(FIELD_NAME_LOCATION) @Nullable String location,
+ @JsonProperty(FIELD_NAME_ENDPOINT) @Nullable String endpoint,
@JsonProperty(FIELD_NAME_TASK_MANAGER_ID) @Nullable String taskManagerId,
@JsonProperty(FIELD_NAME_CONCURRENT_EXCEPTIONS)
Collection concurrentExceptions) {
@@ -351,6 +385,7 @@ public RootExceptionInfo(
failureLabels,
taskName,
location,
+ endpoint,
taskManagerId);
this.concurrentExceptions = concurrentExceptions;
}
@@ -386,7 +421,7 @@ public String toString() {
.add("stacktrace='" + getStacktrace() + "'")
.add("timestamp=" + getTimestamp())
.add("taskName='" + getTaskName() + "'")
- .add("location='" + getLocation() + "'")
+ .add("endpoint='" + getEndpoint() + "'")
.add("concurrentExceptions=" + getConcurrentExceptions())
.toString();
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java
index de6a388292779..c0b2e34dfe2b7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java
@@ -100,7 +100,8 @@ public int hashCode() {
/** Detailed information about task managers. */
@Schema(name = "JobVertexTaskManagerInfo")
public static class TaskManagersInfo {
- public static final String TASK_MANAGERS_FIELD_HOST = "host";
+ @Deprecated public static final String TASK_MANAGERS_FIELD_HOST = "host";
+ public static final String TASK_MANAGERS_FIELD_ENDPOINT = "endpoint";
public static final String TASK_MANAGERS_FIELD_STATUS = "status";
public static final String TASK_MANAGERS_FIELD_START_TIME = "start-time";
public static final String TASK_MANAGERS_FIELD_END_TIME = "end-time";
@@ -113,6 +114,9 @@ public static class TaskManagersInfo {
@JsonProperty(TASK_MANAGERS_FIELD_HOST)
private final String host;
+ @JsonProperty(TASK_MANAGERS_FIELD_ENDPOINT)
+ private final String endpoint;
+
@JsonProperty(TASK_MANAGERS_FIELD_STATUS)
private final ExecutionState status;
@@ -140,6 +144,7 @@ public static class TaskManagersInfo {
@JsonCreator
public TaskManagersInfo(
@JsonProperty(TASK_MANAGERS_FIELD_HOST) String host,
+ @JsonProperty(TASK_MANAGERS_FIELD_ENDPOINT) String endpoint,
@JsonProperty(TASK_MANAGERS_FIELD_STATUS) ExecutionState status,
@JsonProperty(TASK_MANAGERS_FIELD_START_TIME) long startTime,
@JsonProperty(TASK_MANAGERS_FIELD_END_TIME) long endTime,
@@ -151,6 +156,7 @@ public TaskManagersInfo(
@JsonProperty(TASK_MANAGERS_FIELD_AGGREGATED)
AggregatedTaskDetailsInfo aggregated) {
this.host = checkNotNull(host);
+ this.endpoint = checkNotNull(endpoint);
this.status = checkNotNull(status);
this.startTime = startTime;
this.endTime = endTime;
@@ -171,6 +177,7 @@ public boolean equals(Object o) {
}
TaskManagersInfo that = (TaskManagersInfo) o;
return Objects.equals(host, that.host)
+ && Objects.equals(endpoint, that.endpoint)
&& Objects.equals(status, that.status)
&& startTime == that.startTime
&& endTime == that.endTime
@@ -185,6 +192,7 @@ public boolean equals(Object o) {
public int hashCode() {
return Objects.hash(
host,
+ endpoint,
status,
startTime,
endTime,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java
index 63c540834934f..122429e3c0d49 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java
@@ -78,12 +78,15 @@ default Collection> getResponseTypeParameters() {
* @return short description
*/
default String operationId() {
+ final String className = getClass().getSimpleName();
+
if (getHttpMethod() != HttpMethodWrapper.GET) {
throw new UnsupportedOperationException(
- "The default implementation is only supported for GET calls. Please override 'operationId()'.");
+ "The default implementation is only supported for GET calls. Please override 'operationId()' in '"
+ + className
+ + "'.");
}
- final String className = getClass().getSimpleName();
final int headersSuffixStart = className.lastIndexOf("Headers");
if (headersSuffixStart == -1) {
throw new IllegalStateException(
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java
index 7da481f2ad9f8..67cf50778e8cb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesInfo.java
@@ -92,7 +92,8 @@ public int hashCode() {
public static final class SubtaskTimeInfo {
public static final String FIELD_NAME_SUBTASK = "subtask";
- public static final String FIELD_NAME_HOST = "host";
+ @Deprecated public static final String FIELD_NAME_HOST = "host";
+ public static final String FIELD_NAME_ENDPOINT = "endpoint";
public static final String FIELD_NAME_DURATION = "duration";
public static final String FIELD_NAME_TIMESTAMPS = "timestamps";
@@ -102,6 +103,9 @@ public static final class SubtaskTimeInfo {
@JsonProperty(FIELD_NAME_HOST)
private final String host;
+ @JsonProperty(FIELD_NAME_ENDPOINT)
+ private final String endpoint;
+
@JsonProperty(FIELD_NAME_DURATION)
private final long duration;
@@ -111,10 +115,12 @@ public static final class SubtaskTimeInfo {
public SubtaskTimeInfo(
@JsonProperty(FIELD_NAME_SUBTASK) int subtask,
@JsonProperty(FIELD_NAME_HOST) String host,
+ @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint,
@JsonProperty(FIELD_NAME_DURATION) long duration,
@JsonProperty(FIELD_NAME_TIMESTAMPS) Map timestamps) {
this.subtask = subtask;
this.host = checkNotNull(host);
+ this.endpoint = checkNotNull(endpoint);
this.duration = duration;
this.timestamps = checkNotNull(timestamps);
}
@@ -132,13 +138,14 @@ public boolean equals(Object o) {
SubtaskTimeInfo that = (SubtaskTimeInfo) o;
return subtask == that.subtask
&& Objects.equals(host, that.host)
+ && Objects.equals(endpoint, that.endpoint)
&& duration == that.duration
&& Objects.equals(timestamps, that.timestamps);
}
@Override
public int hashCode() {
- return Objects.hash(subtask, host, duration, timestamps);
+ return Objects.hash(subtask, host, endpoint, duration, timestamps);
}
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java
index fd8ed6f087be5..b38f57c987896 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java
@@ -41,6 +41,9 @@
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import io.swagger.v3.oas.annotations.media.DiscriminatorMapping;
+import io.swagger.v3.oas.annotations.media.Schema;
+
import javax.annotation.Nullable;
import java.util.Collection;
@@ -64,6 +67,19 @@
value = CheckpointStatistics.PendingCheckpointStatistics.class,
name = "in_progress")
})
+@Schema(
+ discriminatorProperty = "className",
+ discriminatorMapping = {
+ @DiscriminatorMapping(
+ value = "completed",
+ schema = CheckpointStatistics.CompletedCheckpointStatistics.class),
+ @DiscriminatorMapping(
+ value = "failed",
+ schema = CheckpointStatistics.FailedCheckpointStatistics.class),
+ @DiscriminatorMapping(
+ value = "in_progress",
+ schema = CheckpointStatistics.PendingCheckpointStatistics.class),
+ })
public class CheckpointStatistics implements ResponseBody {
public static final String FIELD_NAME_ID = "id";
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java
index 642509a965c7e..613916eb3d762 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/SubtaskCheckpointStatistics.java
@@ -23,6 +23,9 @@
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+import io.swagger.v3.oas.annotations.media.DiscriminatorMapping;
+import io.swagger.v3.oas.annotations.media.Schema;
+
import java.util.Objects;
/** Checkpoint statistics for a subtask. */
@@ -38,6 +41,17 @@
value = SubtaskCheckpointStatistics.PendingSubtaskCheckpointStatistics.class,
name = "pending")
})
+@Schema(
+ discriminatorProperty = "className",
+ discriminatorMapping = {
+ @DiscriminatorMapping(
+ value = "completed",
+ schema =
+ SubtaskCheckpointStatistics.CompletedSubtaskCheckpointStatistics.class),
+ @DiscriminatorMapping(
+ value = "pending",
+ schema = SubtaskCheckpointStatistics.PendingSubtaskCheckpointStatistics.class),
+ })
public class SubtaskCheckpointStatistics {
public static final String FIELD_NAME_INDEX = "index";
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java
index 1a8463ab5597c..df0fed46d0b94 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsInfo.java
@@ -54,7 +54,9 @@ public class SubtaskExecutionAttemptDetailsInfo implements ResponseBody {
public static final String FIELD_NAME_ATTEMPT = "attempt";
- public static final String FIELD_NAME_HOST = "host";
+ @Deprecated public static final String FIELD_NAME_HOST = "host";
+
+ public static final String FIELD_NAME_ENDPOINT = "endpoint";
public static final String FIELD_NAME_START_TIME = "start-time";
@@ -84,6 +86,9 @@ public class SubtaskExecutionAttemptDetailsInfo implements ResponseBody {
@JsonProperty(FIELD_NAME_HOST)
private final String host;
+ @JsonProperty(FIELD_NAME_ENDPOINT)
+ private final String endpoint;
+
@JsonProperty(FIELD_NAME_START_TIME)
private final long startTime;
@@ -118,6 +123,7 @@ public SubtaskExecutionAttemptDetailsInfo(
@JsonProperty(FIELD_NAME_STATUS) ExecutionState status,
@JsonProperty(FIELD_NAME_ATTEMPT) int attempt,
@JsonProperty(FIELD_NAME_HOST) String host,
+ @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint,
@JsonProperty(FIELD_NAME_START_TIME) long startTime,
@JsonProperty(FIELD_NAME_END_TIME) long endTime,
@JsonProperty(FIELD_NAME_DURATION) long duration,
@@ -131,6 +137,7 @@ public SubtaskExecutionAttemptDetailsInfo(
this.status = Preconditions.checkNotNull(status);
this.attempt = attempt;
this.host = Preconditions.checkNotNull(host);
+ this.endpoint = Preconditions.checkNotNull(endpoint);
this.startTime = startTime;
this.startTimeCompatible = startTime;
this.endTime = endTime;
@@ -153,10 +160,15 @@ public int getAttempt() {
return attempt;
}
+ @Deprecated
public String getHost() {
return host;
}
+ public String getEndpoint() {
+ return endpoint;
+ }
+
public long getStartTime() {
return startTime;
}
@@ -203,7 +215,8 @@ public static SubtaskExecutionAttemptDetailsInfo create(
final long now = System.currentTimeMillis();
final TaskManagerLocation location = execution.getAssignedResourceLocation();
- final String locationString = location == null ? "(unassigned)" : location.getHostname();
+ final String host = location == null ? "(unassigned)" : location.getHostname();
+ final String endpoint = location == null ? "(unassigned)" : location.getEndpoint();
String taskmanagerId =
location == null ? "(unassigned)" : location.getResourceID().toString();
@@ -235,7 +248,8 @@ public static SubtaskExecutionAttemptDetailsInfo create(
execution.getParallelSubtaskIndex(),
status,
execution.getAttemptNumber(),
- locationString,
+ host,
+ endpoint,
startTime,
endTime,
duration,
@@ -260,6 +274,7 @@ public boolean equals(Object o) {
&& status == that.status
&& attempt == that.attempt
&& Objects.equals(host, that.host)
+ && Objects.equals(endpoint, that.endpoint)
&& startTime == that.startTime
&& startTimeCompatible == that.startTimeCompatible
&& endTime == that.endTime
@@ -277,6 +292,7 @@ public int hashCode() {
status,
attempt,
host,
+ endpoint,
startTime,
startTimeCompatible,
endTime,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java
index e245a19388b53..8035ff2a35512 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtasksAllAccumulatorsInfo.java
@@ -90,7 +90,8 @@ public int hashCode() {
public static class SubtaskAccumulatorsInfo {
public static final String FIELD_NAME_SUBTASK_INDEX = "subtask";
public static final String FIELD_NAME_ATTEMPT_NUM = "attempt";
- public static final String FIELD_NAME_HOST = "host";
+ @Deprecated public static final String FIELD_NAME_HOST = "host";
+ public static final String FIELD_NAME_ENDPOINT = "endpoint";
public static final String FIELD_NAME_USER_ACCUMULATORS = "user-accumulators";
@JsonProperty(FIELD_NAME_SUBTASK_INDEX)
@@ -102,6 +103,9 @@ public static class SubtaskAccumulatorsInfo {
@JsonProperty(FIELD_NAME_HOST)
private final String host;
+ @JsonProperty(FIELD_NAME_ENDPOINT)
+ private final String endpoint;
+
@JsonProperty(FIELD_NAME_USER_ACCUMULATORS)
private final Collection userAccumulators;
@@ -110,12 +114,14 @@ public SubtaskAccumulatorsInfo(
@JsonProperty(FIELD_NAME_SUBTASK_INDEX) int subtaskIndex,
@JsonProperty(FIELD_NAME_ATTEMPT_NUM) int attemptNum,
@JsonProperty(FIELD_NAME_HOST) String host,
+ @JsonProperty(FIELD_NAME_ENDPOINT) String endpoint,
@JsonProperty(FIELD_NAME_USER_ACCUMULATORS)
Collection userAccumulators) {
this.subtaskIndex = subtaskIndex;
this.attemptNum = attemptNum;
this.host = Preconditions.checkNotNull(host);
+ this.endpoint = Preconditions.checkNotNull(endpoint);
this.userAccumulators = Preconditions.checkNotNull(userAccumulators);
}
@@ -131,12 +137,13 @@ public boolean equals(Object o) {
return subtaskIndex == that.subtaskIndex
&& attemptNum == that.attemptNum
&& Objects.equals(host, that.host)
+ && Objects.equals(endpoint, that.endpoint)
&& Objects.equals(userAccumulators, that.userAccumulators);
}
@Override
public int hashCode() {
- return Objects.hash(subtaskIndex, attemptNum, host, userAccumulators);
+ return Objects.hash(subtaskIndex, attemptNum, host, endpoint, userAccumulators);
}
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java
index 25c623e18461e..a3b043eb987e5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java
@@ -21,6 +21,7 @@
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.core.failure.FailureEnricher;
@@ -122,6 +123,11 @@ public SchedulerNG createInstance(
shuffleMaster,
partitionTracker);
+ final CheckpointsCleaner checkpointsCleaner =
+ new CheckpointsCleaner(
+ jobMasterConfiguration.getBoolean(
+ CheckpointingOptions.CLEANER_PARALLEL_MODE));
+
return new DefaultScheduler(
log,
jobGraph,
@@ -130,7 +136,7 @@ public SchedulerNG createInstance(
schedulerComponents.getStartUpAction(),
new ScheduledExecutorServiceAdapter(futureExecutor),
userCodeLoader,
- new CheckpointsCleaner(),
+ checkpointsCleaner,
checkpointRecoveryFactory,
jobManagerJobMetricGroup,
schedulerComponents.getSchedulingStrategyFactory(),
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java
index 4ee22c9584840..67615ea79cc29 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java
@@ -108,6 +108,7 @@
import org.apache.flink.runtime.scheduler.adaptive.allocator.SlotAllocator;
import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism;
import org.apache.flink.runtime.scheduler.adaptive.scalingpolicy.EnforceMinimalIncreaseRescalingController;
+import org.apache.flink.runtime.scheduler.adaptive.scalingpolicy.EnforceParallelismChangeRescalingController;
import org.apache.flink.runtime.scheduler.adaptive.scalingpolicy.RescalingController;
import org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntry;
import org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry;
@@ -204,6 +205,8 @@ public class AdaptiveScheduler
private final RescalingController rescalingController;
+ private final RescalingController forceRescalingController;
+
private final Duration initialResourceAllocationTimeout;
private final Duration resourceStabilizationTimeout;
@@ -294,6 +297,8 @@ public AdaptiveScheduler(
this.rescalingController = new EnforceMinimalIncreaseRescalingController(configuration);
+ this.forceRescalingController = new EnforceParallelismChangeRescalingController();
+
this.initialResourceAllocationTimeout = initialResourceAllocationTimeout;
this.resourceStabilizationTimeout = resourceStabilizationTimeout;
@@ -1162,16 +1167,24 @@ private ExecutionGraph createExecutionGraphAndRestoreState(
LOG);
}
+ /**
+ * In regular mode, rescale the job if added resource meets {@link
+ * JobManagerOptions#MIN_PARALLELISM_INCREASE}. In force mode rescale if the parallelism has
+ * changed.
+ */
@Override
- public boolean shouldRescale(ExecutionGraph executionGraph) {
+ public boolean shouldRescale(ExecutionGraph executionGraph, boolean forceRescale) {
final Optional maybeNewParallelism =
slotAllocator.determineParallelism(
jobInformation, declarativeSlotPool.getAllSlotsInformation());
return maybeNewParallelism
.filter(
- vertexParallelism ->
- rescalingController.shouldRescale(
- getCurrentParallelism(executionGraph), vertexParallelism))
+ vertexParallelism -> {
+ RescalingController rescalingControllerToUse =
+ forceRescale ? forceRescalingController : rescalingController;
+ return rescalingControllerToUse.shouldRescale(
+ getCurrentParallelism(executionGraph), vertexParallelism);
+ })
.isPresent();
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java
index 9cdb9afc7f286..10f872bbbd578 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java
@@ -19,6 +19,8 @@
package org.apache.flink.runtime.scheduler.adaptive;
import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.core.execution.SavepointFormatType;
import org.apache.flink.runtime.JobException;
import org.apache.flink.runtime.checkpoint.CheckpointScheduling;
@@ -39,6 +41,7 @@
import javax.annotation.Nullable;
import java.time.Duration;
+import java.time.Instant;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledFuture;
@@ -47,6 +50,11 @@
class Executing extends StateWithExecutionGraph implements ResourceListener {
private final Context context;
+ private final Instant lastRescale;
+ // only one schedule at the time
+ private boolean rescaleScheduled = false;
+ private final Duration scalingIntervalMin;
+ @Nullable private final Duration scalingIntervalMax;
Executing(
ExecutionGraph executionGraph,
@@ -55,7 +63,10 @@ class Executing extends StateWithExecutionGraph implements ResourceListener {
Logger logger,
Context context,
ClassLoader userCodeClassLoader,
- List failureCollection) {
+ List failureCollection,
+ Duration scalingIntervalMin,
+ @Nullable Duration scalingIntervalMax,
+ Instant lastRescale) {
super(
context,
executionGraph,
@@ -67,11 +78,29 @@ class Executing extends StateWithExecutionGraph implements ResourceListener {
this.context = context;
Preconditions.checkState(
executionGraph.getState() == JobStatus.RUNNING, "Assuming running execution graph");
+ this.scalingIntervalMin = scalingIntervalMin;
+ this.scalingIntervalMax = scalingIntervalMax;
+ // Executing is recreated with each restart (when we rescale)
+ // we consider the first execution of the pipeline as a rescale event
+ this.lastRescale = lastRescale;
+ Preconditions.checkState(
+ !scalingIntervalMin.isNegative(),
+ "%s must be positive integer or 0",
+ JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key());
+ if (scalingIntervalMax != null) {
+ Preconditions.checkState(
+ scalingIntervalMax.compareTo(scalingIntervalMin) > 0,
+ "%s(%d) must be greater than %s(%d)",
+ JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+ scalingIntervalMax,
+ JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN.key(),
+ scalingIntervalMin);
+ }
deploy();
// check if new resources have come available in the meantime
- context.runIfState(this, this::maybeRescale, Duration.ZERO);
+ rescaleWhenCooldownPeriodIsOver();
}
@Override
@@ -124,23 +153,74 @@ private void handleDeploymentFailure(ExecutionVertex executionVertex, JobExcepti
@Override
public void onNewResourcesAvailable() {
- maybeRescale();
+ rescaleWhenCooldownPeriodIsOver();
}
@Override
public void onNewResourceRequirements() {
- maybeRescale();
+ rescaleWhenCooldownPeriodIsOver();
}
+ /** Force rescaling as long as the target parallelism is different from the current one. */
+ private void forceRescale() {
+ if (context.shouldRescale(getExecutionGraph(), true)) {
+ getLogger()
+ .info(
+ "Added resources are still there after {} time({}), force a rescale.",
+ JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+ scalingIntervalMax);
+ context.goToRestarting(
+ getExecutionGraph(),
+ getExecutionGraphHandler(),
+ getOperatorCoordinatorHandler(),
+ Duration.ofMillis(0L),
+ getFailures());
+ }
+ }
+
+ /**
+ * Rescale the job if {@link Context#shouldRescale} is true. Otherwise, force a rescale using
+ * {@link Executing#forceRescale()} after {@link
+ * JobManagerOptions#SCHEDULER_SCALING_INTERVAL_MAX}.
+ */
private void maybeRescale() {
- if (context.shouldRescale(getExecutionGraph())) {
- getLogger().info("Can change the parallelism of job. Restarting job.");
+ rescaleScheduled = false;
+ if (context.shouldRescale(getExecutionGraph(), false)) {
+ getLogger().info("Can change the parallelism of the job. Restarting the job.");
context.goToRestarting(
getExecutionGraph(),
getExecutionGraphHandler(),
getOperatorCoordinatorHandler(),
Duration.ofMillis(0L),
getFailures());
+ } else if (scalingIntervalMax != null) {
+ getLogger()
+ .info(
+ "The longer the pipeline runs, the more the (small) resource gain is worth the restarting time. "
+ + "Last resource added does not meet {}, force a rescale after {} time({}) if the resource is still there.",
+ JobManagerOptions.MIN_PARALLELISM_INCREASE,
+ JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX.key(),
+ scalingIntervalMax);
+ if (timeSinceLastRescale().compareTo(scalingIntervalMax) > 0) {
+ forceRescale();
+ } else {
+ // schedule a force rescale in JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX time
+ context.runIfState(this, this::forceRescale, scalingIntervalMax);
+ }
+ }
+ }
+
+ private Duration timeSinceLastRescale() {
+ return Duration.between(lastRescale, Instant.now());
+ }
+
+ private void rescaleWhenCooldownPeriodIsOver() {
+ if (timeSinceLastRescale().compareTo(scalingIntervalMin) > 0) {
+ maybeRescale();
+ } else if (!rescaleScheduled) {
+ rescaleScheduled = true;
+ // schedule maybeRescale resetting the cooldown period
+ context.runIfState(this, this::maybeRescale, scalingIntervalMin);
}
}
@@ -196,9 +276,10 @@ interface Context
* Asks if we should rescale the currently executing job.
*
* @param executionGraph executionGraph for making the scaling decision.
+ * @param forceRescale should we force rescaling
* @return true, if we should rescale
*/
- boolean shouldRescale(ExecutionGraph executionGraph);
+ boolean shouldRescale(ExecutionGraph executionGraph, boolean forceRescale);
/**
* Runs the given action after a delay if the state at this time equals the expected state.
@@ -244,6 +325,7 @@ public Class getStateClass() {
}
public Executing getState() {
+ final Configuration jobConfiguration = executionGraph.getJobConfiguration();
return new Executing(
executionGraph,
executionGraphHandler,
@@ -251,7 +333,10 @@ public Executing getState() {
log,
context,
userCodeClassLoader,
- failureCollection);
+ failureCollection,
+ jobConfiguration.get(JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MIN),
+ jobConfiguration.get(JobManagerOptions.SCHEDULER_SCALING_INTERVAL_MAX),
+ Instant.now());
}
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java
index b02907acb7fd6..edf82d9a42fbd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceMinimalIncreaseRescalingController.java
@@ -24,8 +24,8 @@
import static org.apache.flink.configuration.JobManagerOptions.MIN_PARALLELISM_INCREASE;
/**
- * Simple scaling policy for a reactive mode. The user can configure a minimum cumulative
- * parallelism increase to allow a scale up.
+ * Simple scaling policy. The user can configure a minimum cumulative parallelism increase to allow
+ * a scale up.
*/
public class EnforceMinimalIncreaseRescalingController implements RescalingController {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java
new file mode 100644
index 0000000000000..94c11cf0250b3
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/scalingpolicy/EnforceParallelismChangeRescalingController.java
@@ -0,0 +1,41 @@
+/*
+ * 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.runtime.scheduler.adaptive.scalingpolicy;
+
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.scheduler.adaptive.allocator.VertexParallelism;
+
+/**
+ * Simple scaling policy. It just checks that the new parallelism is different (either increase or
+ * decrease) from the current parallelism.
+ */
+public class EnforceParallelismChangeRescalingController implements RescalingController {
+ @Override
+ public boolean shouldRescale(
+ VertexParallelism currentParallelism, VertexParallelism newParallelism) {
+ for (JobVertexID vertex : currentParallelism.getVertices()) {
+ int parallelismChange =
+ newParallelism.getParallelism(vertex)
+ - currentParallelism.getParallelism(vertex);
+ if (parallelismChange != 0) {
+ return true;
+ }
+ }
+ return false;
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java
index b0ab00e26ac29..6435ff81762e6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntry.java
@@ -237,6 +237,10 @@ public String getFQDNHostname() {
return fqdnHostname;
}
+ public String getEndpoint() {
+ return String.format("%s:%d", fqdnHostname, port);
+ }
+
@Override
public String toString() {
return new StringJoiner(
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java
index c50fcb0c0f89b..6024f60afdb66 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java
@@ -23,6 +23,7 @@
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.io.network.TaskEventPublisher;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
import java.net.InetAddress;
import java.util.concurrent.Executor;
@@ -42,6 +43,7 @@ public class ShuffleEnvironmentContext {
private final String[] tmpDirPaths;
private final Executor ioExecutor;
+ private final ScheduledExecutor scheduledExecutor;
public ShuffleEnvironmentContext(
Configuration configuration,
@@ -53,7 +55,8 @@ public ShuffleEnvironmentContext(
String[] tmpDirPaths,
TaskEventPublisher eventPublisher,
MetricGroup parentMetricGroup,
- Executor ioExecutor) {
+ Executor ioExecutor,
+ ScheduledExecutor scheduledExecutor) {
this.configuration = checkNotNull(configuration);
this.taskExecutorResourceId = checkNotNull(taskExecutorResourceId);
this.networkMemorySize = networkMemorySize;
@@ -62,6 +65,7 @@ public ShuffleEnvironmentContext(
this.eventPublisher = checkNotNull(eventPublisher);
this.parentMetricGroup = checkNotNull(parentMetricGroup);
this.ioExecutor = ioExecutor;
+ this.scheduledExecutor = scheduledExecutor;
this.numberOfSlots = numberOfSlots;
this.tmpDirPaths = checkNotNull(tmpDirPaths);
}
@@ -98,6 +102,10 @@ public Executor getIoExecutor() {
return ioExecutor;
}
+ public ScheduledExecutor getScheduledExecutor() {
+ return scheduledExecutor;
+ }
+
public int getNumberOfSlots() {
return numberOfSlots;
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
new file mode 100644
index 0000000000000..85f12329f5715
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
@@ -0,0 +1,142 @@
+/*
+ * 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.runtime.state;
+
+import javax.annotation.Nonnull;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+
+/** Abstract superclass for all {@link IncrementalKeyedStateHandle}. */
+public abstract class AbstractIncrementalStateHandle implements IncrementalKeyedStateHandle {
+ private static final long serialVersionUID = 1L;
+
+ /** The checkpoint Id. */
+ protected final long checkpointId;
+
+ /**
+ * UUID to identify the backend which created this state handle. This is in creating the key for
+ * the {@link SharedStateRegistry}.
+ */
+ protected final UUID backendIdentifier;
+
+ /** The key-group range covered by this state handle. */
+ protected final KeyGroupRange keyGroupRange;
+
+ /** Shared state in the incremental checkpoint. */
+ protected final List sharedState;
+
+ /** Primary meta data state of the incremental checkpoint. */
+ protected final StreamStateHandle metaStateHandle;
+
+ /** Unique id for this state handle. */
+ protected final StateHandleID stateHandleId;
+
+ public AbstractIncrementalStateHandle(
+ UUID backendIdentifier,
+ KeyGroupRange keyGroupRange,
+ long checkpointId,
+ List sharedState,
+ StreamStateHandle metaStateHandle,
+ StateHandleID stateHandleId) {
+ this.checkpointId = checkpointId;
+ this.keyGroupRange = keyGroupRange;
+ this.backendIdentifier = backendIdentifier;
+ this.sharedState = sharedState;
+ this.metaStateHandle = metaStateHandle;
+ this.stateHandleId = stateHandleId;
+ }
+
+ @Override
+ public long getCheckpointId() {
+ return checkpointId;
+ }
+
+ @Nonnull
+ @Override
+ public UUID getBackendIdentifier() {
+ return backendIdentifier;
+ }
+
+ @Override
+ public KeyGroupRange getKeyGroupRange() {
+ return keyGroupRange;
+ }
+
+ @Nonnull
+ @Override
+ public List getSharedStateHandles() {
+ return sharedState;
+ }
+
+ @Nonnull
+ @Override
+ public StreamStateHandle getMetaDataStateHandle() {
+ return metaStateHandle;
+ }
+
+ @Override
+ public StateHandleID getStateHandleId() {
+ return stateHandleId;
+ }
+
+ @Override
+ public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
+ return KeyGroupRange.EMPTY_KEY_GROUP_RANGE.equals(
+ getKeyGroupRange().getIntersection(keyGroupRange))
+ ? null
+ : this;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ AbstractIncrementalStateHandle that = (AbstractIncrementalStateHandle) o;
+ return Objects.equals(stateHandleId, that.stateHandleId);
+ }
+
+ @Override
+ public int hashCode() {
+ return stateHandleId.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "AbstractIncrementalStateHandle{"
+ + "checkpointId="
+ + checkpointId
+ + ", backendIdentifier="
+ + backendIdentifier
+ + ", keyGroupRange="
+ + keyGroupRange
+ + ", sharedState="
+ + sharedState
+ + ", metaStateHandle="
+ + metaStateHandle
+ + ", stateHandleId="
+ + stateHandleId
+ + '}';
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java
deleted file mode 100644
index 3f922a2e9680c..0000000000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.runtime.state;
-
-import javax.annotation.Nonnull;
-
-/**
- * This class is a keyed state handle based on a directory. It combines a {@link
- * DirectoryStateHandle} and a {@link KeyGroupRange}.
- */
-public class DirectoryKeyedStateHandle implements KeyedStateHandle {
-
- private static final long serialVersionUID = 1L;
-
- /** The directory state handle. */
- @Nonnull private final DirectoryStateHandle directoryStateHandle;
-
- /** The key-group range. */
- @Nonnull private final KeyGroupRange keyGroupRange;
-
- private final StateHandleID stateHandleId;
-
- public DirectoryKeyedStateHandle(
- @Nonnull DirectoryStateHandle directoryStateHandle,
- @Nonnull KeyGroupRange keyGroupRange) {
-
- this.directoryStateHandle = directoryStateHandle;
- this.keyGroupRange = keyGroupRange;
- this.stateHandleId = StateHandleID.randomStateHandleId();
- }
-
- @Nonnull
- public DirectoryStateHandle getDirectoryStateHandle() {
- return directoryStateHandle;
- }
-
- @Nonnull
- @Override
- public KeyGroupRange getKeyGroupRange() {
- return keyGroupRange;
- }
-
- @Override
- public void discardState() throws Exception {
- directoryStateHandle.discardState();
- }
-
- @Override
- public long getStateSize() {
- return directoryStateHandle.getStateSize();
- }
-
- @Override
- public long getCheckpointedSize() {
- return getStateSize();
- }
-
- @Override
- public KeyedStateHandle getIntersection(KeyGroupRange otherKeyGroupRange) {
- return this.keyGroupRange.getIntersection(otherKeyGroupRange).getNumberOfKeyGroups() > 0
- ? this
- : null;
- }
-
- @Override
- public StateHandleID getStateHandleId() {
- return stateHandleId;
- }
-
- @Override
- public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
- // Nothing to do, this is for local use only.
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- DirectoryKeyedStateHandle that = (DirectoryKeyedStateHandle) o;
-
- if (!getDirectoryStateHandle().equals(that.getDirectoryStateHandle())) {
- return false;
- }
- return getKeyGroupRange().equals(that.getKeyGroupRange());
- }
-
- @Override
- public int hashCode() {
- int result = getDirectoryStateHandle().hashCode();
- result = 31 * result + getKeyGroupRange().hashCode();
- return result;
- }
-
- @Override
- public String toString() {
- return "DirectoryKeyedStateHandle{"
- + "directoryStateHandle="
- + directoryStateHandle
- + ", keyGroupRange="
- + keyGroupRange
- + '}';
- }
-}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java
index 9ce3cbd332fcb..f162efa936bac 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java
@@ -42,6 +42,9 @@ public interface IncrementalKeyedStateHandle
@Nonnull
List getSharedStateHandles();
+ @Nonnull
+ StreamStateHandle getMetaDataStateHandle();
+
/** A Holder of StreamStateHandle and the corresponding localPath. */
final class HandleAndLocalPath implements Serializable {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
index d782cf886bd5a..ac457f00622b6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
@@ -32,22 +32,11 @@
* DirectoryStateHandle} that represents the directory of the native RocksDB snapshot, the key
* groups, and a stream state handle for Flink's state meta data file.
*/
-public class IncrementalLocalKeyedStateHandle extends DirectoryKeyedStateHandle
- implements IncrementalKeyedStateHandle {
+public class IncrementalLocalKeyedStateHandle extends AbstractIncrementalStateHandle {
private static final long serialVersionUID = 1L;
- /** Id of the checkpoint that created this state handle. */
- @Nonnegative private final long checkpointId;
-
- /** UUID to identify the backend which created this state handle. */
- @Nonnull private final UUID backendIdentifier;
-
- /** Handle to Flink's state meta data. */
- @Nonnull private final StreamStateHandle metaDataState;
-
- /** All shared state handles and the corresponding localPath used by the checkpoint. */
- @Nonnull private final List sharedState;
+ private final DirectoryStateHandle directoryStateHandle;
public IncrementalLocalKeyedStateHandle(
@Nonnull UUID backendIdentifier,
@@ -57,21 +46,14 @@ public IncrementalLocalKeyedStateHandle(
@Nonnull StreamStateHandle metaDataState,
@Nonnull List sharedState) {
- super(directoryStateHandle, keyGroupRange);
- this.backendIdentifier = backendIdentifier;
- this.checkpointId = checkpointId;
- this.metaDataState = metaDataState;
- this.sharedState = new ArrayList<>(sharedState);
- }
-
- @Nonnull
- public StreamStateHandle getMetaDataState() {
- return metaDataState;
- }
-
- @Override
- public long getCheckpointId() {
- return checkpointId;
+ super(
+ backendIdentifier,
+ keyGroupRange,
+ checkpointId,
+ new ArrayList<>(sharedState),
+ metaDataState,
+ StateHandleID.randomStateHandleId());
+ this.directoryStateHandle = directoryStateHandle;
}
@Override
@@ -81,52 +63,23 @@ public CheckpointBoundKeyedStateHandle rebound(long checkpointId) {
checkpointId,
getDirectoryStateHandle(),
getKeyGroupRange(),
- getMetaDataState(),
+ getMetaDataStateHandle(),
getSharedStateHandles());
}
- @Override
- @Nonnull
- public UUID getBackendIdentifier() {
- return backendIdentifier;
- }
-
- @Override
- @Nonnull
- public List getSharedStateHandles() {
- return sharedState;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- if (!super.equals(o)) {
- return false;
- }
-
- IncrementalLocalKeyedStateHandle that = (IncrementalLocalKeyedStateHandle) o;
-
- return getMetaDataState().equals(that.getMetaDataState());
- }
-
@Override
public void discardState() throws Exception {
Exception collectedEx = null;
try {
- super.discardState();
+ directoryStateHandle.discardState();
} catch (Exception e) {
collectedEx = e;
}
try {
- metaDataState.discardState();
+ metaStateHandle.discardState();
} catch (Exception e) {
collectedEx = ExceptionUtils.firstOrSuppressed(e, collectedEx);
}
@@ -138,22 +91,30 @@ public void discardState() throws Exception {
@Override
public long getStateSize() {
- return super.getStateSize() + metaDataState.getStateSize();
- }
-
- @Override
- public int hashCode() {
- int result = super.hashCode();
- result = 31 * result + getMetaDataState().hashCode();
- return result;
+ return directoryStateHandle.getStateSize() + metaStateHandle.getStateSize();
}
@Override
public String toString() {
return "IncrementalLocalKeyedStateHandle{"
- + "metaDataState="
- + metaDataState
+ + "directoryStateHandle="
+ + directoryStateHandle
+ "} "
+ super.toString();
}
+
+ @Override
+ public void registerSharedStates(SharedStateRegistry stateRegistry, long checkpointID) {
+ // Nothing to do, this is for local use only.
+ }
+
+ @Override
+ public long getCheckpointedSize() {
+ return directoryStateHandle.getStateSize();
+ }
+
+ @Nonnull
+ public DirectoryStateHandle getDirectoryStateHandle() {
+ return directoryStateHandle;
+ }
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java
index fd524d74f2adc..41b9a0466cc2b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java
@@ -56,7 +56,7 @@
* should not be called from production code. This means this class is also not suited to serve as a
* key, e.g. in hash maps.
*/
-public class IncrementalRemoteKeyedStateHandle implements IncrementalKeyedStateHandle {
+public class IncrementalRemoteKeyedStateHandle extends AbstractIncrementalStateHandle {
public static final long UNKNOWN_CHECKPOINTED_SIZE = -1L;
@@ -65,31 +65,11 @@ public class IncrementalRemoteKeyedStateHandle implements IncrementalKeyedStateH
private static final long serialVersionUID = -8328808513197388231L;
- /**
- * UUID to identify the backend which created this state handle. This is in creating the key for
- * the {@link SharedStateRegistry}.
- */
- private final UUID backendIdentifier;
-
- /** The key-group range covered by this state handle. */
- private final KeyGroupRange keyGroupRange;
-
- /** The checkpoint Id. */
- private final long checkpointId;
-
- /** Shared state in the incremental checkpoint. */
- private final List sharedState;
-
/** Private state in the incremental checkpoint. */
private final List privateState;
- /** Primary meta data state of the incremental checkpoint. */
- private final StreamStateHandle metaStateHandle;
-
private final long persistedSizeOfThisCheckpoint;
- private final StateHandleID stateHandleId;
-
/**
* Once the shared states are registered, it is the {@link SharedStateRegistry}'s responsibility
* to cleanup those shared states. But in the cases where the state handle is discarded before
@@ -147,18 +127,19 @@ protected IncrementalRemoteKeyedStateHandle(
StreamStateHandle metaStateHandle,
long persistedSizeOfThisCheckpoint,
StateHandleID stateHandleId) {
- this.backendIdentifier = Preconditions.checkNotNull(backendIdentifier);
- this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange);
- this.checkpointId = checkpointId;
- this.sharedState = Preconditions.checkNotNull(sharedState);
+ super(
+ Preconditions.checkNotNull(backendIdentifier),
+ Preconditions.checkNotNull(keyGroupRange),
+ checkpointId,
+ sharedState,
+ metaStateHandle,
+ stateHandleId);
this.privateState = Preconditions.checkNotNull(privateState);
- this.metaStateHandle = Preconditions.checkNotNull(metaStateHandle);
this.sharedStateRegistry = null;
this.persistedSizeOfThisCheckpoint =
persistedSizeOfThisCheckpoint == UNKNOWN_CHECKPOINTED_SIZE
? getStateSize()
: persistedSizeOfThisCheckpoint;
- this.stateHandleId = stateHandleId;
}
public static IncrementalRemoteKeyedStateHandle restore(
@@ -181,16 +162,6 @@ public static IncrementalRemoteKeyedStateHandle restore(
stateHandleId);
}
- @Override
- public KeyGroupRange getKeyGroupRange() {
- return keyGroupRange;
- }
-
- @Override
- public long getCheckpointId() {
- return checkpointId;
- }
-
@Override
public CheckpointBoundKeyedStateHandle rebound(long checkpointId) {
return new IncrementalRemoteKeyedStateHandle(
@@ -212,15 +183,6 @@ public List getPrivateState() {
return privateState;
}
- public StreamStateHandle getMetaStateHandle() {
- return metaStateHandle;
- }
-
- @Nonnull
- public UUID getBackendIdentifier() {
- return backendIdentifier;
- }
-
@Nonnull
@Override
public List getSharedStateHandles() {
@@ -231,19 +193,6 @@ public SharedStateRegistry getSharedStateRegistry() {
return sharedStateRegistry;
}
- @Override
- public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) {
- return KeyGroupRange.EMPTY_KEY_GROUP_RANGE.equals(
- this.keyGroupRange.getIntersection(keyGroupRange))
- ? null
- : this;
- }
-
- @Override
- public StateHandleID getStateHandleId() {
- return stateHandleId;
- }
-
@Override
public void discardState() throws Exception {
@@ -357,75 +306,16 @@ IncrementalRemoteKeyedStateHandle copy() {
stateHandleId);
}
- /**
- * This method is should only be called in tests! This should never serve as key in a hash map.
- */
- @VisibleForTesting
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- IncrementalRemoteKeyedStateHandle that = (IncrementalRemoteKeyedStateHandle) o;
-
- if (!getStateHandleId().equals(that.getStateHandleId())) {
- return false;
- }
- if (getCheckpointId() != that.getCheckpointId()) {
- return false;
- }
- if (!getBackendIdentifier().equals(that.getBackendIdentifier())) {
- return false;
- }
- if (!getKeyGroupRange().equals(that.getKeyGroupRange())) {
- return false;
- }
- if (!getSharedState().equals(that.getSharedState())) {
- return false;
- }
- if (!getPrivateState().equals(that.getPrivateState())) {
- return false;
- }
- return getMetaStateHandle().equals(that.getMetaStateHandle());
- }
-
- /** This method should only be called in tests! This should never serve as key in a hash map. */
- @VisibleForTesting
- @Override
- public int hashCode() {
- int result = getBackendIdentifier().hashCode();
- result = 31 * result + getKeyGroupRange().hashCode();
- result = 31 * result + (int) (getCheckpointId() ^ (getCheckpointId() >>> 32));
- result = 31 * result + getSharedState().hashCode();
- result = 31 * result + getPrivateState().hashCode();
- result = 31 * result + getMetaStateHandle().hashCode();
- result = 31 * result + getStateHandleId().hashCode();
- return result;
- }
-
@Override
public String toString() {
return "IncrementalRemoteKeyedStateHandle{"
- + "backendIdentifier="
- + backendIdentifier
- + ", stateHandleId="
- + stateHandleId
- + ", keyGroupRange="
- + keyGroupRange
- + ", checkpointId="
- + checkpointId
- + ", sharedState="
- + sharedState
- + ", privateState="
+ + "privateState="
+ privateState
- + ", metaStateHandle="
- + metaStateHandle
+ + ", persistedSizeOfThisCheckpoint="
+ + persistedSizeOfThisCheckpoint
+ ", registered="
+ (sharedStateRegistry != null)
- + '}';
+ + "} "
+ + super.toString();
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java
index ddfb9163cb085..0574c6f491ea1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ListDelimitedSerializer.java
@@ -21,7 +21,6 @@
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputDeserializer;
import org.apache.flink.core.memory.DataOutputSerializer;
-import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.util.Preconditions;
import java.io.IOException;
@@ -39,7 +38,8 @@ public final class ListDelimitedSerializer {
private final DataInputDeserializer dataInputView = new DataInputDeserializer();
private final DataOutputSerializer dataOutputView = new DataOutputSerializer(128);
- public List deserializeList(byte[] valueBytes, TypeSerializer elementSerializer) {
+ public List deserializeList(byte[] valueBytes, TypeSerializer elementSerializer)
+ throws IOException {
if (valueBytes == null) {
return null;
}
@@ -76,17 +76,13 @@ public byte[] serializeList(List valueList, TypeSerializer elementSeri
/** Deserializes a single element from a serialized list. */
public static T deserializeNextElement(
- DataInputDeserializer in, TypeSerializer elementSerializer) {
- try {
+ DataInputDeserializer in, TypeSerializer elementSerializer) throws IOException {
+ if (in.available() > 0) {
+ T element = elementSerializer.deserialize(in);
if (in.available() > 0) {
- T element = elementSerializer.deserialize(in);
- if (in.available() > 0) {
- in.readByte();
- }
- return element;
+ in.readByte();
}
- } catch (IOException e) {
- throw new FlinkRuntimeException("Unexpected list element deserialization failure", e);
+ return element;
}
return null;
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java
index 7cb363b91c332..35d6c78e7eff5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java
@@ -29,6 +29,8 @@
import java.util.ArrayList;
import java.util.List;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
/**
* Implementation of operator list state.
*
@@ -129,8 +131,12 @@ public void update(List values) {
@Override
public void addAll(List values) {
- if (values != null && !values.isEmpty()) {
- internalList.addAll(values);
+ Preconditions.checkNotNull(values, "List of values to add cannot be null.");
+ if (!values.isEmpty()) {
+ for (S value : values) {
+ checkNotNull(value, "Any value to add to a list cannot be null.");
+ add(value);
+ }
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
index 2d786c44d0d97..6c5c7feae06c9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
@@ -174,7 +174,7 @@ private static KeyedStateHandle castToAbsolutePath(
StreamStateHandle castMetaStateHandle =
restoreFileStateHandle(
- incrementalRemoteKeyedStateHandle.getMetaStateHandle());
+ incrementalRemoteKeyedStateHandle.getMetaDataStateHandle());
List castSharedStates =
incrementalRemoteKeyedStateHandle.getSharedState().stream()
.map(
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java
index fbf8914f87d6a..59a86de87f992 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java
@@ -28,8 +28,6 @@
import org.apache.flink.runtime.state.internal.InternalAggregatingState;
import org.apache.flink.util.Preconditions;
-import java.io.IOException;
-
/**
* Heap-backed partitioned {@link AggregatingState} that is snapshotted into files.
*
@@ -93,20 +91,14 @@ public OUT get() {
}
@Override
- public void add(IN value) throws IOException {
+ public void add(IN value) throws Exception {
final N namespace = currentNamespace;
if (value == null) {
clear();
return;
}
-
- try {
- stateTable.transform(namespace, value, aggregateTransformation);
- } catch (Exception e) {
- throw new IOException(
- "Exception while applying AggregateFunction in aggregating state", e);
- }
+ stateTable.transform(namespace, value, aggregateTransformation);
}
// ------------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
index 0f32d0bbc270a..919700f539d6a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java
@@ -30,6 +30,7 @@
import org.apache.flink.util.Preconditions;
import java.io.ByteArrayOutputStream;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -106,7 +107,7 @@ public byte[] getSerializedValue(
final TypeSerializer