You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by sj...@apache.org on 2020/05/18 17:43:26 UTC

[flink] branch release-1.11 updated (2aa45cd -> e564a3a)

This is an automated email from the ASF dual-hosted git repository.

sjwiesman pushed a change to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 2aa45cd  [FLINK-17687][tests] Simplify collection of Mesos logs
     new 3424d66  [hotfix][state-processor-api] Remove BoundedStreamConfig
     new e564a3a  [FLINK-17506][state-processor-api] SavepointEnvironment should honour 'io.tmp.dirs' property

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../flink/state/api/BootstrapTransformation.java   | 16 ++++---
 .../state/api/runtime/BoundedStreamConfig.java     | 51 ----------------------
 .../state/api/runtime/SavepointEnvironment.java    |  3 +-
 3 files changed, 12 insertions(+), 58 deletions(-)
 delete mode 100644 flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BoundedStreamConfig.java


[flink] 01/02: [hotfix][state-processor-api] Remove BoundedStreamConfig

Posted by sj...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sjwiesman pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 3424d660a3209fbbd34e644de29328001cf69013
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Mon May 4 20:08:48 2020 -0500

    [hotfix][state-processor-api] Remove BoundedStreamConfig
---
 .../flink/state/api/BootstrapTransformation.java   | 17 +++++---
 .../state/api/runtime/BoundedStreamConfig.java     | 51 ----------------------
 2 files changed, 11 insertions(+), 57 deletions(-)

diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java
index 5e2a7c2..1c6127b 100644
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.MapPartitionOperator;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.checkpoint.OperatorState;
 import org.apache.flink.runtime.jobgraph.OperatorID;
@@ -36,7 +37,7 @@ import org.apache.flink.state.api.output.TaggedOperatorSubtaskState;
 import org.apache.flink.state.api.output.operators.BroadcastStateBootstrapOperator;
 import org.apache.flink.state.api.output.partitioner.HashSelector;
 import org.apache.flink.state.api.output.partitioner.KeyGroupRangePartitioner;
-import org.apache.flink.state.api.runtime.BoundedStreamConfig;
+import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 
@@ -175,12 +176,16 @@ public class BootstrapTransformation<T> {
 
 	@VisibleForTesting
 	StreamConfig getConfig(OperatorID operatorID, StateBackend stateBackend, StreamOperator<TaggedOperatorSubtaskState> operator) {
-		final StreamConfig config;
-		if (keyType == null) {
-			config = new BoundedStreamConfig();
-		} else {
+		final StreamConfig config = new StreamConfig(new Configuration());
+		config.setChainStart();
+		config.setCheckpointingEnabled(true);
+		config.setCheckpointMode(CheckpointingMode.EXACTLY_ONCE);
+
+		if (keyType != null) {
 			TypeSerializer<?> keySerializer = keyType.createSerializer(dataSet.getExecutionEnvironment().getConfig());
-			config = new BoundedStreamConfig(keySerializer, originalKeySelector);
+
+			config.setStateKeySerializer(keySerializer);
+			config.setStatePartitioner(0, originalKeySelector);
 		}
 
 		config.setStreamOperator(operator);
diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BoundedStreamConfig.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BoundedStreamConfig.java
deleted file mode 100644
index 5f3b38b..0000000
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BoundedStreamConfig.java
+++ /dev/null
@@ -1,51 +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.state.api.runtime;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-
-/**
- * A {@link StreamConfig} with default settings.
- */
-@Internal
-public class BoundedStreamConfig extends StreamConfig {
-
-	private static final long serialVersionUID = 1L;
-
-	public BoundedStreamConfig() {
-		super(new Configuration());
-
-		setChainStart();
-		setCheckpointingEnabled(true);
-		setCheckpointMode(CheckpointingMode.EXACTLY_ONCE);
-	}
-
-	public <IN> BoundedStreamConfig(TypeSerializer<?> keySerializer, KeySelector<IN, ?> keySelector) {
-		this();
-
-		setStateKeySerializer(keySerializer);
-		setStatePartitioner(0, keySelector);
-	}
-}
-


[flink] 02/02: [FLINK-17506][state-processor-api] SavepointEnvironment should honour 'io.tmp.dirs' property

Posted by sj...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sjwiesman pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e564a3a7773b199bfa2f0b37d1459ae3430bc6a5
Author: Seth Wiesman <sj...@gmail.com>
AuthorDate: Mon May 4 20:13:48 2020 -0500

    [FLINK-17506][state-processor-api] SavepointEnvironment should honour 'io.tmp.dirs' property
---
 .../main/java/org/apache/flink/state/api/BootstrapTransformation.java  | 3 +--
 .../java/org/apache/flink/state/api/runtime/SavepointEnvironment.java  | 3 ++-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java
index 1c6127b..5587046 100644
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/BootstrapTransformation.java
@@ -26,7 +26,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.MapPartitionOperator;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.checkpoint.OperatorState;
 import org.apache.flink.runtime.jobgraph.OperatorID;
@@ -176,7 +175,7 @@ public class BootstrapTransformation<T> {
 
 	@VisibleForTesting
 	StreamConfig getConfig(OperatorID operatorID, StateBackend stateBackend, StreamOperator<TaggedOperatorSubtaskState> operator) {
-		final StreamConfig config = new StreamConfig(new Configuration());
+		final StreamConfig config = new StreamConfig(dataSet.getExecutionEnvironment().getConfiguration());
 		config.setChainStart();
 		config.setCheckpointingEnabled(true);
 		config.setCheckpointMode(CheckpointingMode.EXACTLY_ONCE);
diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java
index fca7a74..915b676 100644
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
@@ -104,7 +105,7 @@ public class SavepointEnvironment implements Environment {
 
 		this.registry = new KvStateRegistry().createTaskRegistry(jobID, vertexID);
 		this.taskStateManager = new SavepointTaskStateManager(prioritizedOperatorSubtaskState);
-		this.ioManager = new IOManagerAsync();
+		this.ioManager = new IOManagerAsync(ConfigurationUtils.parseTempDirectories(configuration));
 		this.memoryManager = MemoryManager.forDefaultPageSize(64 * 1024 * 1024);
 		this.accumulatorRegistry = new AccumulatorRegistry(jobID, attemptID);
 	}