(flink) branch master updated: [FLINK-33986][runtime] Extend ShuffleMaster to support snapshot and restore state.

2024-05-14 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 65d31e26534 [FLINK-33986][runtime] Extend ShuffleMaster to support 
snapshot and restore state.
65d31e26534 is described below

commit 65d31e26534836909f6b8139c6bd6cd45b91bba4
Author: JunRuiLee 
AuthorDate: Fri Feb 2 18:01:03 2024 +0800

[FLINK-33986][runtime] Extend ShuffleMaster to support snapshot and restore 
state.
---
 .../DefaultShuffleMasterSnapshotContext.java   | 22 +
 .../shuffle/EmptyShuffleMasterSnapshot.java| 37 ++
 .../flink/runtime/shuffle/NettyShuffleMaster.java  | 12 +++
 .../flink/runtime/shuffle/ShuffleMaster.java   | 19 +++
 .../runtime/shuffle/ShuffleMasterSnapshot.java | 31 ++
 .../shuffle/ShuffleMasterSnapshotContext.java  | 22 +
 6 files changed, 143 insertions(+)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/DefaultShuffleMasterSnapshotContext.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/DefaultShuffleMasterSnapshotContext.java
new file mode 100644
index 000..8fcff36eef1
--- /dev/null
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/DefaultShuffleMasterSnapshotContext.java
@@ -0,0 +1,22 @@
+/*
+ * 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.shuffle;
+
+/** Default {@link ShuffleMasterSnapshotContext} implementation. */
+public class DefaultShuffleMasterSnapshotContext implements 
ShuffleMasterSnapshotContext {}
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/EmptyShuffleMasterSnapshot.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/EmptyShuffleMasterSnapshot.java
new file mode 100644
index 000..683ccd07968
--- /dev/null
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/EmptyShuffleMasterSnapshot.java
@@ -0,0 +1,37 @@
+/*
+ * 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.shuffle;
+
+/**
+ * A singleton implementation of {@link ShuffleMasterSnapshot} that represents 
an empty snapshot of
+ * shuffle master.
+ */
+public class EmptyShuffleMasterSnapshot implements ShuffleMasterSnapshot {
+
+private static final EmptyShuffleMasterSnapshot INSTANCE = new 
EmptyShuffleMasterSnapshot();
+
+@Override
+public boolean isIncremental() {
+return false;
+}
+
+public static EmptyShuffleMasterSnapshot getInstance() {
+return INSTANCE;
+}
+}
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java
index 9cce16cf495..461457a29b6 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java
@@ -187,4 +187,16 @@ public class NettyShuffleMaster implements 
ShuffleMaster
 public void unregisterJob(JobID jobId) {
 jobShuffleContexts.remove(jobId);
 }
+
+@Override
+public boolean supportsBatchSnapshot() {
+return true;
+}
+
+@Override
+public void snaps

(flink) branch master updated: [FLINK-35293][hive] Hive source supports dynamic parallelism inference

2024-05-13 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new ddb5a5355f9 [FLINK-35293][hive] Hive source supports dynamic 
parallelism inference
ddb5a5355f9 is described below

commit ddb5a5355f9aca3d223f1fff6581d83dd317c2de
Author: sunxia 
AuthorDate: Tue May 7 14:49:17 2024 +0800

[FLINK-35293][hive] Hive source supports dynamic parallelism inference
---
 .../docs/connectors/table/hive/hive_read_write.md  |  15 +-
 .../docs/connectors/table/hive/hive_read_write.md  |  17 +-
 .../connector/file/src/AbstractFileSource.java |   2 +-
 .../connector/file/table/LimitableBulkFormat.java  |   4 +
 .../HiveDynamicParallelismInferenceFactory.java|  67 +
 .../apache/flink/connectors/hive/HiveOptions.java  |  53 +++-
 .../connectors/hive/HiveParallelismInference.java  |  33 ++-
 .../apache/flink/connectors/hive/HiveSource.java   |  55 +++-
 .../flink/connectors/hive/HiveSourceBuilder.java   |  11 +
 .../hive/HiveSourceDynamicFileEnumerator.java  |   3 +-
 .../connectors/hive/HiveSourceFileEnumerator.java  |   4 +
 .../HiveStaticParallelismInferenceFactory.java |  64 +
 .../flink/connectors/hive/HiveTableSource.java |   3 +-
 .../hive/HiveSourceDynamicFileEnumeratorTest.java  |  17 +-
 .../flink/connectors/hive/HiveSourceITCase.java|   2 +-
 .../flink/connectors/hive/HiveSourceTest.java  | 309 +
 .../flink/table/catalog/hive/HiveTestUtils.java|  21 ++
 17 files changed, 632 insertions(+), 48 deletions(-)

diff --git a/docs/content.zh/docs/connectors/table/hive/hive_read_write.md 
b/docs/content.zh/docs/connectors/table/hive/hive_read_write.md
index bdd3ac4f06d..64f9f7dc8f0 100644
--- a/docs/content.zh/docs/connectors/table/hive/hive_read_write.md
+++ b/docs/content.zh/docs/connectors/table/hive/hive_read_write.md
@@ -136,16 +136,21 @@ Flink 允许你灵活的配置并发推断策略。你可以在 `TableConfig` 
   
   
 
-table.exec.hive.infer-source-parallelism
-true
-Boolean
-如果是 true,会根据 split 的数量推断 source 的并发度。如果是 false,source 
的并发度由配置决定。
+table.exec.hive.infer-source-parallelism.mode
+dynamic
+InferMode
+选择Hive Source并行度推断模式的选项,根据splits数推断并行度。
+'static' 代表静态推断,它会在作业创建阶段推断Source并行度。
+'dynamic' 代表动态推断,它会在作业执行阶段利用运行时信息更准确地推断Source并行度。
+'none' 代表禁用并行度推断。
+注意,它仍然受到已弃用选项 'table.exec.hive.infer-source-parallelism' 的影响,需要其值为 
true 才能启用并行度推断。
+
 
 
 table.exec.hive.infer-source-parallelism.max
 1000
 Integer
-设置 source operator 推断的最大并发度。
+设置 source operator 推断的最大并发度。请注意,默认值仅在静态并行性推断模式下有效。
 
   
 
diff --git a/docs/content/docs/connectors/table/hive/hive_read_write.md 
b/docs/content/docs/connectors/table/hive/hive_read_write.md
index 98742a92be7..9a9834b6579 100644
--- a/docs/content/docs/connectors/table/hive/hive_read_write.md
+++ b/docs/content/docs/connectors/table/hive/hive_read_write.md
@@ -152,16 +152,23 @@ following parameters in `TableConfig` (note that these 
parameters affect all sou
   
   
 
-table.exec.hive.infer-source-parallelism
-true
-Boolean
-If is true, source parallelism is inferred according to splits 
number. If is false, parallelism of source are set by config.
+table.exec.hive.infer-source-parallelism.mode
+dynamic
+InferMode
+An option for selecting the hive source parallelism inference mode 
to infer parallelism according to splits number.
+'static' represents static inference, which will infer source 
parallelism at job creation stage.
+'dynamic' represents dynamic inference, which will infer 
parallelism at job execution stage and could more accurately infer the source 
parallelism.
+'none' represents disabling parallelism inference.
+Note that it is still affected by the deprecated option 
'table.exec.hive.infer-source-parallelism', requiring its value to be true for 
enabling parallelism inference.
+
 
 
 table.exec.hive.infer-source-parallelism.max
 1000
 Integer
-Sets max infer parallelism for source operator.
+Sets max infer parallelism for source operator.
+Note that the default value is effective only in the static 
parallelism inference mode.
+
 
   
 
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
index b14d46b3f9c..c9133172070 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java

(flink) branch master updated (36b1d2acd6d -> 4e6b42046ad)

2024-05-13 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 36b1d2acd6d [FLINK-35302][rest] Ignore unknown fields in REST request 
deserialization
 add 4e6b42046ad [FLINK-34661][runtime] TaskExecutor supports retain 
partitions after JM crashed.

No new revisions were added by this update.

Summary of changes:
 .../apache/flink/runtime/jobmaster/JobMaster.java  |   3 +-
 .../runtime/shuffle/ShuffleServiceOptions.java |   5 +-
 .../flink/runtime/taskexecutor/TaskExecutor.java   |  74 ---
 .../runtime/taskexecutor/TaskExecutorGateway.java  |   6 +-
 .../TaskExecutorPartitionLifecycleTest.java| 105 -
 .../taskexecutor/TaskExecutorSubmissionTest.java   |   2 +-
 .../taskexecutor/TestingTaskExecutorGateway.java   |   2 +-
 7 files changed, 173 insertions(+), 24 deletions(-)



(flink) branch master updated (4611817591c -> 547e4b53ebe)

2024-05-08 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 4611817591c [FLINK-34916][table] Support `ALTER CATALOG SET` syntax
 add 547e4b53ebe [FLINK-35270]Enrich information in logs, making it easier 
for debugging

No new revisions were added by this update.

Summary of changes:
 .../connector/datagen/functions/FromElementsGeneratorFunction.java | 2 +-
 .../main/java/org/apache/flink/fs/osshadoop/writer/OSSCommitter.java   | 3 +--
 .../streaming/runtime/io/checkpointing/BarrierAlignmentUtilTest.java   | 2 +-
 .../flink/connector/testframe/testsuites/SourceTestSuiteBase.java  | 2 +-
 4 files changed, 4 insertions(+), 5 deletions(-)



(flink) 02/03: [FLINK-33984][runtime] Skip maintain subtaskGatewayMap when CheckpointCoordinator is null.

2024-04-07 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 38255652406becbfbcb7cbec557aa5ba9a1ebbb3
Author: JunRuiLee 
AuthorDate: Fri Mar 1 11:03:42 2024 +0800

[FLINK-33984][runtime] Skip maintain subtaskGatewayMap when 
CheckpointCoordinator is null.
---
 .../coordination/OperatorCoordinatorHolder.java| 11 +++--
 .../OperatorCoordinatorHolderTest.java | 48 +-
 2 files changed, 52 insertions(+), 7 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
index f3253f147db..8aff60fb766 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
@@ -110,9 +110,9 @@ public class OperatorCoordinatorHolder
 
 /**
  * A map that manages subtask gateways. It is used to control the 
opening/closing of each
- * gateway during checkpoint. This map should only be read or modified 
when concurrent execution
- * attempt is disabled. Note that concurrent execution attempt is 
currently guaranteed to be
- * disabled when checkpoint is enabled.
+ * gateway during checkpoints. This map should only be read or modified in 
Streaming mode. Given
+ * that the CheckpointCoordinator is guaranteed to be non-null in 
Streaming mode, construction
+ * of this map can be skipped if the CheckpointCoordinator is null.
  */
 private final Map subtaskGatewayMap;
 
@@ -437,9 +437,8 @@ public class OperatorCoordinatorHolder
 final SubtaskGatewayImpl gateway =
 new SubtaskGatewayImpl(sta, mainThreadExecutor, 
unconfirmedEvents);
 
-// When concurrent execution attempts is supported, the checkpoint 
must have been disabled.
-// Thus, we don't need to maintain subtaskGatewayMap
-if (!context.isConcurrentExecutionAttemptsSupported()) {
+// We don't need to maintain subtaskGatewayMap when checkpoint 
coordinator is null.
+if (context.getCheckpointCoordinator() != null) {
 subtaskGatewayMap.put(gateway.getSubtask(), gateway);
 }
 
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java
index 95841dcb634..278214f8901 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolderTest.java
@@ -18,21 +18,36 @@
 
 package org.apache.flink.runtime.operators.coordination;
 
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureManager;
+import org.apache.flink.runtime.checkpoint.CheckpointPlanCalculatorContext;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.CheckpointsCleaner;
+import org.apache.flink.runtime.checkpoint.DefaultCheckpointPlanCalculator;
+import org.apache.flink.runtime.checkpoint.NoOpFailJobCall;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.StandaloneCompletedCheckpointStore;
 import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import 
org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
 import 
org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutorService;
 import org.apache.flink.runtime.executiongraph.TaskInformation;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
+import 
org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
 import 
org.apache.flink.runtime.operators.coordination.EventReceivingTasks.EventWithSubtask;
 import org.apache.flink.runtime.scheduler.GlobalFailureHandler;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
 import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
 
+import

(flink) 01/03: [hotfix][test] Migrate SplitAssignmentTrackerTest to Junit5 and AssertJ.

2024-04-07 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 191e6ceb6bd0e8ea9fb547a83f364fecf2b569c2
Author: JunRuiLee 
AuthorDate: Mon Jan 22 15:39:07 2024 +0800

[hotfix][test] Migrate SplitAssignmentTrackerTest to Junit5 and AssertJ.
---
 .../coordinator/SplitAssignmentTrackerTest.java| 27 ++
 1 file changed, 12 insertions(+), 15 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SplitAssignmentTrackerTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SplitAssignmentTrackerTest.java
index f177fbff03c..3c8c26047ca 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SplitAssignmentTrackerTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SplitAssignmentTrackerTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.source.coordinator;
 
 import org.apache.flink.api.connector.source.mocks.MockSourceSplit;
 
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.Arrays;
 import java.util.Collections;
@@ -30,16 +30,13 @@ import java.util.Map;
 
 import static 
org.apache.flink.runtime.source.coordinator.CoordinatorTestUtils.getSplitsAssignment;
 import static 
org.apache.flink.runtime.source.coordinator.CoordinatorTestUtils.verifyAssignment;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /** Unit test for @link {@link SplitAssignmentTracker}. */
-public class SplitAssignmentTrackerTest {
+class SplitAssignmentTrackerTest {
 
 @Test
-public void testRecordIncrementalSplitAssignment() {
+void testRecordIncrementalSplitAssignment() {
 SplitAssignmentTracker tracker = new 
SplitAssignmentTracker<>();
 tracker.recordSplitAssignment(getSplitsAssignment(3, 0));
 tracker.recordSplitAssignment(getSplitsAssignment(2, 6));
@@ -51,7 +48,7 @@ public class SplitAssignmentTrackerTest {
 }
 
 @Test
-public void testOnCheckpoint() throws Exception {
+void testOnCheckpoint() throws Exception {
 final long checkpointId = 123L;
 SplitAssignmentTracker tracker = new 
SplitAssignmentTracker<>();
 tracker.recordSplitAssignment(getSplitsAssignment(3, 0));
@@ -60,16 +57,16 @@ public class SplitAssignmentTrackerTest {
 tracker.onCheckpoint(checkpointId);
 
 // Verify the uncheckpointed assignments.
-assertTrue(tracker.uncheckpointedAssignments().isEmpty());
+assertThat(tracker.uncheckpointedAssignments()).isEmpty();
 
 // verify assignments put into the checkpoints.
 Map>> 
assignmentsByCheckpoints =
 tracker.assignmentsByCheckpointId();
-assertEquals(1, assignmentsByCheckpoints.size());
+assertThat(assignmentsByCheckpoints.size()).isOne();
 
 Map> assignmentForCheckpoint =
 assignmentsByCheckpoints.get(checkpointId);
-assertNotNull(assignmentForCheckpoint);
+assertThat(assignmentForCheckpoint).isNotNull();
 
 verifyAssignment(Arrays.asList("0"), assignmentForCheckpoint.get(0));
 verifyAssignment(Arrays.asList("1", "2"), 
assignmentForCheckpoint.get(1));
@@ -77,7 +74,7 @@ public class SplitAssignmentTrackerTest {
 }
 
 @Test
-public void testOnCheckpointComplete() throws Exception {
+void testOnCheckpointComplete() throws Exception {
 final long checkpointId1 = 100L;
 final long checkpointId2 = 101L;
 SplitAssignmentTracker tracker = new 
SplitAssignmentTracker<>();
@@ -108,7 +105,7 @@ public class SplitAssignmentTrackerTest {
 
 // Complete the first checkpoint.
 tracker.onCheckpointComplete(checkpointId1);
-assertNull(tracker.assignmentsByCheckpointId(checkpointId1));
+assertThat(tracker.assignmentsByCheckpointId(checkpointId1)).isNull();
 verifyAssignment(
 Arrays.asList("3"), 
tracker.assignmentsByCheckpointId(checkpointId2).get(0));
 verifyAssignment(
@@ -116,7 +113,7 @@ public class SplitAssignmentTrackerTest {
 }
 
 @Test
-public void testGetAndRemoveUncheckpointedAssignment() throws Exception {
+void testGetAndRemoveUncheckpointedAssignment() throws Exception {
 final long checkpointId1 = 100L;
 final long checkpointId2 = 101L;
 SplitAssignmentTracker tracker = new 
SplitAssignmentTracker<>();
@@ -136,7 +133,7 @@ public class SplitAssignmentTrackerTest {
 }
 
 @Test
-public void testGetAndRemoveSplitsAfterSomeCheckpoint() throws Exception {
+void testGetAndRemoveSplitsA

(flink) branch master updated (32ab253a044 -> 558ca75da2f)

2024-04-07 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 32ab253a044 [hotfix] Correct the option key for sortPartition
 new 191e6ceb6bd [hotfix][test] Migrate SplitAssignmentTrackerTest to 
Junit5 and AssertJ.
 new 38255652406 [FLINK-33984][runtime] Skip maintain subtaskGatewayMap 
when CheckpointCoordinator is null.
 new 558ca75da2f [FLINK-33984][runtime] Support batch snapshot for 
OperatorCoordinator.

The 3 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:
 .../source/hybrid/HybridSourceSplitEnumerator.java |  4 +-
 .../src/impl/ContinuousFileSplitEnumerator.java|  4 +-
 .../file/src/impl/StaticFileSplitEnumerator.java   |  4 +-
 .../hive/ContinuousHiveSplitEnumerator.java|  4 +-
 .../connector/source/SupportsBatchSnapshot.java| 12 ++--
 .../source/lib/util/IteratorSourceEnumerator.java  |  3 +-
 .../coordination/OperatorCoordinator.java  | 14 +
 .../coordination/OperatorCoordinatorHolder.java| 14 ++---
 .../RecreateOnResetOperatorCoordinator.java| 11 
 .../operators/coordination/SubtaskGatewayImpl.java |  4 +-
 .../source/coordinator/SourceCoordinator.java  | 67 --
 .../coordinator/SourceCoordinatorContext.java  |  4 ++
 .../coordinator/SourceCoordinatorSerdeUtils.java   | 64 +
 .../source/coordinator/SplitAssignmentTracker.java | 22 +++
 .../OperatorCoordinatorHolderTest.java | 48 +++-
 .../source/coordinator/SourceCoordinatorTest.java  | 36 
 .../coordinator/SplitAssignmentTrackerTest.java| 49 +++-
 .../source/coordinator/TestingSplitEnumerator.java |  3 +-
 18 files changed, 324 insertions(+), 43 deletions(-)
 copy 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/listener/CatalogModificationEvent.java
 => 
flink-core/src/main/java/org/apache/flink/api/connector/source/SupportsBatchSnapshot.java
 (71%)



(flink) 03/03: [FLINK-33984][runtime] Support batch snapshot for OperatorCoordinator.

2024-04-07 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 558ca75da2fcec875d1e04a8d75a24fd0ad42ccc
Author: JunRuiLee 
AuthorDate: Fri Mar 1 11:04:04 2024 +0800

[FLINK-33984][runtime] Support batch snapshot for OperatorCoordinator.

This closes #24415.
---
 .../source/hybrid/HybridSourceSplitEnumerator.java |  4 +-
 .../src/impl/ContinuousFileSplitEnumerator.java|  4 +-
 .../file/src/impl/StaticFileSplitEnumerator.java   |  4 +-
 .../hive/ContinuousHiveSplitEnumerator.java|  4 +-
 .../connector/source/SupportsBatchSnapshot.java| 29 ++
 .../source/lib/util/IteratorSourceEnumerator.java  |  3 +-
 .../coordination/OperatorCoordinator.java  | 14 +
 .../coordination/OperatorCoordinatorHolder.java|  3 +-
 .../RecreateOnResetOperatorCoordinator.java| 11 
 .../operators/coordination/SubtaskGatewayImpl.java |  4 +-
 .../source/coordinator/SourceCoordinator.java  | 67 --
 .../coordinator/SourceCoordinatorContext.java  |  4 ++
 .../coordinator/SourceCoordinatorSerdeUtils.java   | 64 +
 .../source/coordinator/SplitAssignmentTracker.java | 22 +++
 .../source/coordinator/SourceCoordinatorTest.java  | 36 
 .../coordinator/SplitAssignmentTrackerTest.java| 22 +++
 .../source/coordinator/TestingSplitEnumerator.java |  3 +-
 17 files changed, 284 insertions(+), 14 deletions(-)

diff --git 
a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java
 
b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java
index 3cba9635d83..48bec9753c5 100644
--- 
a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java
+++ 
b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumerator.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.connector.source.SourceSplit;
 import org.apache.flink.api.connector.source.SplitEnumerator;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
 import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.api.connector.source.SupportsBatchSnapshot;
 import org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits;
 import org.apache.flink.core.io.SimpleVersionedSerializer;
 import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup;
@@ -66,7 +67,8 @@ import java.util.function.BiConsumer;
  * returned splits were processed, delegation to the current underlying 
enumerator resumes.
  */
 public class HybridSourceSplitEnumerator
-implements SplitEnumerator {
+implements SplitEnumerator,
+SupportsBatchSnapshot {
 private static final Logger LOG = 
LoggerFactory.getLogger(HybridSourceSplitEnumerator.class);
 
 private final SplitEnumeratorContext context;
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java
index 8102e0decba..33ca4e45131 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/ContinuousFileSplitEnumerator.java
@@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.connector.source.SourceEvent;
 import org.apache.flink.api.connector.source.SplitEnumerator;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SupportsBatchSnapshot;
 import org.apache.flink.connector.file.src.FileSourceSplit;
 import org.apache.flink.connector.file.src.PendingSplitsCheckpoint;
 import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
@@ -49,7 +50,8 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
 /** A continuously monitoring enumerator. */
 @Internal
 public class ContinuousFileSplitEnumerator
-implements SplitEnumerator> {
+implements SplitEnumerator>,
+SupportsBatchSnapshot {
 
 private static final Logger LOG = 
LoggerFactory.getLogger(ContinuousFileSplitEnumerator.class);
 
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/StaticFileSplitEnumerator.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/impl/StaticFileSplitEnumerator.java
index 140f52053b1..fb56767240b 100644
--- 
a/flink-conn

(flink) branch master updated: [FLINK-33985][runtime] Support obtain all partitions existing in cluster through ShuffleMaster.

2024-04-02 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new a4470966295 [FLINK-33985][runtime] Support obtain all partitions 
existing in cluster through ShuffleMaster.
a4470966295 is described below

commit a44709662956b306fe686623d00358a6b076f637
Author: JunRuiLee 
AuthorDate: Wed Mar 13 16:40:53 2024 +0800

[FLINK-33985][runtime] Support obtain all partitions existing in cluster 
through ShuffleMaster.

This closes #24553.
---
 .../io/network/NettyShuffleEnvironment.java|  7 ++
 .../io/network/partition/ResultPartition.java  |  4 ++
 .../network/partition/ResultPartitionManager.java  | 16 +
 .../partition/TaskExecutorPartitionTracker.java|  8 +++
 .../TaskExecutorPartitionTrackerImpl.java  | 15 +
 .../apache/flink/runtime/jobmaster/JobMaster.java  | 24 +++
 .../flink/runtime/jobmaster/JobMasterGateway.java  |  7 ++
 ...ntext.java => DefaultPartitionWithMetrics.java} | 35 +-
 ...ffleContext.java => DefaultShuffleMetrics.java} | 29 
 .../flink/runtime/shuffle/JobShuffleContext.java   |  6 ++
 .../runtime/shuffle/JobShuffleContextImpl.java |  6 ++
 .../flink/runtime/shuffle/NettyShuffleMaster.java  | 22 ++
 ...uffleContext.java => PartitionWithMetrics.java} | 24 ++-
 .../flink/runtime/shuffle/ShuffleEnvironment.java  | 13 
 .../flink/runtime/shuffle/ShuffleMaster.java   | 13 
 ...{JobShuffleContext.java => ShuffleMetrics.java} | 23 ++-
 .../flink/runtime/taskexecutor/TaskExecutor.java   | 23 +++
 .../runtime/taskexecutor/TaskExecutorGateway.java  | 13 
 .../taskexecutor/partition/PartitionTable.java |  5 ++
 .../partition/ResultPartitionManagerTest.java  | 20 ++
 .../TaskExecutorPartitionTrackerImplTest.java  | 23 +++
 .../TestingTaskExecutorPartitionTracker.java   |  5 ++
 .../flink/runtime/jobmaster/JobMasterTest.java | 78 ++
 .../taskexecutor/TaskExecutorSubmissionTest.java   | 73 
 .../taskexecutor/TestingTaskExecutorGateway.java   | 14 
 .../TestingTaskExecutorGatewayBuilder.java | 13 
 .../taskexecutor/partition/PartitionTableTest.java |  9 +++
 27 files changed, 458 insertions(+), 70 deletions(-)

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 f3ae8ff7d2b..c04678c3d4f 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
@@ -45,6 +45,7 @@ import 
org.apache.flink.runtime.shuffle.NettyShuffleDescriptor;
 import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
 import org.apache.flink.runtime.shuffle.ShuffleEnvironment;
 import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext;
+import org.apache.flink.runtime.shuffle.ShuffleMetrics;
 import 
org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration;
 import org.apache.flink.util.Preconditions;
 
@@ -198,6 +199,12 @@ public class NettyShuffleEnvironment
 return resultPartitionManager.getUnreleasedPartitions();
 }
 
+@Override
+public Optional 
getMetricsIfPartitionOccupyingLocalResource(
+ResultPartitionID partitionId) {
+return resultPartitionManager.getMetricsOfPartition(partitionId);
+}
+
 // 

 //  Create Output Writers and Input Readers
 // 

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
index 6361e07ce3a..6cbcfc0c598 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java
@@ -215,6 +215,10 @@ public abstract class ResultPartition implements 
ResultPartitionWriter {
 return partitionType;
 }
 
+public ResultPartitionBytesCounter getResultPartitionBytes() {
+return resultPartitionBytes;
+}
+
 // 
 
 @Override
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 5013cd7075f..7dd952c3286 100644
--- 
a/flink-runt

(flink) branch master updated (f38d8ca43f6 -> ec1311c8eb8)

2024-04-02 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from f38d8ca43f6 [FLINK-34978][State] Introduce Asynchronous State APIs 
(#24595)
 add ec1311c8eb8 [FLINK-33982][core] Introduce new config options for Batch 
Job Recovery

No new revisions were added by this update.

Summary of changes:
 .../generated/batch_execution_configuration.html   | 18 
 .../generated/expert_scheduling_section.html   | 30 
 .../flink/configuration/BatchExecutionOptions.java | 29 
 .../flink/configuration/JobEventStoreOptions.java  | 53 ++
 4 files changed, 130 insertions(+)
 create mode 100644 
flink-core/src/main/java/org/apache/flink/configuration/JobEventStoreOptions.java



(flink) 02/02: [FLINK-32513][core] Add predecessor caching

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit 940b3bbda5b10abe3a41d60467d33fd424c7dae6
Author: Jeyhun Karimov 
AuthorDate: Sun Mar 10 15:56:02 2024 +0100

[FLINK-32513][core] Add predecessor caching

This closes #24475.
---
 .../org/apache/flink/api/dag/Transformation.java   |  28 +++-
 .../apache/flink/api/dag/TransformationTest.java   |   2 +-
 .../AbstractBroadcastStateTransformation.java  |  12 +-
 .../AbstractMultipleInputTransformation.java   |  12 +-
 .../api/transformations/CacheTransformation.java   |   2 +-
 .../transformations/CoFeedbackTransformation.java  |   2 +-
 .../transformations/FeedbackTransformation.java|   2 +-
 .../transformations/LegacySinkTransformation.java  |   2 +-
 .../LegacySourceTransformation.java|   2 +-
 .../transformations/OneInputTransformation.java|   2 +-
 .../transformations/PartitionTransformation.java   |   2 +-
 .../api/transformations/ReduceTransformation.java  |   2 +-
 .../transformations/SideOutputTransformation.java  |   2 +-
 .../api/transformations/SinkTransformation.java|   2 +-
 .../api/transformations/SourceTransformation.java  |   2 +-
 .../TimestampsAndWatermarksTransformation.java |   2 +-
 .../transformations/TwoInputTransformation.java|  18 ++-
 .../api/transformations/UnionTransformation.java   |  16 +-
 .../api/graph/StreamGraphGeneratorTest.java|   2 +-
 .../GetTransitivePredecessorsTest.java | 162 +
 .../TableOperatorWrapperGeneratorTest.java |   2 +-
 21 files changed, 237 insertions(+), 41 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java 
b/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
index a0448697dd1..07c64907c82 100644
--- a/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
+++ b/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
@@ -19,6 +19,7 @@
 package org.apache.flink.api.dag;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.operators.ResourceSpec;
@@ -161,6 +162,13 @@ public abstract class Transformation {
 private final Map 
managedMemoryOperatorScopeUseCaseWeights =
 new HashMap<>();
 
+/**
+ * This map is a cache that stores transitive predecessors and used in 
{@code
+ * getTransitivePredecessors()}.
+ */
+private final Map, List>> 
predecessorsCache =
+new HashMap<>();
+
 /** Slot scope use cases that this transformation needs managed memory 
for. */
 private final Set managedMemorySlotScopeUseCases = 
new HashSet<>();
 
@@ -230,6 +238,12 @@ public abstract class Transformation {
 return name;
 }
 
+/** Returns the predecessorsCache of this {@code Transformation}. */
+@VisibleForTesting
+Map, List>> getPredecessorsCache() {
+return predecessorsCache;
+}
+
 /** Changes the description of this {@code Transformation}. */
 public void setDescription(String description) {
 this.description = Preconditions.checkNotNull(description);
@@ -578,7 +592,19 @@ public abstract class Transformation {
  *
  * @return The list of transitive predecessors.
  */
-public abstract List> getTransitivePredecessors();
+protected abstract List> 
getTransitivePredecessorsInternal();
+
+/**
+ * Returns all transitive predecessor {@code Transformation}s of this 
{@code Transformation}.
+ * This is, for example, used when determining whether a feedback edge of 
an iteration actually
+ * has the iteration head as a predecessor. This method is just a wrapper 
on top of {@code
+ * getTransitivePredecessorsInternal} method with public access. It uses 
caching internally.
+ *
+ * @return The list of transitive predecessors.
+ */
+public final List> getTransitivePredecessors() {
+return predecessorsCache.computeIfAbsent(this, key -> 
getTransitivePredecessorsInternal());
+}
 
 /**
  * Returns the {@link Transformation transformations} that are the 
immediate predecessors of the
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java 
b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
index ced452cd14f..c200fac33fc 100644
--- a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
@@ -142,7 +142,7 @@ class TransformationTest {
 }
 
 @Override
-public List> getTransitivePredecessors() {
+prot

(flink) 01/02: [hotfix][tests] Migrate TransformationTest to Junit5

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit 906f860cb6f5a9a70984354e2341a6a3bd855cdc
Author: Jeyhun Karimov 
AuthorDate: Sat Mar 16 10:04:00 2024 +0100

[hotfix][tests] Migrate TransformationTest to Junit5
---
 .../apache/flink/api/dag/TransformationTest.java   | 75 +-
 1 file changed, 43 insertions(+), 32 deletions(-)

diff --git 
a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java 
b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
index c14ab0d8e3d..ced452cd14f 100644
--- a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
@@ -22,10 +22,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.core.memory.ManagedMemoryUseCase;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -33,23 +32,21 @@ import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link Transformation}. */
-public class TransformationTest extends TestLogger {
+class TransformationTest {
 
 private Transformation transformation;
 
-@Before
+@BeforeEach
 public void setUp() {
 transformation = new TestTransformation<>("t", null, 1);
 }
 
 @Test
-public void testGetNewNodeIdIsThreadSafe() throws Exception {
+void testGetNewNodeIdIsThreadSafe() throws Exception {
 final int numThreads = 10;
 final int numIdsPerThread = 100;
 
@@ -84,43 +81,57 @@ public class TransformationTest extends TestLogger {
 final Set deduplicatedIds =
 
idLists.stream().flatMap(List::stream).collect(Collectors.toSet());
 
-assertEquals(numThreads * numIdsPerThread, deduplicatedIds.size());
+assertThat(numThreads * 
numIdsPerThread).isEqualTo(deduplicatedIds.size());
 }
 
 @Test
-public void testDeclareManagedMemoryUseCase() {
+void testDeclareManagedMemoryUseCase() {
 transformation.declareManagedMemoryUseCaseAtOperatorScope(
 ManagedMemoryUseCase.OPERATOR, 123);
 
transformation.declareManagedMemoryUseCaseAtSlotScope(ManagedMemoryUseCase.STATE_BACKEND);
 assertThat(
-transformation
-.getManagedMemoryOperatorScopeUseCaseWeights()
-.get(ManagedMemoryUseCase.OPERATOR),
-is(123));
-assertThat(
-transformation.getManagedMemorySlotScopeUseCases(),
-contains(ManagedMemoryUseCase.STATE_BACKEND));
+transformation
+.getManagedMemoryOperatorScopeUseCaseWeights()
+.get(ManagedMemoryUseCase.OPERATOR))
+.isEqualTo(123);
+assertThat(transformation.getManagedMemorySlotScopeUseCases())
+.contains(ManagedMemoryUseCase.STATE_BACKEND);
 }
 
-@Test(expected = IllegalArgumentException.class)
-public void testDeclareManagedMemoryOperatorScopeUseCaseFailWrongScope() {
-
transformation.declareManagedMemoryUseCaseAtOperatorScope(ManagedMemoryUseCase.PYTHON,
 123);
+@Test
+void testDeclareManagedMemoryOperatorScopeUseCaseFailWrongScope() {
+assertThatThrownBy(
+() ->
+
transformation.declareManagedMemoryUseCaseAtOperatorScope(
+ManagedMemoryUseCase.PYTHON, 123))
+.isInstanceOf(IllegalArgumentException.class);
 }
 
-@Test(expected = IllegalArgumentException.class)
-public void testDeclareManagedMemoryOperatorScopeUseCaseFailZeroWeight() {
-
transformation.declareManagedMemoryUseCaseAtOperatorScope(ManagedMemoryUseCase.OPERATOR,
 0);
+@Test
+void testDeclareManagedMemoryOperatorScopeUseCaseFailZeroWeight() {
+assertThatThrownBy(
+() ->
+
transformation.declareManagedMemoryUseCaseAtOperatorScope(
+ManagedMemoryUseCase.OPERATOR, 0))
+.isInstanceOf(IllegalArgumentException.class);
 }

(flink) branch release-1.18 updated (0e54a36249a -> 940b3bbda5b)

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


from 0e54a36249a [hotfix] Fix maven property typo in root pom.xml
 new 906f860cb6f [hotfix][tests] Migrate TransformationTest to Junit5
 new 940b3bbda5b [FLINK-32513][core] Add predecessor caching

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:
 .../org/apache/flink/api/dag/Transformation.java   |  28 +++-
 .../apache/flink/api/dag/TransformationTest.java   |  77 +-
 .../AbstractBroadcastStateTransformation.java  |  12 +-
 .../AbstractMultipleInputTransformation.java   |  12 +-
 .../api/transformations/CacheTransformation.java   |   2 +-
 .../transformations/CoFeedbackTransformation.java  |   2 +-
 .../transformations/FeedbackTransformation.java|   2 +-
 .../transformations/LegacySinkTransformation.java  |   2 +-
 .../LegacySourceTransformation.java|   2 +-
 .../transformations/OneInputTransformation.java|   2 +-
 .../transformations/PartitionTransformation.java   |   2 +-
 .../api/transformations/ReduceTransformation.java  |   2 +-
 .../transformations/SideOutputTransformation.java  |   2 +-
 .../api/transformations/SinkTransformation.java|   2 +-
 .../api/transformations/SourceTransformation.java  |   2 +-
 .../TimestampsAndWatermarksTransformation.java |   2 +-
 .../transformations/TwoInputTransformation.java|  18 ++-
 .../api/transformations/UnionTransformation.java   |  16 +-
 .../api/graph/StreamGraphGeneratorTest.java|   2 +-
 .../GetTransitivePredecessorsTest.java | 162 +
 .../TableOperatorWrapperGeneratorTest.java |   2 +-
 21 files changed, 280 insertions(+), 73 deletions(-)
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/transformations/GetTransitivePredecessorsTest.java



(flink) 02/02: [FLINK-32513][core] Add predecessor caching

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit 5ec4bf2f18168001b5cbb9012f331d3405228516
Author: Jeyhun Karimov 
AuthorDate: Sun Mar 10 15:56:02 2024 +0100

[FLINK-32513][core] Add predecessor caching

This closes #24475.
---
 .../org/apache/flink/api/dag/Transformation.java   |  27 +++-
 .../apache/flink/api/dag/TransformationTest.java   |   2 +-
 .../AbstractBroadcastStateTransformation.java  |  12 +-
 .../AbstractMultipleInputTransformation.java   |  12 +-
 .../api/transformations/CacheTransformation.java   |   2 +-
 .../transformations/CoFeedbackTransformation.java  |   2 +-
 .../transformations/FeedbackTransformation.java|   2 +-
 .../transformations/LegacySinkTransformation.java  |   2 +-
 .../LegacySourceTransformation.java|   2 +-
 .../transformations/OneInputTransformation.java|   2 +-
 .../transformations/PartitionTransformation.java   |   2 +-
 .../api/transformations/ReduceTransformation.java  |   2 +-
 .../transformations/SideOutputTransformation.java  |   2 +-
 .../api/transformations/SinkTransformation.java|   2 +-
 .../api/transformations/SourceTransformation.java  |   2 +-
 .../SourceTransformationWrapper.java   |   2 +-
 .../TimestampsAndWatermarksTransformation.java |   2 +-
 .../transformations/TwoInputTransformation.java|  18 ++-
 .../api/transformations/UnionTransformation.java   |  16 +-
 .../api/graph/StreamGraphGeneratorTest.java|   2 +-
 .../GetTransitivePredecessorsTest.java | 162 +
 .../TableOperatorWrapperGeneratorTest.java |   2 +-
 22 files changed, 237 insertions(+), 42 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java 
b/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
index 6256f9624f6..f2033a1fd2b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
+++ b/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java
@@ -162,6 +162,13 @@ public abstract class Transformation {
 private final Map 
managedMemoryOperatorScopeUseCaseWeights =
 new HashMap<>();
 
+/**
+ * This map is a cache that stores transitive predecessors and used in 
{@code
+ * getTransitivePredecessors()}.
+ */
+private final Map, List>> 
predecessorsCache =
+new HashMap<>();
+
 /** Slot scope use cases that this transformation needs managed memory 
for. */
 private final Set managedMemorySlotScopeUseCases = 
new HashSet<>();
 
@@ -231,6 +238,12 @@ public abstract class Transformation {
 return name;
 }
 
+/** Returns the predecessorsCache of this {@code Transformation}. */
+@VisibleForTesting
+Map, List>> getPredecessorsCache() {
+return predecessorsCache;
+}
+
 /** Changes the description of this {@code Transformation}. */
 public void setDescription(String description) {
 this.description = Preconditions.checkNotNull(description);
@@ -579,7 +592,19 @@ public abstract class Transformation {
  *
  * @return The list of transitive predecessors.
  */
-public abstract List> getTransitivePredecessors();
+protected abstract List> 
getTransitivePredecessorsInternal();
+
+/**
+ * Returns all transitive predecessor {@code Transformation}s of this 
{@code Transformation}.
+ * This is, for example, used when determining whether a feedback edge of 
an iteration actually
+ * has the iteration head as a predecessor. This method is just a wrapper 
on top of {@code
+ * getTransitivePredecessorsInternal} method with public access. It uses 
caching internally.
+ *
+ * @return The list of transitive predecessors.
+ */
+public final List> getTransitivePredecessors() {
+return predecessorsCache.computeIfAbsent(this, key -> 
getTransitivePredecessorsInternal());
+}
 
 /**
  * Returns the {@link Transformation transformations} that are the 
immediate predecessors of the
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java 
b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
index ced452cd14f..c200fac33fc 100644
--- a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
@@ -142,7 +142,7 @@ class TransformationTest {
 }
 
 @Override
-public List> getTransitivePredecessors() {
+protected List> getTransitivePredecessorsInternal() {
 return Collections.emptyList();
 }
 
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractBroadcastStateTransformation.java
 
b/flink-streaming-java/src/main/jav

(flink) branch release-1.19 updated (6eeae5fe6c9 -> 5ec4bf2f181)

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


from 6eeae5fe6c9 Update japicmp configuration for 1.19.0
 new 94c32618ac7 [hotfix][tests] Migrate TransformationTest to Junit5
 new 5ec4bf2f181 [FLINK-32513][core] Add predecessor caching

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:
 .../org/apache/flink/api/dag/Transformation.java   |  27 +++-
 .../apache/flink/api/dag/TransformationTest.java   |  77 +-
 .../AbstractBroadcastStateTransformation.java  |  12 +-
 .../AbstractMultipleInputTransformation.java   |  12 +-
 .../api/transformations/CacheTransformation.java   |   2 +-
 .../transformations/CoFeedbackTransformation.java  |   2 +-
 .../transformations/FeedbackTransformation.java|   2 +-
 .../transformations/LegacySinkTransformation.java  |   2 +-
 .../LegacySourceTransformation.java|   2 +-
 .../transformations/OneInputTransformation.java|   2 +-
 .../transformations/PartitionTransformation.java   |   2 +-
 .../api/transformations/ReduceTransformation.java  |   2 +-
 .../transformations/SideOutputTransformation.java  |   2 +-
 .../api/transformations/SinkTransformation.java|   2 +-
 .../api/transformations/SourceTransformation.java  |   2 +-
 .../SourceTransformationWrapper.java   |   2 +-
 .../TimestampsAndWatermarksTransformation.java |   2 +-
 .../transformations/TwoInputTransformation.java|  18 ++-
 .../api/transformations/UnionTransformation.java   |  16 +-
 .../api/graph/StreamGraphGeneratorTest.java|   2 +-
 .../GetTransitivePredecessorsTest.java | 162 +
 .../TableOperatorWrapperGeneratorTest.java |   2 +-
 22 files changed, 280 insertions(+), 74 deletions(-)
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/transformations/GetTransitivePredecessorsTest.java



(flink) 01/02: [hotfix][tests] Migrate TransformationTest to Junit5

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit 94c32618ac7211fe19a53b4124dc26288fdf944a
Author: Jeyhun Karimov 
AuthorDate: Sat Mar 16 10:04:00 2024 +0100

[hotfix][tests] Migrate TransformationTest to Junit5
---
 .../apache/flink/api/dag/TransformationTest.java   | 75 +-
 1 file changed, 43 insertions(+), 32 deletions(-)

diff --git 
a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java 
b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
index c14ab0d8e3d..ced452cd14f 100644
--- a/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/dag/TransformationTest.java
@@ -22,10 +22,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.core.memory.ManagedMemoryUseCase;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.util.TestLogger;
 
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -33,23 +32,21 @@ import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link Transformation}. */
-public class TransformationTest extends TestLogger {
+class TransformationTest {
 
 private Transformation transformation;
 
-@Before
+@BeforeEach
 public void setUp() {
 transformation = new TestTransformation<>("t", null, 1);
 }
 
 @Test
-public void testGetNewNodeIdIsThreadSafe() throws Exception {
+void testGetNewNodeIdIsThreadSafe() throws Exception {
 final int numThreads = 10;
 final int numIdsPerThread = 100;
 
@@ -84,43 +81,57 @@ public class TransformationTest extends TestLogger {
 final Set deduplicatedIds =
 
idLists.stream().flatMap(List::stream).collect(Collectors.toSet());
 
-assertEquals(numThreads * numIdsPerThread, deduplicatedIds.size());
+assertThat(numThreads * 
numIdsPerThread).isEqualTo(deduplicatedIds.size());
 }
 
 @Test
-public void testDeclareManagedMemoryUseCase() {
+void testDeclareManagedMemoryUseCase() {
 transformation.declareManagedMemoryUseCaseAtOperatorScope(
 ManagedMemoryUseCase.OPERATOR, 123);
 
transformation.declareManagedMemoryUseCaseAtSlotScope(ManagedMemoryUseCase.STATE_BACKEND);
 assertThat(
-transformation
-.getManagedMemoryOperatorScopeUseCaseWeights()
-.get(ManagedMemoryUseCase.OPERATOR),
-is(123));
-assertThat(
-transformation.getManagedMemorySlotScopeUseCases(),
-contains(ManagedMemoryUseCase.STATE_BACKEND));
+transformation
+.getManagedMemoryOperatorScopeUseCaseWeights()
+.get(ManagedMemoryUseCase.OPERATOR))
+.isEqualTo(123);
+assertThat(transformation.getManagedMemorySlotScopeUseCases())
+.contains(ManagedMemoryUseCase.STATE_BACKEND);
 }
 
-@Test(expected = IllegalArgumentException.class)
-public void testDeclareManagedMemoryOperatorScopeUseCaseFailWrongScope() {
-
transformation.declareManagedMemoryUseCaseAtOperatorScope(ManagedMemoryUseCase.PYTHON,
 123);
+@Test
+void testDeclareManagedMemoryOperatorScopeUseCaseFailWrongScope() {
+assertThatThrownBy(
+() ->
+
transformation.declareManagedMemoryUseCaseAtOperatorScope(
+ManagedMemoryUseCase.PYTHON, 123))
+.isInstanceOf(IllegalArgumentException.class);
 }
 
-@Test(expected = IllegalArgumentException.class)
-public void testDeclareManagedMemoryOperatorScopeUseCaseFailZeroWeight() {
-
transformation.declareManagedMemoryUseCaseAtOperatorScope(ManagedMemoryUseCase.OPERATOR,
 0);
+@Test
+void testDeclareManagedMemoryOperatorScopeUseCaseFailZeroWeight() {
+assertThatThrownBy(
+() ->
+
transformation.declareManagedMemoryUseCaseAtOperatorScope(
+ManagedMemoryUseCase.OPERATOR, 0))
+.isInstanceOf(IllegalArgumentException.class);
 }

(flink) branch master updated (779459168c4 -> 8dcb0ae9063)

2024-03-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 779459168c4 [FLINK-34516][checkpoint] Rename the new introduced API to 
get/setCheckpointingConsistencyMode
 add 8c20e0f8053 [hotfix][tests] Migrate TransformationTest to Junit5
 add 8dcb0ae9063 [FLINK-32513][core] Add predecessor caching

No new revisions were added by this update.

Summary of changes:
 .../org/apache/flink/api/dag/Transformation.java   |  28 +++-
 .../apache/flink/api/dag/TransformationTest.java   |  77 +-
 .../AbstractBroadcastStateTransformation.java  |  12 +-
 .../AbstractMultipleInputTransformation.java   |  12 +-
 .../api/transformations/CacheTransformation.java   |   2 +-
 .../transformations/CoFeedbackTransformation.java  |   2 +-
 .../transformations/FeedbackTransformation.java|   2 +-
 .../transformations/LegacySinkTransformation.java  |   2 +-
 .../LegacySourceTransformation.java|   2 +-
 .../transformations/OneInputTransformation.java|   2 +-
 .../transformations/PartitionTransformation.java   |   2 +-
 .../api/transformations/ReduceTransformation.java  |   2 +-
 .../transformations/SideOutputTransformation.java  |   2 +-
 .../api/transformations/SinkTransformation.java|   2 +-
 .../api/transformations/SourceTransformation.java  |   2 +-
 .../SourceTransformationWrapper.java   |   2 +-
 .../TimestampsAndWatermarksTransformation.java |   2 +-
 .../transformations/TwoInputTransformation.java|  18 ++-
 .../api/transformations/UnionTransformation.java   |  16 +-
 .../api/graph/StreamGraphGeneratorTest.java|   2 +-
 .../GetTransitivePredecessorsTest.java | 162 +
 .../TableOperatorWrapperGeneratorTest.java |   2 +-
 22 files changed, 281 insertions(+), 74 deletions(-)
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/transformations/GetTransitivePredecessorsTest.java



(flink) branch master updated: [FLINK-34731][runtime] Remove SpeculativeScheduler and incorporate its features into AdaptiveBatchScheduler.

2024-03-21 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new cf0d75c4bb3 [FLINK-34731][runtime] Remove SpeculativeScheduler and 
incorporate its features into AdaptiveBatchScheduler.
cf0d75c4bb3 is described below

commit cf0d75c4bb324825a057dc72243bb6a2046f8479
Author: JunRuiLee 
AuthorDate: Tue Mar 19 15:27:52 2024 +0800

[FLINK-34731][runtime] Remove SpeculativeScheduler and incorporate its 
features into AdaptiveBatchScheduler.

This closes #24524.
---
 .../adaptivebatch/AdaptiveBatchScheduler.java  | 115 +-
 .../AdaptiveBatchSchedulerFactory.java | 185 +---
 ...ava => DefaultSpeculativeExecutionHandler.java} | 245 ++---
 .../DummySpeculativeExecutionHandler.java  |  72 ++
 .../adaptivebatch/SpeculativeExecutionHandler.java |  82 +++
 .../runtime/scheduler/DefaultSchedulerBuilder.java |  88 +++-
 .../runtime/scheduler/DefaultSchedulerTest.java|  11 +
 ...ulerTest.java => SpeculativeExecutionTest.java} |  97 
 .../ExecutionTimeBasedSlowTaskDetectorTest.java|   4 +-
 ...ITCase.java => SpeculativeExecutionITCase.java} |   8 +-
 10 files changed, 549 insertions(+), 358 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
index 83fb50f1514..3d95135e4e9 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
@@ -23,10 +23,12 @@ import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobStatus;
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BatchExecutionOptions;
 import org.apache.flink.configuration.Configuration;
 import 
org.apache.flink.configuration.JobManagerOptions.HybridPartitionDataConsumeConstraint;
 import org.apache.flink.core.failure.FailureEnricher;
 import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.blocklist.BlocklistOperations;
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
 import org.apache.flink.runtime.checkpoint.CheckpointsCleaner;
 import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
@@ -44,6 +46,7 @@ import 
org.apache.flink.runtime.executiongraph.MarkPartitionFinishedStrategy;
 import org.apache.flink.runtime.executiongraph.ParallelismAndInputInfos;
 import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
 import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.failover.FailureHandlingResult;
 import 
org.apache.flink.runtime.executiongraph.failover.RestartBackoffTimeStrategy;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -75,6 +78,8 @@ import org.apache.flink.util.concurrent.ScheduledExecutor;
 
 import org.slf4j.Logger;
 
+import javax.annotation.Nullable;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -113,6 +118,8 @@ public class AdaptiveBatchScheduler extends 
DefaultScheduler {
 private final Map>
 sourceParallelismFuturesByJobVertexId;
 
+private final SpeculativeExecutionHandler speculativeExecutionHandler;
+
 public AdaptiveBatchScheduler(
 final Logger log,
 final JobGraph jobGraph,
@@ -138,7 +145,8 @@ public class AdaptiveBatchScheduler extends 
DefaultScheduler {
 final ShuffleMaster shuffleMaster,
 final Time rpcTimeout,
 final VertexParallelismAndInputInfosDecider 
vertexParallelismAndInputInfosDecider,
-int defaultMaxParallelism,
+final int defaultMaxParallelism,
+final BlocklistOperations blocklistOperations,
 final HybridPartitionDataConsumeConstraint 
hybridPartitionDataConsumeConstraint,
 final Map forwardGroupsByJobVertexId)
 throws Exception {
@@ -183,10 +191,40 @@ public class AdaptiveBatchScheduler extends 
DefaultScheduler {
 this.hybridPartitionDataConsumeConstraint = 
hybridPartitionDataConsumeConstraint;
 
 this.sourceParallelismFuturesByJobVertexId = new HashMap<>();
+
+speculativeExecutionHandler =
+createSpeculativeExecutionHandler(
+log, jobMasterConfiguration, executionVertexVersioner, 
blocklistOperations);
+}
+
+private SpeculativeExecutionHandler createSpeculati

(flink) branch release-1.19 updated: [FLINK-34725][dist] Fix wrong config file dir when `config-parser-utils.sh` script attempt to retrieve the Java Home.

2024-03-21 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.19 by this push:
 new f53c5628e43 [FLINK-34725][dist] Fix wrong config file dir when 
`config-parser-utils.sh` script attempt to retrieve the Java Home.
f53c5628e43 is described below

commit f53c5628e43777b4b924ec81224acc3df938800a
Author: JunRuiLee 
AuthorDate: Tue Mar 19 17:04:51 2024 +0800

[FLINK-34725][dist] Fix wrong config file dir when `config-parser-utils.sh` 
script attempt to retrieve the Java Home.

This closes #24527.
---
 flink-dist/src/main/flink-bin/bin/config-parser-utils.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh 
b/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh
index 6e7f41b94d3..e714428583a 100755
--- a/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh
+++ b/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh
@@ -25,7 +25,7 @@ if [ "$#" -lt 3 ]; then
 fi
 
 source "$2"/bash-java-utils.sh
-setJavaRun "$FLINK_CONF_DIR"
+setJavaRun "$1"
 
 ARGS=("${@:1}")
 result=$(updateAndGetFlinkConfiguration "${ARGS[@]}")



(flink) branch master updated: [FLINK-34725][dist] Fix wrong config file dir when `config-parser-utils.sh` script attempt to retrieve the Java Home.

2024-03-21 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 3f4a80989fe [FLINK-34725][dist] Fix wrong config file dir when 
`config-parser-utils.sh` script attempt to retrieve the Java Home.
3f4a80989fe is described below

commit 3f4a80989fe7243983926f09fac2283f6fa63693
Author: JunRuiLee 
AuthorDate: Tue Mar 19 17:04:51 2024 +0800

[FLINK-34725][dist] Fix wrong config file dir when `config-parser-utils.sh` 
script attempt to retrieve the Java Home.

This closes #24527.
---
 flink-dist/src/main/flink-bin/bin/config-parser-utils.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh 
b/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh
index 6e7f41b94d3..e714428583a 100755
--- a/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh
+++ b/flink-dist/src/main/flink-bin/bin/config-parser-utils.sh
@@ -25,7 +25,7 @@ if [ "$#" -lt 3 ]; then
 fi
 
 source "$2"/bash-java-utils.sh
-setJavaRun "$FLINK_CONF_DIR"
+setJavaRun "$1"
 
 ARGS=("${@:1}")
 result=$(updateAndGetFlinkConfiguration "${ARGS[@]}")



(flink) branch release-1.19 updated: Revert "[FLINK-33532][network] Move the serialization of ShuffleDescriptorGroup out of the RPC main thread]"

2024-03-06 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.19 by this push:
 new 837f8e58485 Revert "[FLINK-33532][network] Move the serialization of 
ShuffleDescriptorGroup out of the RPC main thread]"
837f8e58485 is described below

commit 837f8e584850bdcbc586a54f58e3fe953a816be8
Author: caodizhou 
AuthorDate: Wed Mar 6 14:11:56 2024 +0800

Revert "[FLINK-33532][network] Move the serialization of 
ShuffleDescriptorGroup out of the RPC main thread]"

This reverts commit d18a4bfe596fc580f8280750fa3bfa22007671d9.

(cherry picked from commit 7a709bf323c1cce3440887fe937311bae119aab0)
---
 .../org/apache/flink/runtime/blob/BlobWriter.java  | 11 ++--
 .../deployment/CachedShuffleDescriptors.java   |  2 +-
 .../deployment/InputGateDeploymentDescriptor.java  | 41 ++-
 .../deployment/TaskDeploymentDescriptor.java   | 19 ---
 .../TaskDeploymentDescriptorFactory.java   | 58 --
 .../deployment/CachedShuffleDescriptorsTest.java   | 30 ++-
 .../TaskDeploymentDescriptorTestUtils.java |  9 ++--
 .../partition/consumer/SingleInputGateTest.java|  6 ++-
 8 files changed, 83 insertions(+), 93 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobWriter.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobWriter.java
index 555cccfb7ca..2d5292b42cb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobWriter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobWriter.java
@@ -28,7 +28,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Optional;
 
 /** BlobWriter is used to upload data to the BLOB store. */
 public interface BlobWriter {
@@ -103,13 +102,11 @@ public interface BlobWriter {
 if (serializedValue.getByteArray().length < 
blobWriter.getMinOffloadingSize()) {
 return Either.Left(serializedValue);
 } else {
-return offloadWithException(serializedValue, jobId, blobWriter)
-.map(Either::, PermanentBlobKey>Right)
-.orElse(Either.Left(serializedValue));
+return offloadWithException(serializedValue, jobId, blobWriter);
 }
 }
 
-static  Optional offloadWithException(
+static  Either, PermanentBlobKey> 
offloadWithException(
 SerializedValue serializedValue, JobID jobId, BlobWriter 
blobWriter) {
 Preconditions.checkNotNull(serializedValue);
 Preconditions.checkNotNull(jobId);
@@ -117,10 +114,10 @@ public interface BlobWriter {
 try {
 final PermanentBlobKey permanentBlobKey =
 blobWriter.putPermanent(jobId, 
serializedValue.getByteArray());
-return Optional.of(permanentBlobKey);
+return Either.Right(permanentBlobKey);
 } catch (IOException e) {
 LOG.warn("Failed to offload value for job {} to BLOB store.", 
jobId, e);
-return Optional.empty();
+return Either.Left(serializedValue);
 }
 }
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/CachedShuffleDescriptors.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/CachedShuffleDescriptors.java
index 4ddacbd671a..b8e0b44006f 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/CachedShuffleDescriptors.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/CachedShuffleDescriptors.java
@@ -87,7 +87,7 @@ public class CachedShuffleDescriptors {
 new ShuffleDescriptorGroup(
 toBeSerialized.toArray(new 
ShuffleDescriptorAndIndex[0]));
 MaybeOffloaded 
serializedShuffleDescriptorGroup =
-
shuffleDescriptorSerializer.trySerializeAndOffloadShuffleDescriptor(
+
shuffleDescriptorSerializer.serializeAndTryOffloadShuffleDescriptor(
 shuffleDescriptorGroup, numConsumers);
 
 toBeSerialized.clear();
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 4e02c699331..333a91e0a73 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
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.blob.PermanentBlobKey;
 import org.apache.flink.run

(flink) branch master updated: [FLINK-33241][doc] Align config option generation documentation for Flink's config documentation.

2024-02-20 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 8fe005cf132 [FLINK-33241][doc] Align config option generation 
documentation for Flink's config documentation.
8fe005cf132 is described below

commit 8fe005cf1325b7477d7e1808a46bd80798165029
Author: JunRuiLee 
AuthorDate: Tue Feb 20 12:14:43 2024 +0800

[FLINK-33241][doc] Align config option generation documentation for Flink's 
config documentation.
---
 docs/README.md | 13 ++---
 1 file changed, 2 insertions(+), 11 deletions(-)

diff --git a/docs/README.md b/docs/README.md
index bc59ea579eb..025dcdb0a75 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -56,18 +56,9 @@ Replace  with the name of your connector, 
e.g., `elasticsearch`
 
 ## Generate configuration tables
 
-Configuration descriptions are auto generated from code. To trigger the 
generation you need to run in the project root:
+Configuration descriptions are auto generated from code. To trigger the 
generation, you need to run a command in the project root (see [Configuration 
documentation](https://github.com/apache/flink/blob/master/flink-docs/README.md#configuration-documentation).)
 
-```
-mvn -Pgenerate-config-docs install -Dfast -DskipTests
-```
-
-The resulting html files will be written to `layouts/shortcodes/generated`. 
Tables are regenerated each time the command is invoked.
-These tables can be directly included into the documentation:
-
-```
-{{< generated/file_name >}}
-```
+The resulting html files will be written to `layouts/shortcodes/generated`.
 
 # Contribute
 



(flink) 02/03: [FLINK-34247][doc] Update the usage of "env.java.home" in doc.

2024-02-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 04dd91f2b6c830b9ac0e445f72938e3d6f479edd
Author: JunRuiLee 
AuthorDate: Sun Feb 4 16:40:35 2024 +0800

[FLINK-34247][doc] Update the usage of "env.java.home" in doc.
---
 .../docs/deployment/resource-providers/standalone/overview.md  | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git 
a/docs/content.zh/docs/deployment/resource-providers/standalone/overview.md 
b/docs/content.zh/docs/deployment/resource-providers/standalone/overview.md
index 229d028c24c..fd658ba5c4d 100644
--- a/docs/content.zh/docs/deployment/resource-providers/standalone/overview.md
+++ b/docs/content.zh/docs/deployment/resource-providers/standalone/overview.md
@@ -61,7 +61,7 @@ Flink 运行在所有*类 UNIX 环境*下,例如 **Linux**,**Mac OS X** 和
 
 Flink 需要 master 和所有 worker 节点设置 `JAVA_HOME` 环境变量,并指向你的 Java 安装目录。
 
-你可以在 `conf/flink-conf.yaml` 文件中通过 `env.java.home` 配置项来设置此变量。
+你可以在 [Flink 配置文件]({{< ref "docs/deployment/config#flink-configuration-file" 
>}})中通过 `env.java.home` 配置项来设置此变量。 需要注意的是,该配置项必须以 flattened 的格式(即一行键值对格式)在配置文件中。
 
 {{< top >}}
 
@@ -82,11 +82,11 @@ cd flink-*
 
 ### 配置 Flink
 
-在解压完文件后,你需要编辑 *conf/flink-conf.yaml* 文件来为集群配置 Flink。
+在解压完文件后,你需要编辑 [*Flink 配置文件*]({{< ref 
"docs/deployment/config#flink-configuration-file" >}})来为集群配置 Flink。
 
 设置 `jobmanager.rpc.address` 配置项指向 master 节点。你也应该通过设置 
`jobmanager.memory.process.size` 和 `taskmanager.memory.process.size` 配置项来定义 
Flink 允许在每个节点上分配的最大内存值。
 
-这些值的单位是 MB。如果一些 worker 节点上有你想分配到 Flink 系统的多余内存,你可以在这些特定节点的 
*conf/flink-conf.yaml* 文件中重写 `taskmanager.memory.process.size` 或 
`taskmanager.memory.flink.size` 的默认值。
+这些值的单位是 MB。如果一些 worker 节点上有你想分配到 Flink 系统的多余内存,你可以在这些特定节点的 [*Flink 配置文件*]({{< 
ref "docs/deployment/config#flink-configuration-file" >}}) 文件中重写 
`taskmanager.memory.process.size` 或 `taskmanager.memory.flink.size` 的默认值。
 
 最后,你必须提供集群上会被用作为 worker 节点的所有节点列表,也就是运行 TaskManager 的节点。编辑文件 *conf/workers* 
并输入每个 worker 节点的 IP 或主机名。
 
@@ -99,7 +99,7 @@ cd flink-*
 
   
 
-  /path/to/flink/conf/flink-conf.yaml
+  /path/to/flink/conf/config.yaml
 jobmanager.rpc.address: 10.0.0.1
 
   
@@ -192,7 +192,7 @@ By default, the job manager will pick a *random port* for 
inter process communic
 
 ### Example: Standalone Cluster with 2 JobManagers
 
-1. **Configure high availability mode and ZooKeeper quorum** in 
`conf/flink-conf.yaml`:
+1. **Configure high availability mode and ZooKeeper quorum** in [Flink 
configuration file]({{< ref "docs/deployment/config#flink-configuration-file" 
>}}):
 

high-availability.type: zookeeper



(flink) 03/03: [FLINK-34247][doc] Update the usage of flink-conf.yaml in doc.

2024-02-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e9bea09510e18c6143e6e14ca17a894abfaf92bf
Author: JunRuiLee 
AuthorDate: Sun Feb 4 16:47:52 2024 +0800

[FLINK-34247][doc] Update the usage of flink-conf.yaml in doc.

This closes #24251.
---
 .../content.zh/docs/connectors/table/filesystem.md |  4 +--
 .../docs/deployment/advanced/historyserver.md  |  2 +-
 docs/content.zh/docs/deployment/cli.md |  6 ++--
 docs/content.zh/docs/deployment/config.md  |  2 +-
 .../docs/deployment/filesystems/azure.md   |  6 ++--
 docs/content.zh/docs/deployment/filesystems/gcs.md |  6 ++--
 docs/content.zh/docs/deployment/filesystems/oss.md |  6 ++--
 docs/content.zh/docs/deployment/filesystems/s3.md  | 12 +++
 .../content.zh/docs/deployment/ha/kubernetes_ha.md |  2 +-
 docs/content.zh/docs/deployment/ha/zookeeper_ha.md |  4 +--
 .../docs/deployment/memory/mem_migration.md|  8 ++---
 .../content.zh/docs/deployment/metric_reporters.md |  2 +-
 .../resource-providers/native_kubernetes.md|  4 +--
 .../resource-providers/standalone/docker.md| 14 
 .../resource-providers/standalone/kubernetes.md| 32 +-
 .../docs/deployment/resource-providers/yarn.md |  2 +-
 .../docs/deployment/security/security-ssl.md   |  2 +-
 docs/content.zh/docs/deployment/trace_reporters.md |  2 +-
 .../docs/dev/datastream/execution/parallel.md  |  2 +-
 .../datastream/fault-tolerance/checkpointing.md|  2 +-
 .../datastream/fault-tolerance/state_backends.md   |  2 +-
 .../docs/dev/python/environment_variables.md   |  2 +-
 docs/content.zh/docs/dev/table/catalogs.md |  4 +--
 .../dev/table/hive-compatibility/hiveserver2.md|  2 +-
 .../docs/dev/table/sql-gateway/overview.md |  4 +--
 docs/content.zh/docs/dev/table/sqlClient.md|  2 +-
 docs/content.zh/docs/ops/debugging/flame_graphs.md |  2 +-
 docs/content.zh/docs/ops/metrics.md|  4 +--
 docs/content.zh/docs/ops/rest_api.md   |  2 +-
 docs/content.zh/docs/ops/state/state_backends.md   | 22 ++---
 .../docs/ops/state/task_failure_recovery.md|  8 ++---
 .../docs/try-flink/flink-operations-playground.md  |  4 +--
 docs/content/docs/connectors/table/filesystem.md   |  4 +--
 .../docs/deployment/advanced/historyserver.md  |  2 +-
 docs/content/docs/deployment/cli.md|  6 ++--
 docs/content/docs/deployment/config.md |  2 +-
 docs/content/docs/deployment/filesystems/azure.md  |  6 ++--
 docs/content/docs/deployment/filesystems/gcs.md|  6 ++--
 docs/content/docs/deployment/filesystems/oss.md|  6 ++--
 docs/content/docs/deployment/filesystems/s3.md | 12 +++
 docs/content/docs/deployment/ha/kubernetes_ha.md   |  2 +-
 docs/content/docs/deployment/ha/zookeeper_ha.md|  4 +--
 .../docs/deployment/memory/mem_migration.md| 12 +++
 docs/content/docs/deployment/metric_reporters.md   |  2 +-
 .../resource-providers/native_kubernetes.md|  4 +--
 .../resource-providers/standalone/docker.md| 14 
 .../resource-providers/standalone/kubernetes.md| 38 +++---
 .../resource-providers/standalone/overview.md  |  4 +--
 .../docs/deployment/resource-providers/yarn.md |  2 +-
 .../docs/deployment/security/security-ssl.md   |  2 +-
 docs/content/docs/deployment/trace_reporters.md|  2 +-
 .../docs/dev/datastream/execution/parallel.md  |  2 +-
 .../datastream/fault-tolerance/checkpointing.md|  2 +-
 .../datastream/fault-tolerance/state_backends.md   |  2 +-
 .../docs/dev/python/environment_variables.md   |  2 +-
 docs/content/docs/dev/table/catalogs.md|  4 +--
 docs/content/docs/dev/table/config.md  |  4 +--
 .../docs/dev/table/sql-gateway/hiveserver2.md  |  2 +-
 .../content/docs/dev/table/sql-gateway/overview.md |  4 +--
 docs/content/docs/dev/table/sqlClient.md   |  2 +-
 docs/content/docs/ops/debugging/flame_graphs.md|  2 +-
 docs/content/docs/ops/metrics.md   |  4 +--
 docs/content/docs/ops/rest_api.md  |  2 +-
 docs/content/docs/ops/state/savepoints.md  |  2 +-
 docs/content/docs/ops/state/state_backends.md  | 24 +++---
 .../docs/ops/state/task_failure_recovery.md|  8 ++---
 .../docs/try-flink/flink-operations-playground.md  |  2 +-
 docs/content/docs/try-flink/local_installation.md  |  2 +-
 68 files changed, 189 insertions(+), 189 deletions(-)

diff --git a/docs/content.zh/docs/connectors/table/filesystem.md 
b/docs/content.zh/docs/connectors/table/filesystem.md
index b28a41d9467..17d05045a20 100644
--- a/docs/content.zh/docs/connectors/table/filesystem.md
+++ b/docs/content.zh/docs/connectors/table/filesystem.md
@@ -241,8 +241,8 @@ CREATE TABLE MyUserTableWithFilepath (
 
 **注意:** 对于 bulk

(flink) 01/03: [FLINK-34247][doc] Add documentation of new Flink configuration file config.yaml.

2024-02-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 5b61baadd02ccdfa702834e2e63aeb8d1d9e1250
Author: JunRuiLee 
AuthorDate: Sun Feb 4 16:38:55 2024 +0800

[FLINK-34247][doc] Add documentation of new Flink configuration file 
config.yaml.
---
 docs/content.zh/docs/deployment/config.md   |  88 ++-
 docs/content/docs/deployment/config.md  |  88 ++-
 docs/layouts/shortcodes/config_file.html| 126 
 docs/layouts/shortcodes/config_file_zh.html | 126 
 4 files changed, 424 insertions(+), 4 deletions(-)

diff --git a/docs/content.zh/docs/deployment/config.md 
b/docs/content.zh/docs/deployment/config.md
index 35b85240d03..d3119d4cf8f 100644
--- a/docs/content.zh/docs/deployment/config.md
+++ b/docs/content.zh/docs/deployment/config.md
@@ -27,16 +27,100 @@ under the License.
 
 # 配置参数
 
-All configuration is done in `conf/flink-conf.yaml`, which is expected to be a 
flat collection of [YAML key value 
pairs](http://www.yaml.org/spec/1.2/spec.html) with format `key: value`.
+All configuration can be set in Flink configuration file in the `conf/` 
directory (see [Flink Configuration File section](#flink-configuration-file)).
 
 The configuration is parsed and evaluated when the Flink processes are 
started. Changes to the configuration file require restarting the relevant 
processes.
 
-The out of the box configuration will use your default Java installation. You 
can manually set the environment variable `JAVA_HOME` or the configuration key 
`env.java.home` in `conf/flink-conf.yaml` if you want to manually override the 
Java runtime to use.
+The out of the box configuration will use your default Java installation. You 
can manually set the environment variable `JAVA_HOME` or the configuration key 
`env.java.home` in Flink configuration file if you want to manually override 
the Java runtime to use.
 
 You can specify a different configuration directory location by defining the 
`FLINK_CONF_DIR` environment variable. For resource providers which provide 
non-session deployments, you can specify per-job configurations this way. Make 
a copy of the `conf` directory from the Flink distribution and modify the 
settings on a per-job basis. Note that this is not supported in Docker or 
standalone Kubernetes deployments. On Docker-based deployments, you can use the 
`FLINK_PROPERTIES` environment v [...]
 
 On session clusters, the provided configuration will only be used for 
configuring [execution](#execution) parameters, e.g. configuration parameters 
affecting the job, not the underlying cluster.
 
+# Flink 配置文件
+
+自 Flink-1.19 版本起,Flink 正式引入了对标准 YAML 1.2 
语法的完整支持。与之前版本中仅支持简单键值对的配置方式相比,这一更新为用户提供了更加灵活和强大的配置能力。为了利用这一新特性,用户需使用新引入的配置文件 
`config.yaml`。原有的 `flink-conf.yaml` 配置文件不再推荐使用,并在即将到来的 Flink-2.0 
版本中不再支持。为了确保平滑迁移,建议用户尽早将现有作业配置迁移到新的配置方式。
+
+本节将帮助用户理解如何通过 `config.yaml` 配置文件对 Flink 集群和作业进行配置,以及如何将老配置迁移至新的配置文件中。
+
+### 用法
+
+从 Flink-1.19 版本开始,默认的配置文件已更改为 `config.yaml`,并置于 `conf/` 目录下。用户在进行配置时应直接修改此文件。
+
+如果用户希望继续使用 Flink-1.19 之前的配置文件 `flink-conf.yaml`,只需将该文件拷贝到 `conf/` 目录下。一旦检测到 
`flink-conf.yaml` 文件,Flink 会优先使用其作为配置文件。
+
+`config.yaml` 的配置方式如下:
+
+ Config Key
+
+- 用户可以按照 Nested 的格式来组织 Config Key,如:
+
+```config.yaml
+restart-strategy:
+  type: failure-rate
+  failure-rate:
+delay: 1 s
+failure-rate-interval: 1 min
+max-failures-per-interval: 1
+```
+
+- 此外,用户也可以按 Flatten 的格式组织 Config Key,如:
+
+```flink-conf.yaml
+restart-strategy.type: failure-rate
+restart-strategy.failure-rate.delay: 1 s
+restart-strategy.failure-rate.failure-rate-interval: 1 min
+restart-strategy.failure-rate.max-failures-per-interval: 1
+```
+
+ Config Value
+
+`config.yaml` 配置文件支持用户按 [YAML 1.2 core 
schema](https://yaml.org/spec/1.2.2/#103-core-schema) 进行 value 的配置。
+
+用户可按如下格式配置 Value 对应的Config Type:
+
+{{< config_file_zh >}}
+
+此外,用户还可以按字符串格式配置所有 Config Type 的 Value,只需将原始值使用单引号或双引号括起来。
+
+### 从 flink-conf.yaml 迁移至 config.yaml
+ 行为变更
+`config.yaml` 严格遵循 YAML1.2 语法,与 `flink-conf.yaml` 在大部分情况下都完全兼容,除了以下场景发生了行为变更:
+
+- Null value:
+- `flink-conf.yaml`:仅支持留空。
+- `config.yaml`:支持留空, null, Null, NULL 和 `~` 。
+
+- 注释:
+- `flink-conf.yaml`:每一行中首次出现 `#` 以后的都认为是注释。
+- `config.yaml`:`#` 号和它前面的内容之间至少有一个空格或者 `#` 号位于一行的开头时,后续内容才认为是注释。
+
+- 需要转义的特殊字符:
+- `flink-conf.yaml`:仅需要对 List 和 Map的元素进行转义
+  - List 中的元素如果含有分号 ";" ,则需要进行转义。
+  - Map 中的元素如果含有逗号 "," 和冒号 ":" ,则需要进行转义。
+- `config.yaml`:需要对 YAML 1.2 
规范中的特殊字符进行转义,特殊字符的定义见[链接](https://yaml.org/spec/1.2.2/#53-indicator-characters)。
+
+- 重复Key:
+- `flink-conf.yaml`:允许重复Key,取文件中最末端出现的对应Key的 key-value pair。
+- `config.yaml`:不允许重复Key,加载配置时将报错。
+
+- 对非法配置的处理:
+- `flink-conf.yaml`:非法的 key-value pair 将被忽略。
+- `config.yaml`:加载配置时将报错。
+
+ Migration Tool
+为了方便用户迁移,Flink 提供了一个配置文件迁移脚本,使用这个脚本可以自动

(flink) branch master updated (6be30b16799 -> e9bea09510e)

2024-02-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 6be30b16799 [FLINK-34323][table-planner] Fix named params in session 
window tvf
 new 5b61baadd02 [FLINK-34247][doc] Add documentation of new Flink 
configuration file config.yaml.
 new 04dd91f2b6c [FLINK-34247][doc] Update the usage of "env.java.home" in 
doc.
 new e9bea09510e [FLINK-34247][doc] Update the usage of flink-conf.yaml in 
doc.

The 3 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:
 .../content.zh/docs/connectors/table/filesystem.md |   4 +-
 .../docs/deployment/advanced/historyserver.md  |   2 +-
 docs/content.zh/docs/deployment/cli.md |   6 +-
 docs/content.zh/docs/deployment/config.md  |  88 +-
 .../docs/deployment/filesystems/azure.md   |   6 +-
 docs/content.zh/docs/deployment/filesystems/gcs.md |   6 +-
 docs/content.zh/docs/deployment/filesystems/oss.md |   6 +-
 docs/content.zh/docs/deployment/filesystems/s3.md  |  12 +-
 .../content.zh/docs/deployment/ha/kubernetes_ha.md |   2 +-
 docs/content.zh/docs/deployment/ha/zookeeper_ha.md |   4 +-
 .../docs/deployment/memory/mem_migration.md|   8 +-
 .../content.zh/docs/deployment/metric_reporters.md |   2 +-
 .../resource-providers/native_kubernetes.md|   4 +-
 .../resource-providers/standalone/docker.md|  14 +--
 .../resource-providers/standalone/kubernetes.md|  32 +++---
 .../resource-providers/standalone/overview.md  |  10 +-
 .../docs/deployment/resource-providers/yarn.md |   2 +-
 .../docs/deployment/security/security-ssl.md   |   2 +-
 docs/content.zh/docs/deployment/trace_reporters.md |   2 +-
 .../docs/dev/datastream/execution/parallel.md  |   2 +-
 .../datastream/fault-tolerance/checkpointing.md|   2 +-
 .../datastream/fault-tolerance/state_backends.md   |   2 +-
 .../docs/dev/python/environment_variables.md   |   2 +-
 docs/content.zh/docs/dev/table/catalogs.md |   4 +-
 .../dev/table/hive-compatibility/hiveserver2.md|   2 +-
 .../docs/dev/table/sql-gateway/overview.md |   4 +-
 docs/content.zh/docs/dev/table/sqlClient.md|   2 +-
 docs/content.zh/docs/ops/debugging/flame_graphs.md |   2 +-
 docs/content.zh/docs/ops/metrics.md|   4 +-
 docs/content.zh/docs/ops/rest_api.md   |   2 +-
 docs/content.zh/docs/ops/state/state_backends.md   |  22 ++--
 .../docs/ops/state/task_failure_recovery.md|   8 +-
 .../docs/try-flink/flink-operations-playground.md  |   4 +-
 docs/content/docs/connectors/table/filesystem.md   |   4 +-
 .../docs/deployment/advanced/historyserver.md  |   2 +-
 docs/content/docs/deployment/cli.md|   6 +-
 docs/content/docs/deployment/config.md |  88 +-
 docs/content/docs/deployment/filesystems/azure.md  |   6 +-
 docs/content/docs/deployment/filesystems/gcs.md|   6 +-
 docs/content/docs/deployment/filesystems/oss.md|   6 +-
 docs/content/docs/deployment/filesystems/s3.md |  12 +-
 docs/content/docs/deployment/ha/kubernetes_ha.md   |   2 +-
 docs/content/docs/deployment/ha/zookeeper_ha.md|   4 +-
 .../docs/deployment/memory/mem_migration.md|  12 +-
 docs/content/docs/deployment/metric_reporters.md   |   2 +-
 .../resource-providers/native_kubernetes.md|   4 +-
 .../resource-providers/standalone/docker.md|  14 +--
 .../resource-providers/standalone/kubernetes.md|  38 +++
 .../resource-providers/standalone/overview.md  |   4 +-
 .../docs/deployment/resource-providers/yarn.md |   2 +-
 .../docs/deployment/security/security-ssl.md   |   2 +-
 docs/content/docs/deployment/trace_reporters.md|   2 +-
 .../docs/dev/datastream/execution/parallel.md  |   2 +-
 .../datastream/fault-tolerance/checkpointing.md|   2 +-
 .../datastream/fault-tolerance/state_backends.md   |   2 +-
 .../docs/dev/python/environment_variables.md   |   2 +-
 docs/content/docs/dev/table/catalogs.md|   4 +-
 docs/content/docs/dev/table/config.md  |   4 +-
 .../docs/dev/table/sql-gateway/hiveserver2.md  |   2 +-
 .../content/docs/dev/table/sql-gateway/overview.md |   4 +-
 docs/content/docs/dev/table/sqlClient.md   |   2 +-
 docs/content/docs/ops/debugging/flame_graphs.md|   2 +-
 docs/content/docs/ops/metrics.md   |   4 +-
 docs/content/docs/ops/rest_api.md  |   2 +-
 docs/content/docs/ops/state/savepoints.md  |   2 +-
 docs/content/docs/ops/state/state_backends.md  |  24 ++--
 .../docs/ops/state/task_failure_recovery.md|   8 +-
 .../docs/try-flink/flink-operations-playground.md  |

(flink) branch master updated (2be1ea801cf -> dd3e60a4b1e)

2024-02-01 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 2be1ea801cf [FLINK-34126][configuration] Correct the description of 
config option jobmanager.scheduler
 add dd3e60a4b1e [FLINK-34132][runtime] Correct the error message and doc 
that AdaptiveBatch only supports all edges being BLOCKING or 
HYBRID_FULL/HYBRID_SELECTIVE.

No new revisions were added by this update.

Summary of changes:
 docs/content.zh/docs/deployment/elastic_scaling.md| 2 +-
 docs/content/docs/deployment/elastic_scaling.md   | 2 +-
 .../scheduler/adaptivebatch/AdaptiveBatchSchedulerFactory.java| 8 ++--
 3 files changed, 8 insertions(+), 4 deletions(-)



(flink) branch master updated: [FLINK-34126][configuration] Correct the description of config option jobmanager.scheduler

2024-01-31 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 2be1ea801cf [FLINK-34126][configuration] Correct the description of 
config option jobmanager.scheduler
2be1ea801cf is described below

commit 2be1ea801cf616d0d0a82729829245c205caaad8
Author: JunRuiLee 
AuthorDate: Wed Jan 17 13:54:55 2024 +0800

[FLINK-34126][configuration] Correct the description of config option 
jobmanager.scheduler

This closes #24112.
---
 .../generated/all_jobmanager_section.html  |  2 +-
 .../generated/expert_scheduling_section.html   |  2 +-
 .../generated/job_manager_configuration.html   |  2 +-
 .../flink/configuration/JobManagerOptions.java | 51 ++
 4 files changed, 36 insertions(+), 21 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/all_jobmanager_section.html 
b/docs/layouts/shortcodes/generated/all_jobmanager_section.html
index 79b0e7f804b..626aa50f386 100644
--- a/docs/layouts/shortcodes/generated/all_jobmanager_section.html
+++ b/docs/layouts/shortcodes/generated/all_jobmanager_section.html
@@ -120,7 +120,7 @@
 jobmanager.scheduler
 Default
 Enum
-Determines which scheduler implementation is used to schedule 
tasks. Accepted values are:'Default': Default 
scheduler'Adaptive': Adaptive scheduler. More details can be found here.'AdaptiveBatch':
 Adaptive batch scheduler. More details can be found "Adaptive": Adaptive scheduler. More details can be found here
 
 jobstore.cache-size
diff --git a/docs/layouts/shortcodes/generated/expert_scheduling_section.html 
b/docs/layouts/shortcodes/generated/expert_scheduling_section.html
index 56837bda5e3..0c9c877c844 100644
--- a/docs/layouts/shortcodes/generated/expert_scheduling_section.html
+++ b/docs/layouts/shortcodes/generated/expert_scheduling_section.html
@@ -102,7 +102,7 @@
 jobmanager.scheduler
 Default
 Enum
-Determines which scheduler implementation is used to schedule 
tasks. Accepted values are:'Default': Default 
scheduler'Adaptive': Adaptive scheduler. More details can be found here.'AdaptiveBatch':
 Adaptive batch scheduler. More details can be found "Adaptive": Adaptive scheduler. More details can be found here
 
 scheduler-mode
diff --git a/docs/layouts/shortcodes/generated/job_manager_configuration.html 
b/docs/layouts/shortcodes/generated/job_manager_configuration.html
index d07705d7018..b82ae03c3d7 100644
--- a/docs/layouts/shortcodes/generated/job_manager_configuration.html
+++ b/docs/layouts/shortcodes/generated/job_manager_configuration.html
@@ -174,7 +174,7 @@
 jobmanager.scheduler
 Default
 Enum
-Determines which scheduler implementation is used to schedule 
tasks. Accepted values are:'Default': Default 
scheduler'Adaptive': Adaptive scheduler. More details can be found here.'AdaptiveBatch':
 Adaptive batch scheduler. More details can be found "Adaptive": Adaptive scheduler. More details can be found here
 
 jobstore.cache-size
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 0378e380923..1e6c68c3506 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
@@ -18,9 +18,11 @@
 
 package org.apache.flink.configuration;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.configuration.description.Description;
+import org.apache.flink.configuration.description.InlineElement;
 
 import java.time.Duration;
 
@@ -458,29 +460,42 @@ public class JobManagerOptions {
 .withDescription(
 Description.builder()
 .text(
-"Determines which scheduler 
implementation is used to schedule tasks. Accepted values are:")
-.list(
-text("'Default': Default 
scheduler"),
-text(
-"'Adaptive': Adaptive 
scheduler. More details can be found %s.",
-link(
-
"{{.Site.BaseURL}}{{.Site.LanguagePrefix}}/docs/deployment/elastic_scaling#adaptive-scheduler&quo

(flink) branch master updated: [FLINK-34206][runtime] Fix potential job failure due to concurrent global failure and source paralelism inference

2024-01-31 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new b737b718596 [FLINK-34206][runtime] Fix potential job failure due to 
concurrent global failure and source paralelism inference
b737b718596 is described below

commit b737b71859672e8020881ce2abf998735ee98abb
Author: sunxia 
AuthorDate: Tue Jan 30 14:26:26 2024 +0800

[FLINK-34206][runtime] Fix potential job failure due to concurrent global 
failure and source paralelism inference

This closes #24223.
---
 .../runtime/scheduler/ExecutionVertexVersioner.java  |  2 +-
 .../scheduler/adaptivebatch/AdaptiveBatchScheduler.java  | 16 ++--
 .../apache/flink/test/streaming/runtime/CacheITCase.java |  2 --
 3 files changed, 15 insertions(+), 5 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexVersioner.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexVersioner.java
index b0a0b17db0d..a86d22fc40c 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexVersioner.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexVersioner.java
@@ -92,7 +92,7 @@ public class ExecutionVertexVersioner {
 ExecutionVertexVersion::getExecutionVertexId, 
Function.identity()));
 }
 
-ExecutionVertexVersion getExecutionVertexVersion(ExecutionVertexID 
executionVertexId) {
+public ExecutionVertexVersion getExecutionVertexVersion(ExecutionVertexID 
executionVertexId) {
 final long currentVersion = getCurrentVersion(executionVertexId);
 return new ExecutionVertexVersion(executionVertexId, currentVersion);
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
index 97b4b24f8ac..83fb50f1514 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchScheduler.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.scheduler.adaptivebatch;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobStatus;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
 import 
org.apache.flink.configuration.JobManagerOptions.HybridPartitionDataConsumeConstraint;
@@ -62,6 +63,7 @@ import org.apache.flink.runtime.scheduler.DefaultScheduler;
 import org.apache.flink.runtime.scheduler.ExecutionGraphFactory;
 import org.apache.flink.runtime.scheduler.ExecutionOperations;
 import org.apache.flink.runtime.scheduler.ExecutionSlotAllocatorFactory;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersion;
 import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner;
 import org.apache.flink.runtime.scheduler.VertexParallelismStore;
 import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
@@ -187,8 +189,10 @@ public class AdaptiveBatchScheduler extends 
DefaultScheduler {
 protected void startSchedulingInternal() {
 tryComputeSourceParallelismThenRunAsync(
 (Void value, Throwable throwable) -> {
-initializeVerticesIfPossible();
-super.startSchedulingInternal();
+if (getExecutionGraph().getState() == JobStatus.CREATED) {
+initializeVerticesIfPossible();
+super.startSchedulingInternal();
+}
 });
 }
 
@@ -196,8 +200,16 @@ public class AdaptiveBatchScheduler extends 
DefaultScheduler {
 protected void onTaskFinished(final Execution execution, final IOMetrics 
ioMetrics) {
 checkNotNull(ioMetrics);
 updateResultPartitionBytesMetrics(ioMetrics.getResultPartitionBytes());
+ExecutionVertexVersion currentVersion =
+
executionVertexVersioner.getExecutionVertexVersion(execution.getVertex().getID());
 tryComputeSourceParallelismThenRunAsync(
 (Void value, Throwable throwable) -> {
+if (executionVertexVersioner.isModified(currentVersion)) {
+log.debug(
+"Initialization of vertices will be skipped, 
because the execution"
++ " vertex version has been 
modified.");
+return;
+}
 initializeVerticesIfPossible();
 super.onTaskFin

(flink) 01/02: [hotfix][test] Update test_configuration.py to test test_parse_jars_value.

2024-01-29 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 6da6e764a4f7a5f74c646a1b96422f585b414674
Author: JunRuiLee 
AuthorDate: Mon Jan 29 17:35:56 2024 +0800

[hotfix][test] Update test_configuration.py to test test_parse_jars_value.
---
 flink-python/pyflink/common/tests/test_configuration.py | 15 +++
 1 file changed, 15 insertions(+)

diff --git a/flink-python/pyflink/common/tests/test_configuration.py 
b/flink-python/pyflink/common/tests/test_configuration.py
index ab2fd273ee0..978bba538a7 100644
--- a/flink-python/pyflink/common/tests/test_configuration.py
+++ b/flink-python/pyflink/common/tests/test_configuration.py
@@ -18,6 +18,7 @@
 from copy import deepcopy
 
 from pyflink.common import Configuration
+from pyflink.java_gateway import get_gateway
 from pyflink.testing.test_case_utils import PyFlinkTestCase
 
 
@@ -163,3 +164,17 @@ class ConfigurationTests(PyFlinkTestCase):
 self.assertEqual(conf, conf2)
 
 self.assertEqual(str(conf), "{k1=v1, k2=1}")
+
+def test_parse_jars_value(self):
+jvm = get_gateway().jvm
+# test parse YAML list
+value = "- jar1\n- jar2\n- jar3"
+expected_result = ['jar1', 'jar2', 'jar3']
+result = Configuration.parse_jars_value(value, jvm)
+self.assertEqual(result, expected_result)
+
+# test parse legacy pattern
+value = "jar1;jar2;jar3"
+expected_result = ['jar1', 'jar2', 'jar3']
+result = Configuration.parse_jars_value(value, jvm)
+self.assertEqual(result, expected_result)



(flink) branch master updated (3b4fa6c424b -> 081051a2cac)

2024-01-29 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 3b4fa6c424b [FLINK-34103][examples] Include flink-connector-datagen 
for AsyncIO example and bundle it into the dist by default
 new 6da6e764a4f [hotfix][test] Update test_configuration.py to test 
test_parse_jars_value.
 new 081051a2cac [FLINK-34257][core] Update Flink YAML Parser to Support 
YAML 1.2 Specification.

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-core/pom.xml |   7 +
 .../flink/configuration/YamlParserUtils.java   | 167 +++--
 .../flink/configuration/YamlParserUtilsTest.java   |  63 +++-
 flink-dist/src/main/resources/META-INF/NOTICE  |   1 +
 flink-python/dev/dev-requirements.txt  |   2 +-
 flink-python/pyflink/common/configuration.py   |   5 +-
 .../pyflink/common/tests/test_configuration.py |  15 ++
 flink-python/pyflink/pyflink_gateway_server.py |   5 +-
 flink-python/setup.py  |   2 +-
 9 files changed, 174 insertions(+), 93 deletions(-)



(flink) 02/02: [FLINK-34257][core] Update Flink YAML Parser to Support YAML 1.2 Specification.

2024-01-29 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 081051a2cacaddf6dfe613da061f15f28a015a41
Author: JunRuiLee 
AuthorDate: Mon Jan 29 17:35:19 2024 +0800

[FLINK-34257][core] Update Flink YAML Parser to Support YAML 1.2 
Specification.

This closes #24213.
---
 flink-core/pom.xml |   7 +
 .../flink/configuration/YamlParserUtils.java   | 167 +++--
 .../flink/configuration/YamlParserUtilsTest.java   |  63 +++-
 flink-dist/src/main/resources/META-INF/NOTICE  |   1 +
 flink-python/dev/dev-requirements.txt  |   2 +-
 flink-python/pyflink/common/configuration.py   |   5 +-
 flink-python/pyflink/pyflink_gateway_server.py |   5 +-
 flink-python/setup.py  |   2 +-
 8 files changed, 159 insertions(+), 93 deletions(-)

diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index 917ef53cfaf..618ac8c15dc 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -96,6 +96,13 @@ under the License.


 
+   
+   
+   org.snakeyaml
+   snakeyaml-engine
+   2.6
+   
+


org.apache.commons
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java 
b/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
index ae9280b3a86..a8b845043f6 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
@@ -20,20 +20,21 @@ package org.apache.flink.configuration;
 
 import org.apache.flink.util.TimeUtils;
 
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.DumperOptions;
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.LoaderOptions;
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.Yaml;
-import 
org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.Constructor;
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.error.Mark;
-import 
org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.error.MarkedYAMLException;
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.error.YAMLException;
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Node;
-import org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.nodes.Tag;
-import 
org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Represent;
-import 
org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.snakeyaml.engine.v2.api.Dump;
+import org.snakeyaml.engine.v2.api.DumpSettings;
+import org.snakeyaml.engine.v2.api.Load;
+import org.snakeyaml.engine.v2.api.LoadSettings;
+import org.snakeyaml.engine.v2.common.FlowStyle;
+import org.snakeyaml.engine.v2.exceptions.Mark;
+import org.snakeyaml.engine.v2.exceptions.MarkedYamlEngineException;
+import org.snakeyaml.engine.v2.exceptions.YamlEngineException;
+import org.snakeyaml.engine.v2.nodes.Node;
+import org.snakeyaml.engine.v2.nodes.ScalarNode;
+import org.snakeyaml.engine.v2.nodes.Tag;
+import org.snakeyaml.engine.v2.representer.StandardRepresenter;
+import org.snakeyaml.engine.v2.schema.CoreSchema;
 
 import javax.annotation.Nonnull;
 
@@ -47,6 +48,7 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
 /**
  * This class contains utility methods to load standard yaml file and convert 
object to standard
@@ -56,26 +58,30 @@ public class YamlParserUtils {
 
 private static final Logger LOG = 
LoggerFactory.getLogger(YamlParserUtils.class);
 
-private static final Yaml yaml;
+private static final DumpSettings blockerDumperSettings =
+DumpSettings.builder()
+.setDefaultFlowStyle(FlowStyle.BLOCK)
+// Disable split long lines to avoid add unexpected line 
breaks
+.setSplitLines(false)
+.setSchema(new CoreSchema())
+.build();
 
-private static final DumperOptions dumperOptions = new DumperOptions();
+private static final DumpSettings flowDumperSettings =
+DumpSettings.builder()
+.setDefaultFlowStyle(FlowStyle.FLOW)
+// Disable split long lines to avoid add unexpected line 
breaks
+.setSplitLines(false)
+.setSchema(new CoreSchema())
+.build();
 
-private static final LoaderOptions loaderOptions = new LoaderOptions();
+private static final Dump blockerDumper =
+new Dump(blockerDumperSettings, new 
FlinkConfigRepresenter

(flink) branch master updated (2e56caf11c0 -> ddbf87f2a7a)

2024-01-28 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 2e56caf11c0 [FLINK-34122][core] Deprecate old serialization config 
methods and options
 add ddbf87f2a7a [FLINK-34245][python] Fix config retrieval logic from 
nested YAML in pyflink_gateway_server with flattened keys.

No new revisions were added by this update.

Summary of changes:
 flink-python/pyflink/pyflink_gateway_server.py | 15 ++-
 1 file changed, 14 insertions(+), 1 deletion(-)



(flink) branch master updated: [FLINK-34145][connector/filesystem] support dynamic source parallelism inference in batch jobs

2024-01-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 11631cb5956 [FLINK-34145][connector/filesystem] support dynamic source 
parallelism inference in batch jobs
11631cb5956 is described below

commit 11631cb59568df60d40933fb13c8433062ed9290
Author: sunxia 
AuthorDate: Wed Jan 24 14:26:03 2024 +0800

[FLINK-34145][connector/filesystem] support dynamic source parallelism 
inference in batch jobs

This closes #24186.
---
 .../connector/file/src/AbstractFileSource.java |  6 ++-
 .../flink/connector/file/src/FileSource.java   | 25 +++-
 .../file/src/FileSourceTextLinesITCase.java| 46 +-
 3 files changed, 73 insertions(+), 4 deletions(-)

diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
index 6dbed75747b..f4fb463e10e 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/AbstractFileSource.java
@@ -69,7 +69,7 @@ public abstract class AbstractFileSource
 
 private static final long serialVersionUID = 1L;
 
-private final Path[] inputPaths;
+final Path[] inputPaths;
 
 private final FileEnumerator.Provider enumeratorFactory;
 
@@ -100,6 +100,10 @@ public abstract class AbstractFileSource
 //  Getters
 // 
 
+FileEnumerator.Provider getEnumeratorFactory() {
+return enumeratorFactory;
+}
+
 public FileSplitAssigner.Provider getAssignerFactory() {
 return assignerFactory;
 }
diff --git 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSource.java
 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSource.java
index da76f790627..7d3f545fc02 100644
--- 
a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSource.java
+++ 
b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/src/FileSource.java
@@ -19,6 +19,7 @@
 package org.apache.flink.connector.file.src;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.source.DynamicParallelismInference;
 import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
 import 
org.apache.flink.connector.file.src.assigners.LocalityAwareSplitAssigner;
 import 
org.apache.flink.connector.file.src.enumerate.BlockSplittingRecursiveEnumerator;
@@ -32,10 +33,13 @@ import 
org.apache.flink.connector.file.src.reader.StreamFormat;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import javax.annotation.Nullable;
 
+import java.io.IOException;
 import java.time.Duration;
+import java.util.Collection;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -93,7 +97,8 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
  * @param  The type of the events/records produced by this source.
  */
 @PublicEvolving
-public final class FileSource extends AbstractFileSource {
+public final class FileSource extends AbstractFileSource
+implements DynamicParallelismInference {
 
 private static final long serialVersionUID = 1L;
 
@@ -141,6 +146,24 @@ public final class FileSource extends 
AbstractFileSource
 return FileSourceSplitSerializer.INSTANCE;
 }
 
+@Override
+public int inferParallelism(Context dynamicParallelismContext) {
+FileEnumerator fileEnumerator = getEnumeratorFactory().create();
+
+Collection splits;
+try {
+splits =
+fileEnumerator.enumerateSplits(
+inputPaths,
+
dynamicParallelismContext.getParallelismInferenceUpperBound());
+} catch (IOException e) {
+throw new FlinkRuntimeException("Could not enumerate file splits", 
e);
+}
+
+return Math.min(
+splits.size(), 
dynamicParallelismContext.getParallelismInferenceUpperBound());
+}
+
 // 
 //  Entry-point Factory Methods
 // 
diff --git 
a/flink-connectors/flink-connector-files/src/test/java/org/ap

(flink) branch master updated: [FLINK-34206][test] Temporarily disable CacheITCase#testRetryOnCorruptedClusterDataset

2024-01-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 05ee359ebd5 [FLINK-34206][test] Temporarily disable 
CacheITCase#testRetryOnCorruptedClusterDataset
05ee359ebd5 is described below

commit 05ee359ebd564af3dd8ab31975cd479e92ba1785
Author: sunxia 
AuthorDate: Fri Jan 26 19:22:27 2024 +0800

[FLINK-34206][test] Temporarily disable 
CacheITCase#testRetryOnCorruptedClusterDataset

This closes #24201.
---
 .../test/java/org/apache/flink/test/streaming/runtime/CacheITCase.java  | 2 ++
 1 file changed, 2 insertions(+)

diff --git 
a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CacheITCase.java
 
b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CacheITCase.java
index c60595de6b3..8866b205ce1 100644
--- 
a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CacheITCase.java
+++ 
b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CacheITCase.java
@@ -48,6 +48,7 @@ import org.apache.flink.util.OutputTag;
 import org.apache.commons.io.FileUtils;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
 
@@ -217,6 +218,7 @@ public class CacheITCase extends AbstractTestBase {
 }
 
 @Test
+@Disabled
 void testRetryOnCorruptedClusterDataset(@TempDir java.nio.file.Path 
tmpDir) throws Exception {
 File file = prepareTestData(tmpDir);
 



(flink) branch master updated (d55f5f16c23 -> 8fceb101e7e)

2024-01-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from d55f5f16c23 [FLINK-33263][bugfix][table-planner] Update the flink 
shaded guava version used to 32
 add e623c07f4e5 [FLINK-34232][dist] Fix unsupported env.java.home in 
configuration file.
 add 8fceb101e7e [FLINK-34223][dist] Introduce a migration tool to transfer 
legacy config file to new config file.

No new revisions were added by this update.

Summary of changes:
 .../apache/flink/configuration/Configuration.java  |  3 +-
 .../src/main/flink-bin/bin/bash-java-utils.sh  | 67 +++---
 .../src/main/flink-bin/bin/config-parser-utils.sh  |  1 +
 flink-dist/src/main/flink-bin/bin/config.sh|  1 +
 ...nfig-parser-utils.sh => migrate-config-file.sh} | 39 +++--
 flink-dist/src/test/bin/runBashJavaUtilsCmd.sh |  9 ++-
 .../org/apache/flink/dist/BashJavaUtilsITCase.java | 63 
 flink-dist/src/test/resources/flink-conf.yaml  | 36 
 flink-end-to-end-tests/test-scripts/common.sh  |  1 +
 .../runtime/util/ConfigurationParserUtils.java | 31 ++
 .../flink/runtime/util/bash/BashJavaUtils.java |  7 ++-
 .../flink/runtime/util/bash/FlinkConfigLoader.java |  7 +++
 .../runtime/util/bash/FlinkConfigLoaderTest.java   | 61 
 13 files changed, 298 insertions(+), 28 deletions(-)
 copy flink-dist/src/main/flink-bin/bin/{config-parser-utils.sh => 
migrate-config-file.sh} (57%)
 create mode 100644 flink-dist/src/test/resources/flink-conf.yaml



(flink) branch master updated: [FLINK-33577][dist] Change the default config file to config.yaml in flink-dist.

2024-01-25 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 9721ce835f5 [FLINK-33577][dist] Change the default config file to 
config.yaml in flink-dist.
9721ce835f5 is described below

commit 9721ce835f5a7f28f2ad187346e009633307097b
Author: JunRuiLee 
AuthorDate: Wed Aug 23 22:48:30 2023 +0800

[FLINK-33577][dist] Change the default config file to config.yaml in 
flink-dist.

This closes #24177.
---
 .../generated/kubernetes_config_configuration.html |   2 +-
 .../shortcodes/generated/python_configuration.html |   2 +-
 .../generated/rocksdb_configuration.html   |   2 +-
 .../generated/state_backend_rocksdb_section.html   |   2 +-
 .../api/java/hadoop/mapred/utils/HadoopUtils.java  |   2 +-
 .../common/restartstrategy/RestartStrategies.java  |   2 +-
 .../flink/configuration/ConfigConstants.java   |   4 +-
 .../configuration/ResourceManagerOptions.java  |   2 +-
 flink-dist/src/main/assemblies/bin.xml |   2 +-
 flink-dist/src/main/flink-bin/bin/config.sh|   8 +-
 flink-dist/src/main/resources/config.yaml  | 298 
 flink-dist/src/main/resources/flink-conf.yaml  | 311 -
 .../org/apache/flink/dist/BashJavaUtilsITCase.java |  51 +++-
 .../flink/tests/util/flink/FlinkDistribution.java  |   6 +-
 flink-end-to-end-tests/test-scripts/common.sh  |  18 +-
 .../test-scripts/common_yarn_docker.sh |   4 +-
 .../test-scripts/test_pyflink.sh   |  10 +-
 .../test_yarn_application_kerberos_docker.sh   |   2 +-
 .../test-scripts/test_yarn_job_kerberos_docker.sh  |   2 +-
 .../parquet/ParquetVectorizedInputFormat.java  |   2 +-
 .../java/org/apache/flink/api/java/DataSet.java|   2 +-
 .../configuration/KubernetesConfigOptions.java |   2 +-
 .../decorators/FlinkConfMountDecorator.java|   2 +-
 .../parameters/KubernetesParameters.java   |   2 +-
 flink-python/dev/dev-requirements.txt  |   1 +
 flink-python/pyflink/common/configuration.py   |  13 +-
 flink-python/pyflink/common/restart_strategy.py|   2 +-
 flink-python/pyflink/datastream/state_backend.py   |  16 +-
 .../datastream/stream_execution_environment.py |   2 +-
 flink-python/pyflink/pyflink_gateway_server.py |  68 +++--
 flink-python/pyflink/table/table_config.py |   6 +-
 flink-python/pyflink/table/table_environment.py|  34 ++-
 flink-python/setup.py  |   4 +-
 .../java/org/apache/flink/python/PythonConfig.java |   2 +-
 .../org/apache/flink/python/PythonOptions.java |   2 +-
 .../RestartBackoffTimeStrategyFactoryLoader.java   |   4 +-
 .../runtime/state/CheckpointStorageLoader.java |   4 +-
 .../runtime/state/filesystem/FsStateBackend.java   |   2 +-
 .../runtime/state/memory/MemoryStateBackend.java   |   2 +-
 .../state/EmbeddedRocksDBStateBackend.java |   4 +-
 .../contrib/streaming/state/RocksDBOptions.java|   2 +-
 .../streaming/state/RocksDBStateBackend.java   |   7 +-
 .../state/RocksDBStateBackendConfigTest.java   |   6 +-
 .../environment/StreamExecutionEnvironment.java|   2 +-
 .../gateway/service/context/DefaultContext.java|   4 +-
 .../rest/util/SqlGatewayRestEndpointTestUtils.java |   2 +-
 .../service/context/SessionContextTest.java|   2 +-
 .../org/apache/flink/table/api/TableConfig.java|   4 +-
 .../container/FlinkContainersSettings.java |   4 +-
 .../testframe/container/FlinkImageBuilder.java |  15 +-
 .../java/org/apache/flink/yarn/YarnTestBase.java   |  10 +-
 .../src/main/java/org/apache/flink/yarn/Utils.java |   2 +-
 52 files changed, 510 insertions(+), 456 deletions(-)

diff --git 
a/docs/layouts/shortcodes/generated/kubernetes_config_configuration.html 
b/docs/layouts/shortcodes/generated/kubernetes_config_configuration.html
index 7bbdfd5e404..86d147f6c00 100644
--- a/docs/layouts/shortcodes/generated/kubernetes_config_configuration.html
+++ b/docs/layouts/shortcodes/generated/kubernetes_config_configuration.html
@@ -90,7 +90,7 @@
 kubernetes.flink.conf.dir
 "/opt/flink/conf"
 String
-The flink conf directory that will be mounted in pod. The 
flink-conf.yaml, log4j.properties, logback.xml in this path will be overwritten 
from config map.
+The flink conf directory that will be mounted in pod. The 
config.yaml, log4j.properties, logback.xml in this path will be overwritten 
from config map.
 
 
 kubernetes.flink.log.dir
diff --git a/docs/layouts/shortcodes/generated/python_configuration.html 
b/docs/layouts/shortcodes/generated/python_configuration.html
index 60ef6a9676e..d99c1f2a3b9 100644
--- a/docs/layouts/shortcodes/generated/python_configuration.html
+++ b/docs/layouts/

(flink) branch master updated: [FLINK-34144][docs] Update the documentation about dynamic source parallelism inference

2024-01-25 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 78e31f0dcc4 [FLINK-34144][docs] Update the documentation about dynamic 
source parallelism inference
78e31f0dcc4 is described below

commit 78e31f0dcc4da65d88e18560f3374a47cc0a7c9b
Author: sunxia 
AuthorDate: Wed Jan 24 11:34:19 2024 +0800

[FLINK-34144][docs] Update the documentation about dynamic source 
parallelism inference

This closes #24184.
---
 docs/content.zh/docs/deployment/elastic_scaling.md | 17 -
 docs/content/docs/deployment/elastic_scaling.md| 18 +-
 2 files changed, 33 insertions(+), 2 deletions(-)

diff --git a/docs/content.zh/docs/deployment/elastic_scaling.md 
b/docs/content.zh/docs/deployment/elastic_scaling.md
index d9b09ba7e8c..119be1c419c 100644
--- a/docs/content.zh/docs/deployment/elastic_scaling.md
+++ b/docs/content.zh/docs/deployment/elastic_scaling.md
@@ -211,7 +211,7 @@ Adaptive Batch Scheduler 是一种可以自动调整执行计划的批作业调
   - [`execution.batch.adaptive.auto-parallelism.min-parallelism`]({{< ref 
"docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-min-parallelism): 允许自动设置的并行度最小值。
   - [`execution.batch.adaptive.auto-parallelism.max-parallelism`]({{< ref 
"docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-max-parallelism): 
允许自动设置的并行度最大值,如果该配置项没有配置将使用通过 [`parallelism.default`]({{< ref 
"docs/deployment/config" >}}) 或者 `StreamExecutionEnvironment#setParallelism()` 
设置的默认并行度作为允许自动设置的并行度最大值。
   - [`execution.batch.adaptive.auto-parallelism.avg-data-volume-per-task`]({{< 
ref "docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-avg-data-volume-per-ta): 
期望每个任务平均处理的数据量大小。请注意,当出现数据倾斜,或者确定的并行度达到最大并行度(由于数据过多)时,一些任务实际处理的数据可能会远远超过这个值。
-  - 
[`execution.batch.adaptive.auto-parallelism.default-source-parallelism`]({{< 
ref "docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-default-source-paralle): source 
算子的默认并行度。
+  - 
[`execution.batch.adaptive.auto-parallelism.default-source-parallelism`]({{< 
ref "docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-default-source-paralle): source 
算子可动态推导的最大并行度,若该配置项没有配置将优先使用 
[`execution-batch-adaptive-auto-parallelism-max-parallelism`]({{< ref 
"docs/deployment/config" >}})作为允许动态推导的并行度最大值,若该配置项也没有配置,将使用 
[`parallelism.default`]({{< ref "docs/deployment/config" >}}) 或者 
`StreamExecutionEnvironment#setParallelism()` 设置的默认并行度。
 - 不要指定算子的并行度:
 
 Adaptive Batch Scheduler 只会为用户未指定并行度的算子推导并行度。 所以如果你想算子的并行度被自动推导,需要避免通过算子的 
`setParallelism()` 方法来为其指定并行度。
@@ -220,6 +220,21 @@ Adaptive Batch Scheduler 是一种可以自动调整执行计划的批作业调
   - 配置 `parallelism.default: -1`
   - 不要通过 `ExecutionEnvironment` 的 `setParallelism()` 方法来指定并行度
 
+ 让 Source 支持动态并行度推导
+如果你的作业有用到自定义 {{< gh_link 
file="/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java"
 name="Source" >}},
+你需要让 Source 实现接口 {{< gh_link 
file="/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicParallelismInference.java"
 name="DynamicParallelismInference" >}}。
+```java
+public interface DynamicParallelismInference {
+int inferParallelism(Context context);
+}
+```
+其中 Context 会提供可推导并行度上界、期望每个任务平均处理的数据量大小、动态过滤信息来协助并行度推导。
+Adaptive Batch Scheduler 将会在调度 Source 节点之前调用上述接口,需注意实现中应尽量避免高耗时的操作。
+
+若 Source 
未实现上述接口,[`execution.batch.adaptive.auto-parallelism.default-source-parallelism`]({{<
 ref "docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-default-source-paralle) 将会作为 
Source 节点的并行度。
+
+需注意,Source 动态并行度推导也只会为用户未指定并行度的 Source 算子推导并行度。
+
  性能调优
 
 1. 建议使用 [Sort 
Shuffle](https://flink.apache.org/2021/10/26/sort-shuffle-part1.html) 并且设置 
[`taskmanager.network.memory.buffers-per-channel`]({{< ref 
"docs/deployment/config" >}}#taskmanager-network-memory-buffers-per-channel) 为 
`0`。 这会解耦并行度与需要的网络内存,对于大规模作业,这样可以降低遇到 "Insufficient number of network buffers" 
错误的可能性。
diff --git a/docs/content/docs/deployment/elastic_scaling.md 
b/docs/content/docs/deployment/elastic_scaling.md
index f342f05ffb9..21beb652bed 100644
--- a/docs/content/docs/deployment/elastic_scaling.md
+++ b/docs/content/docs/deployment/elastic_scaling.md
@@ -220,7 +220,7 @@ In addition, there are several related configuration 
options that may need adjus
   - [`execution.batch.adaptive.auto-parallelism.min-parallelism`]({{< ref 
"docs/deployment/config" 
>}}#execution-batch-adaptive-auto-parallelism-min-parallelism): The lower bound 
of allowed parallelism to set adaptively.
   - [`execution.batch.adaptive.auto-parallelism.max-parallelism`]({{< r

(flink-docker) branch dev-master updated: [FLINK-34205] Use BashJavaUtils for Flink configuration management

2024-01-23 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch dev-master
in repository https://gitbox.apache.org/repos/asf/flink-docker.git


The following commit(s) were added to refs/heads/dev-master by this push:
 new 44f0582  [FLINK-34205] Use BashJavaUtils for Flink configuration 
management
44f0582 is described below

commit 44f058287cc956a620b12b6f8ed214e44dc3db77
Author: JunRuiLee 
AuthorDate: Tue Jan 23 15:19:45 2024 +0800

[FLINK-34205] Use BashJavaUtils for Flink configuration management
---
 Dockerfile-ubuntu.template | 21 +
 docker-entrypoint.sh   | 74 +++---
 2 files changed, 72 insertions(+), 23 deletions(-)

diff --git a/Dockerfile-ubuntu.template b/Dockerfile-ubuntu.template
index 7073eec..8d364e0 100644
--- a/Dockerfile-ubuntu.template
+++ b/Dockerfile-ubuntu.template
@@ -81,11 +81,22 @@ RUN set -ex; \
   chown -R flink:flink .; \
   \
   # Replace default REST/RPC endpoint bind address to use the container's 
network interface \
-  sed -i 's/rest.address: localhost/rest.address: 0.0.0.0/g' 
$FLINK_HOME/conf/flink-conf.yaml; \
-  sed -i 's/rest.bind-address: localhost/rest.bind-address: 0.0.0.0/g' 
$FLINK_HOME/conf/flink-conf.yaml; \
-  sed -i 's/jobmanager.bind-host: localhost/jobmanager.bind-host: 0.0.0.0/g' 
$FLINK_HOME/conf/flink-conf.yaml; \
-  sed -i 's/taskmanager.bind-host: localhost/taskmanager.bind-host: 0.0.0.0/g' 
$FLINK_HOME/conf/flink-conf.yaml; \
-  sed -i '/taskmanager.host: localhost/d' $FLINK_HOME/conf/flink-conf.yaml;
+  CONF_FILE="$FLINK_HOME/conf/flink-conf.yaml"; \
+  if [ ! -e "$FLINK_HOME/conf/flink-conf.yaml" ]; then \
+CONF_FILE="${FLINK_HOME}/conf/config.yaml"; \
+/bin/bash "$FLINK_HOME/bin/config-parser-utils.sh" "${FLINK_HOME}/conf" 
"${FLINK_HOME}/bin" "${FLINK_HOME}/lib" \
+"-repKV" "rest.address,localhost,0.0.0.0" \
+"-repKV" "rest.bind-address,localhost,0.0.0.0" \
+"-repKV" "jobmanager.bind-host,localhost,0.0.0.0" \
+"-repKV" "taskmanager.bind-host,localhost,0.0.0.0" \
+"-rmKV" "taskmanager.host=localhost"; \
+  else \
+sed -i 's/rest.address: localhost/rest.address: 0.0.0.0/g' "$CONF_FILE"; \
+sed -i 's/rest.bind-address: localhost/rest.bind-address: 0.0.0.0/g' 
"$CONF_FILE"; \
+sed -i 's/jobmanager.bind-host: localhost/jobmanager.bind-host: 0.0.0.0/g' 
"$CONF_FILE"; \
+sed -i 's/taskmanager.bind-host: localhost/taskmanager.bind-host: 
0.0.0.0/g' "$CONF_FILE"; \
+sed -i '/taskmanager.host: localhost/d' "$CONF_FILE"; \
+  fi;
 
 # Configure container
 COPY docker-entrypoint.sh /
diff --git a/docker-entrypoint.sh b/docker-entrypoint.sh
index 8b0350e..e884375 100755
--- a/docker-entrypoint.sh
+++ b/docker-entrypoint.sh
@@ -23,7 +23,7 @@ COMMAND_HISTORY_SERVER="history-server"
 
 # If unspecified, the hostname of the container is taken as the JobManager 
address
 JOB_MANAGER_RPC_ADDRESS=${JOB_MANAGER_RPC_ADDRESS:-$(hostname -f)}
-CONF_FILE="${FLINK_HOME}/conf/flink-conf.yaml"
+CONF_FILE_DIR="${FLINK_HOME}/conf"
 
 drop_privs_cmd() {
 if [ $(id -u) != 0 ]; then
@@ -59,34 +59,72 @@ copy_plugins_if_required() {
   done
 }
 
-set_config_option() {
-  local option=$1
-  local value=$2
+set_config_options() {
+local config_parser_script="$FLINK_HOME/bin/config-parser-utils.sh"
+local config_dir="$FLINK_HOME/conf"
+local bin_dir="$FLINK_HOME/bin"
+local lib_dir="$FLINK_HOME/lib"
 
-  # escape periods for usage in regular expressions
-  local escaped_option=$(echo ${option} | sed -e "s/\./\\\./g")
+local config_params=""
 
-  # either override an existing entry, or append a new one
-  if grep -E "^${escaped_option}:.*" "${CONF_FILE}" > /dev/null; then
-sed -i -e "s/${escaped_option}:.*/$option: $value/g" "${CONF_FILE}"
-  else
-echo "${option}: ${value}" >> "${CONF_FILE}"
-  fi
+while [ $# -gt 0 ]; do
+local key="$1"
+local value="$2"
+
+config_params+=" -D${key}=${value}"
+
+shift 2
+done
+
+if [ ! -z "${config_params}" ]; then
+eval "${config_parser_script} ${config_dir} ${bin_dir} ${lib_dir} 
${config_params}"
+fi
 }
 
 prepare_configuration() {
-set_config_option jobmanager.rpc.address ${JOB_MANAGER_RPC_ADDRESS}
-set_config_option blob.server.port 6124
-set_config_option query.server.port 6125
+local config_options=()
+
+config_options+=("jobmanager.rpc.address" "${JOB_MANAGER_RPC_ADDRESS}")
+config_options+=("blob.se

(flink) branch master updated: [FLINK-34143] Modify the effective strategy of `execution.batch.adaptive.auto-parallelism.default-source-parallelism`

2024-01-23 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 97caa3c251e [FLINK-34143] Modify the effective strategy of 
`execution.batch.adaptive.auto-parallelism.default-source-parallelism`
97caa3c251e is described below

commit 97caa3c251e416640a6f54ea103912839c346f70
Author: sunxia 
AuthorDate: Tue Jan 23 10:29:17 2024 +0800

[FLINK-34143] Modify the effective strategy of 
`execution.batch.adaptive.auto-parallelism.default-source-parallelism`

This closes #24170.
---
 .../generated/batch_execution_configuration.html   |  4 +--
 .../generated/expert_scheduling_section.html   |  4 +--
 .../flink/configuration/BatchExecutionOptions.java | 12 +--
 ...faultVertexParallelismAndInputInfosDecider.java |  4 +--
 ...tVertexParallelismAndInputInfosDeciderTest.java | 40 ++
 5 files changed, 55 insertions(+), 9 deletions(-)

diff --git 
a/docs/layouts/shortcodes/generated/batch_execution_configuration.html 
b/docs/layouts/shortcodes/generated/batch_execution_configuration.html
index 79af189215e..32c4b3d6e15 100644
--- a/docs/layouts/shortcodes/generated/batch_execution_configuration.html
+++ b/docs/layouts/shortcodes/generated/batch_execution_configuration.html
@@ -16,9 +16,9 @@
 
 
 
execution.batch.adaptive.auto-parallelism.default-source-parallelism
-1
+(none)
 Integer
-The default parallelism of source vertices if jobmanager.scheduler has been set to AdaptiveBatch
+The default parallelism of source vertices or the upper bound 
of source parallelism to set adaptively if jobmanager.scheduler has been set to AdaptiveBatch. Note that execution.batch.adaptive.auto-parallelism.max-parallelism
 will be used if this configuration is not configured. If execution.batch.adaptive.auto-parallelism.max [...]
 
 
 execution.batch.adaptive.auto-parallelism.enabled
diff --git a/docs/layouts/shortcodes/generated/expert_scheduling_section.html 
b/docs/layouts/shortcodes/generated/expert_scheduling_section.html
index 96fc3056099..56837bda5e3 100644
--- a/docs/layouts/shortcodes/generated/expert_scheduling_section.html
+++ b/docs/layouts/shortcodes/generated/expert_scheduling_section.html
@@ -16,9 +16,9 @@
 
 
 
execution.batch.adaptive.auto-parallelism.default-source-parallelism
-1
+(none)
 Integer
-The default parallelism of source vertices if jobmanager.scheduler has been set to AdaptiveBatch
+The default parallelism of source vertices or the upper bound 
of source parallelism to set adaptively if jobmanager.scheduler has been set to AdaptiveBatch. Note that execution.batch.adaptive.auto-parallelism.max-parallelism
 will be used if this configuration is not configured. If execution.batch.adaptive.auto-parallelism.max [...]
 
 
 execution.batch.adaptive.auto-parallelism.enabled
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/BatchExecutionOptions.java
 
b/flink-core/src/main/java/org/apache/flink/configuration/BatchExecutionOptions.java
index a43a6f5c23a..d1d38e1ac1d 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/BatchExecutionOptions.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/BatchExecutionOptions.java
@@ -25,6 +25,7 @@ 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.CoreOptions.DEFAULT_PARALLELISM;
 import static org.apache.flink.configuration.JobManagerOptions.SCHEDULER;
 import static org.apache.flink.configuration.description.TextElement.code;
 
@@ -99,17 +100,22 @@ public class BatchExecutionOptions {
 public static final ConfigOption 
ADAPTIVE_AUTO_PARALLELISM_DEFAULT_SOURCE_PARALLELISM =
 
key("execution.batch.adaptive.auto-parallelism.default-source-parallelism")
 .intType()
-.defaultValue(1)
+.noDefaultValue()
 .withDeprecatedKeys(
 
"jobmanager.adaptive-batch-scheduler.default-source-parallelism")
 .withDescription(
 Description.builder()
 .text(
-"The default parallelism of source 
vertices if %s has been set to %s",
+"The default parallelism of source 
vertices or the upper bound of source parallelism "
++ "to set adaptively if %s 
has been set to %s. No

(flink) 03/03: [FLINK-33721][core] Extend BashJavaUtils to support reading and writing standard yaml file.

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c148b62166d8fec9e7e525a836de890c2f12973b
Author: JunRuiLee 
AuthorDate: Fri Jan 12 21:56:15 2024 +0800

[FLINK-33721][core] Extend BashJavaUtils to support reading and writing 
standard yaml file.

This closes #24091.
---
 .../src/main/flink-bin/bin/bash-java-utils.sh  | 170 +
 .../flink-bin/bin/config-parser-utils.sh}  |  32 ++--
 flink-dist/src/main/flink-bin/bin/config.sh| 143 +--
 flink-dist/src/test/bin/runBashJavaUtilsCmd.sh |   2 +-
 .../src/test/bin/runExtractLoggingOutputs.sh   |   2 +-
 .../org/apache/flink/dist/BashJavaUtilsITCase.java |  94 ++
 .../entrypoint/ModifiableClusterConfiguration.java |  82 +
 ...odifiableClusterConfigurationParserFactory.java |  98 ++
 .../parser/ConfigurationCommandLineOptions.java|  64 +++
 .../runtime/util/ConfigurationParserUtils.java |  68 +++
 .../flink/runtime/util/bash/BashJavaUtils.java |  12 +-
 .../flink/runtime/util/bash/FlinkConfigLoader.java |  18 +-
 ...iableClusterConfigurationParserFactoryTest.java |  96 ++
 .../runtime/util/bash/FlinkConfigLoaderTest.java   | 202 +
 14 files changed, 920 insertions(+), 163 deletions(-)

diff --git a/flink-dist/src/main/flink-bin/bin/bash-java-utils.sh 
b/flink-dist/src/main/flink-bin/bin/bash-java-utils.sh
new file mode 100755
index 000..f4d02327197
--- /dev/null
+++ b/flink-dist/src/main/flink-bin/bin/bash-java-utils.sh
@@ -0,0 +1,170 @@
+#!/usr/bin/env bash
+
+#  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.
+
+
+UNAME=$(uname -s)
+if [ "${UNAME:0:6}" == "CYGWIN" ]; then
+JAVA_RUN=java
+else
+if [[ -d "$JAVA_HOME" ]]; then
+JAVA_RUN="$JAVA_HOME"/bin/java
+else
+JAVA_RUN=java
+fi
+fi
+
+manglePathList() {
+UNAME=$(uname -s)
+# a path list, for example a java classpath
+if [ "${UNAME:0:6}" == "CYGWIN" ]; then
+echo `cygpath -wp "$1"`
+else
+echo $1
+fi
+}
+
+findFlinkDistJar() {
+local FLINK_DIST
+local LIB_DIR
+if [[ -n "$1" ]]; then
+   LIB_DIR="$1"
+else
+   LIB_DIR="$FLINK_LIB_DIR"
+fi
+FLINK_DIST="$(find "$LIB_DIR" -name 'flink-dist*.jar')"
+local FLINK_DIST_COUNT
+FLINK_DIST_COUNT="$(echo "$FLINK_DIST" | wc -l)"
+
+# If flink-dist*.jar cannot be resolved write error messages to stderr 
since stdout is stored
+# as the classpath and exit function with empty classpath to force process 
failure
+if [[ "$FLINK_DIST" == "" ]]; then
+(>&2 echo "[ERROR] Flink distribution jar not found in 
$FLINK_LIB_DIR.")
+exit 1
+elif [[ "$FLINK_DIST_COUNT" -gt 1 ]]; then
+(>&2 echo "[ERROR] Multiple flink-dist*.jar found in $FLINK_LIB_DIR. 
Please resolve.")
+exit 1
+fi
+
+echo "$FLINK_DIST"
+}
+
+runBashJavaUtilsCmd() {
+local cmd=$1
+local conf_dir=$2
+local class_path=$3
+local dynamic_args=${@:4}
+class_path=`manglePathList "${class_path}"`
+
+local output=`"${JAVA_RUN}" -classpath "${class_path}" 
org.apache.flink.runtime.util.bash.BashJavaUtils ${cmd} --configDir 
"${conf_dir}" $dynamic_args 2>&1 | tail -n 1000`
+if [[ $? -ne 0 ]]; then
+echo "[ERROR] Cannot run BashJavaUtils to execute command ${cmd}." 1>&2
+# Print the output in case the user redirect the log to console.
+echo "$output" 1>&2
+exit 1
+fi
+
+echo "$output"
+}
+
+updateAndGetFlinkConfiguration() {
+local FLINK_CONF_DIR="$1"
+local FLINK_BIN_DIR="$2"
+l

(flink) 02/03: [FLINK-33721][core] Support dump configuration as nested yaml data.

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 1f7622d4d23bfcb76f466469ec36585054864f04
Author: JunRuiLee 
AuthorDate: Fri Jan 12 19:41:27 2024 +0800

[FLINK-33721][core] Support dump configuration as nested yaml data.
---
 .../flink/configuration/ConfigurationUtils.java| 57 ++
 .../flink/configuration/YamlParserUtils.java   | 36 +
 .../configuration/ConfigurationUtilsTest.java  | 87 ++
 .../flink/configuration/YamlParserUtilsTest.java   | 58 +++
 .../flink/tests/util/flink/FlinkDistribution.java  |  9 +--
 .../decorators/FlinkConfMountDecorator.java| 20 ++---
 .../flink/kubernetes/KubernetesTestUtils.java  | 33 +---
 .../factory/KubernetesJobManagerFactoryTest.java   | 11 +--
 .../runtime/clusterframework/BootstrapTools.java   |  7 +-
 .../testframe/container/FlinkImageBuilder.java |  6 +-
 10 files changed, 282 insertions(+), 42 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java
 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java
index 58cbd620ede..66f92fe44c4 100755
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java
@@ -28,6 +28,7 @@ import javax.annotation.Nonnull;
 import java.io.File;
 import java.time.Duration;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
@@ -200,6 +201,62 @@ public class ConfigurationUtils {
 : EMPTY;
 }
 
+/**
+ * Converts the provided configuration data into a format suitable for 
writing to a file, based
+ * on the {@code flattenYaml} flag and the {@code standardYaml} attribute 
of the configuration
+ * object.
+ *
+ * Only when {@code flattenYaml} is set to {@code false} and the 
configuration object is
+ * standard yaml, a nested YAML format is used. Otherwise, a flat 
key-value pair format is
+ * output.
+ *
+ * Each entry in the returned list represents a single line that can be 
written directly to a
+ * file.
+ *
+ * Example input (flat map configuration data):
+ *
+ * {@code
+ * {
+ *  "parent.child": "value1",
+ *  "parent.child2": "value2"
+ * }
+ * }
+ *
+ * Example output when {@code flattenYaml} is {@code false} and the 
configuration object is
+ * standard yaml:
+ *
+ * {@code
+ * parent:
+ *   child: value1
+ *   child2: value2
+ * }
+ *
+ * Otherwise, the Example output is:
+ *
+ * {@code
+ * parent.child: value1
+ * parent.child2: value2
+ * }
+ *
+ * @param configuration The configuration to be converted.
+ * @param flattenYaml A boolean flag indicating if the configuration data 
should be output in a
+ * flattened format.
+ * @return A list of strings, where each string represents a line of the 
file-writable data in
+ * the chosen format.
+ */
+public static List convertConfigToWritableLines(
+Configuration configuration, boolean flattenYaml) {
+if (configuration.standardYaml && !flattenYaml) {
+return YamlParserUtils.convertAndDumpYamlFromFlatMap(
+Collections.unmodifiableMap(configuration.confData));
+} else {
+Map fileWritableMap = 
configuration.toFileWritableMap();
+return fileWritableMap.entrySet().stream()
+.map(entry -> entry.getKey() + ": " + entry.getValue())
+.collect(Collectors.toList());
+}
+}
+
 /**
  * Creates a dynamic parameter list {@code String} of the passed 
configuration map.
  *
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java 
b/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
index c4388c61d71..ae9280b3a86 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
@@ -42,7 +42,10 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.time.Duration;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -131,6 +134,39 @@ public class YamlParserUtils {
 }
 }
 
+/**
+ * Converts a flat map into a nested map structure and outputs the result 
as a list of
+ * YAML-formatted strings. Each item in the list represents a single line 
of the YAML data. The
+ * method

(flink) branch master updated (4db6e72ed76 -> c148b62166d)

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 4db6e72ed76 [FLINK-34029][runtime-web] Support different profiling 
mode on Flink WEB
 new bcd448b2f1e [hotfix] Migrate BashJavaUtilsITCase and JavaBashTestBase 
to Junit5 and assertJ.
 new 1f7622d4d23 [FLINK-33721][core] Support dump configuration as nested 
yaml data.
 new c148b62166d [FLINK-33721][core] Extend BashJavaUtils to support 
reading and writing standard yaml file.

The 3 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/configuration/ConfigurationUtils.java|  57 ++
 .../flink/configuration/YamlParserUtils.java   |  36 
 .../configuration/ConfigurationUtilsTest.java  |  87 +
 .../flink/configuration/YamlParserUtilsTest.java   |  58 ++
 .../src/main/flink-bin/bin/bash-java-utils.sh  | 170 +
 .../flink-bin/bin/config-parser-utils.sh}  |  32 ++--
 flink-dist/src/main/flink-bin/bin/config.sh| 143 +--
 flink-dist/src/test/bin/runBashJavaUtilsCmd.sh |   2 +-
 .../src/test/bin/runExtractLoggingOutputs.sh   |   2 +-
 .../org/apache/flink/dist/BashJavaUtilsITCase.java | 152 
 .../org/apache/flink/dist/JavaBashTestBase.java|  16 +-
 .../flink/tests/util/flink/FlinkDistribution.java  |   9 +-
 .../decorators/FlinkConfMountDecorator.java|  20 +-
 .../flink/kubernetes/KubernetesTestUtils.java  |  33 ++--
 .../factory/KubernetesJobManagerFactoryTest.java   |  11 +-
 .../runtime/clusterframework/BootstrapTools.java   |   7 +-
 ...on.java => ModifiableClusterConfiguration.java} |  56 --
 ...odifiableClusterConfigurationParserFactory.java |  98 ++
 .../parser/ConfigurationCommandLineOptions.java|  64 +++
 .../runtime/util/ConfigurationParserUtils.java |  68 +++
 .../flink/runtime/util/bash/BashJavaUtils.java |  12 +-
 .../flink/runtime/util/bash/FlinkConfigLoader.java |  18 +-
 ...iableClusterConfigurationParserFactoryTest.java |  96 ++
 .../runtime/util/bash/FlinkConfigLoaderTest.java   | 202 +
 .../testframe/container/FlinkImageBuilder.java |   6 +-
 25 files changed, 1193 insertions(+), 262 deletions(-)
 create mode 100755 flink-dist/src/main/flink-bin/bin/bash-java-utils.sh
 copy flink-dist/src/{test/bin/runExtractLoggingOutputs.sh => 
main/flink-bin/bin/config-parser-utils.sh} (65%)
 copy 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/{ClusterConfiguration.java
 => ModifiableClusterConfiguration.java} (50%)
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ModifiableClusterConfigurationParserFactory.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/parser/ConfigurationCommandLineOptions.java
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ModifiableClusterConfigurationParserFactoryTest.java



(flink) 01/03: [hotfix] Migrate BashJavaUtilsITCase and JavaBashTestBase to Junit5 and assertJ.

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit bcd448b2f1efecc701079c1a0f7f565a59817f22
Author: JunRuiLee 
AuthorDate: Mon Dec 4 16:28:16 2023 +0800

[hotfix] Migrate BashJavaUtilsITCase and JavaBashTestBase to Junit5 and 
assertJ.
---
 .../org/apache/flink/dist/BashJavaUtilsITCase.java | 58 ++
 .../org/apache/flink/dist/JavaBashTestBase.java| 16 +++---
 2 files changed, 34 insertions(+), 40 deletions(-)

diff --git 
a/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java 
b/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java
index a1180da0a78..e9c0e1cf8c2 100644
--- a/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java
+++ b/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java
@@ -25,8 +25,7 @@ import org.apache.flink.runtime.util.bash.BashJavaUtils;
 
 import org.apache.flink.shaded.guava32.com.google.common.collect.Sets;
 
-import org.hamcrest.Matchers;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -35,10 +34,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import static org.hamcrest.collection.IsArrayWithSize.arrayWithSize;
-import static org.hamcrest.collection.IsIn.isIn;
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertThat;
+import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Tests for BashJavaUtils.
@@ -46,7 +42,7 @@ import static org.junit.Assert.assertThat;
  * This test requires the distribution to be assembled and is hence marked 
as an IT case which
  * run after packaging.
  */
-public class BashJavaUtilsITCase extends JavaBashTestBase {
+class BashJavaUtilsITCase extends JavaBashTestBase {
 
 private static final String RUN_BASH_JAVA_UTILS_CMD_SCRIPT =
 "src/test/bin/runBashJavaUtilsCmd.sh";
@@ -54,7 +50,7 @@ public class BashJavaUtilsITCase extends JavaBashTestBase {
 "src/test/bin/runExtractLoggingOutputs.sh";
 
 @Test
-public void testGetTmResourceParamsConfigs() throws Exception {
+void testGetTmResourceParamsConfigs() throws Exception {
 int expectedResultLines = 2;
 String[] commands = {
 RUN_BASH_JAVA_UTILS_CMD_SCRIPT,
@@ -63,13 +59,13 @@ public class BashJavaUtilsITCase extends JavaBashTestBase {
 };
 List lines = 
Arrays.asList(executeScript(commands).split(System.lineSeparator()));
 
-assertThat(lines.size(), is(expectedResultLines));
+assertThat(lines).hasSize(expectedResultLines);
 ConfigurationUtils.parseJvmArgString(lines.get(0));
 ConfigurationUtils.parseTmResourceDynamicConfigs(lines.get(1));
 }
 
 @Test
-public void testGetTmResourceParamsConfigsWithDynamicProperties() throws 
Exception {
+void testGetTmResourceParamsConfigsWithDynamicProperties() throws 
Exception {
 int expectedResultLines = 2;
 double cpuCores = 39.0;
 String[] commands = {
@@ -80,14 +76,15 @@ public class BashJavaUtilsITCase extends JavaBashTestBase {
 };
 List lines = 
Arrays.asList(executeScript(commands).split(System.lineSeparator()));
 
-assertThat(lines.size(), is(expectedResultLines));
+assertThat(lines).hasSize(expectedResultLines);
 Map configs =
 ConfigurationUtils.parseTmResourceDynamicConfigs(lines.get(1));
-
assertThat(Double.valueOf(configs.get(TaskManagerOptions.CPU_CORES.key())), 
is(cpuCores));
+
assertThat(Double.valueOf(configs.get(TaskManagerOptions.CPU_CORES.key(
+.isEqualTo(cpuCores);
 }
 
 @Test
-public void testGetJmResourceParams() throws Exception {
+void testGetJmResourceParams() throws Exception {
 int expectedResultLines = 2;
 String[] commands = {
 RUN_BASH_JAVA_UTILS_CMD_SCRIPT,
@@ -96,7 +93,7 @@ public class BashJavaUtilsITCase extends JavaBashTestBase {
 };
 List lines = 
Arrays.asList(executeScript(commands).split(System.lineSeparator()));
 
-assertThat(lines.size(), is(expectedResultLines));
+assertThat(lines).hasSize(expectedResultLines);
 
 Map jvmParams = 
ConfigurationUtils.parseJvmArgString(lines.get(0));
 Map dynamicParams = 
parseAndAssertDynamicParameters(lines.get(1));
@@ -104,7 +101,7 @@ public class BashJavaUtilsITCase extends JavaBashTestBase {
 }
 
 @Test
-public void testGetJmResourceParamsWithDynamicProperties() throws 
Exception {
+void testGetJmResourceParamsWithDynamicProperties() throws Exception {
 int expectedResultLines = 2;
 long metaspace = 123456789L;
 String[] commands = {
@@ -115,12 +112,12 @@ public class BashJavaUtilsITCase extends JavaBashTestBase 
{
 };

(flink) 02/02: [FLINK-33768] Support dynamic source parallelism inference for batch jobs

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c3c836216eaaaf24c1add3b490c8f425fda01d7c
Author: sunxia 
AuthorDate: Wed Jan 10 19:15:32 2024 +0800

[FLINK-33768] Support dynamic source parallelism inference for batch jobs

This closes #24087.
---
 .../api/connector/source/DynamicFilteringInfo.java |  29 
 .../source/DynamicParallelismInference.java|  66 +
 .../executiongraph/DefaultExecutionGraph.java  |  27 ++--
 .../runtime/executiongraph/ExecutionGraph.java |  11 +-
 .../runtime/executiongraph/ExecutionJobVertex.java |  89 
 .../SpeculativeExecutionJobVertex.java |  13 +-
 .../coordination/OperatorCoordinatorHolder.java|  29 +++-
 .../RecreateOnResetOperatorCoordinator.java|   1 -
 .../DefaultOperatorCoordinatorHandler.java |   6 +-
 .../scheduler/OperatorCoordinatorHandler.java  |   3 +-
 .../adaptivebatch/AdaptiveBatchScheduler.java  | 161 ++---
 ...faultVertexParallelismAndInputInfosDecider.java |  25 +++-
 .../VertexParallelismAndInputInfosDecider.java |  18 +++
 .../source/coordinator/SourceCoordinator.java  |  57 
 .../coordinator/SourceCoordinatorContext.java  |  36 -
 .../DefaultExecutionGraphConstructionTest.java |   8 +-
 .../executiongraph/EdgeManagerBuildUtilTest.java   |  18 +--
 .../executiongraph/ExecutionJobVertexTest.java |  11 +-
 .../DefaultOperatorCoordinatorHandlerTest.java |  11 +-
 .../runtime/scheduler/DefaultSchedulerBuilder.java |  43 --
 .../runtime/scheduler/SchedulerTestingUtils.java   |   5 +-
 .../SsgNetworkMemoryCalculationUtilsTest.java  |  20 +--
 .../adapter/DefaultExecutionTopologyTest.java  |  19 +--
 .../runtime/scheduler/adaptive/ExecutingTest.java  |   8 +-
 .../adaptive/StateTrackingMockExecutionGraph.java  |   3 +-
 .../TestingOperatorCoordinatorHandler.java |   3 +-
 .../adaptivebatch/AdaptiveBatchSchedulerTest.java  |  34 -
 ...tVertexParallelismAndInputInfosDeciderTest.java |  33 -
 .../source/coordinator/SourceCoordinatorTest.java  |  20 +++
 .../source/coordinator/TestingSplitEnumerator.java |   8 +-
 .../connector/source/DynamicFilteringEvent.java|   3 +-
 .../scheduling/AdaptiveBatchSchedulerITCase.java   |  78 --
 32 files changed, 701 insertions(+), 195 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicFilteringInfo.java
 
b/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicFilteringInfo.java
new file mode 100644
index 000..b5ee8ab5c5e
--- /dev/null
+++ 
b/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicFilteringInfo.java
@@ -0,0 +1,29 @@
+/*
+ * 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.api.connector.source;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/**
+ * A decorative interface that indicates it holds the dynamic partition 
filtering data. The actual
+ * information needs to be obtained from the implementing class
+ * org.apache.flink.table.connector.source.DynamicFilteringEvent.
+ */
+@PublicEvolving
+public interface DynamicFilteringInfo {}
diff --git 
a/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicParallelismInference.java
 
b/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicParallelismInference.java
new file mode 100644
index 000..a9ec578c844
--- /dev/null
+++ 
b/flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicParallelismInference.java
@@ -0,0 +1,66 @@
+/*
+ * 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
+ *
+ * Unl

(flink) branch master updated (27e6ac83617 -> c3c836216ea)

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 27e6ac83617 [FLINK-34166][table] Fix KeyedLookupJoinWrapper 
incorrectly process delete message for inner join when previous lookup result 
is empty
 new feb5f926706 [hotfix] Remove redundant lazy initialization of 
LazyInitializedCoordinatorContext.
 new c3c836216ea [FLINK-33768] Support dynamic source parallelism inference 
for batch jobs

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:
 .../connector/source/DynamicFilteringInfo.java}|  16 +-
 .../source/DynamicParallelismInference.java|  66 +
 .../executiongraph/DefaultExecutionGraph.java  |  27 ++--
 .../runtime/executiongraph/ExecutionGraph.java |  11 +-
 .../runtime/executiongraph/ExecutionJobVertex.java |  89 
 .../SpeculativeExecutionJobVertex.java |  13 +-
 .../coordination/OperatorCoordinatorHolder.java|  30 +++-
 .../RecreateOnResetOperatorCoordinator.java|   1 -
 .../DefaultOperatorCoordinatorHandler.java |   6 +-
 .../scheduler/OperatorCoordinatorHandler.java  |   3 +-
 .../adaptivebatch/AdaptiveBatchScheduler.java  | 161 ++---
 ...faultVertexParallelismAndInputInfosDecider.java |  25 +++-
 .../VertexParallelismAndInputInfosDecider.java |  18 +++
 .../source/coordinator/SourceCoordinator.java  |  57 
 .../coordinator/SourceCoordinatorContext.java  |  36 -
 .../DefaultExecutionGraphConstructionTest.java |   8 +-
 .../executiongraph/EdgeManagerBuildUtilTest.java   |  18 +--
 .../executiongraph/ExecutionJobVertexTest.java |  11 +-
 .../DefaultOperatorCoordinatorHandlerTest.java |  11 +-
 .../runtime/scheduler/DefaultSchedulerBuilder.java |  43 --
 .../runtime/scheduler/SchedulerTestingUtils.java   |   5 +-
 .../SsgNetworkMemoryCalculationUtilsTest.java  |  20 +--
 .../adapter/DefaultExecutionTopologyTest.java  |  19 +--
 .../runtime/scheduler/adaptive/ExecutingTest.java  |   8 +-
 .../adaptive/StateTrackingMockExecutionGraph.java  |   3 +-
 .../TestingOperatorCoordinatorHandler.java |   3 +-
 .../adaptivebatch/AdaptiveBatchSchedulerTest.java  |  34 -
 ...tVertexParallelismAndInputInfosDeciderTest.java |  33 -
 .../source/coordinator/SourceCoordinatorTest.java  |  20 +++
 .../source/coordinator/TestingSplitEnumerator.java |   8 +-
 .../connector/source/DynamicFilteringEvent.java|   3 +-
 .../scheduling/AdaptiveBatchSchedulerITCase.java   |  78 --
 32 files changed, 677 insertions(+), 207 deletions(-)
 copy flink-core/src/main/java/org/apache/flink/{core/io/Versioned.java => 
api/connector/source/DynamicFilteringInfo.java} (70%)
 create mode 100644 
flink-core/src/main/java/org/apache/flink/api/connector/source/DynamicParallelismInference.java



(flink) 01/02: [hotfix] Remove redundant lazy initialization of LazyInitializedCoordinatorContext.

2024-01-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit feb5f9267067e549ee40efd691ae756338cc2e0b
Author: sunxia 
AuthorDate: Tue Jan 9 18:54:10 2024 +0800

[hotfix] Remove redundant lazy initialization of 
LazyInitializedCoordinatorContext.
---
 .../flink/runtime/operators/coordination/OperatorCoordinatorHolder.java  | 1 -
 1 file changed, 1 deletion(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
index a8ddf937de2..ec7e35bc066 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java
@@ -149,7 +149,6 @@ public class OperatorCoordinatorHolder
 
 this.globalFailureHandler = globalFailureHandler;
 this.mainThreadExecutor = mainThreadExecutor;
-context.lazyInitialize(globalFailureHandler, mainThreadExecutor, 
checkpointCoordinator);
 
 context.lazyInitialize(globalFailureHandler, mainThreadExecutor, 
checkpointCoordinator);
 setupAllSubtaskGateways();



(flink) branch master updated (a20d57b18bb -> cdf314d30b5)

2024-01-16 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from a20d57b18bb [FLINK-34102][configuration] Fix the invalid configuration 
when using 'env.log.max' on yarn deployment mode
 add cdf314d30b5 [FLINK-34116][test] Enhance 
GlobalConfigurationTest.testInvalidStandardYamlFile for JDK compatibility.

No new revisions were added by this update.

Summary of changes:
 .../flink/configuration/GlobalConfigurationTest.java   | 14 +++---
 1 file changed, 7 insertions(+), 7 deletions(-)



(flink) branch master updated (40cbf231e3d -> eb8af0c589c)

2024-01-15 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 40cbf231e3d [FLINK-33973] Add new interfaces for SinkV2 to synchronize 
the API with the SourceV2 API
 add 8a205367fbf [hotfix] Migrate 
StreamExecutionEnvironmentComplexConfigurationTest to Junit5 and AssertJ
 add 06b37089f0c [FLINK-33980][core] Extend ReadableConfig with toMap 
method.
 add 290b633c426 [FLINK-33980][core] Migrate configuration in 
StreamExecutionEnvironment, ExecutionConfig and CheckpointConfig.
 add 65b8b3baad6 [FLINK-33980][core] Pass job configuration to jobGraph.
 add d02ef1cebf3 [FLINK-33980][core] Support generate 
RestartBackoffTimeStrategyFactory from job configuration.
 add eb8af0c589c [FLINK-33980][core] Support generate StateBackend and 
CheckpointStorage from job configuration.

No new revisions were added by this update.

Summary of changes:
 .../client/program/StreamContextEnvironment.java   |  28 +--
 .../program/StreamContextEnvironmentTest.java  |   3 -
 .../apache/flink/api/common/ExecutionConfig.java   |  36 +++-
 .../flink/api/common/cache/DistributedCache.java   |  18 +-
 .../flink/configuration/ConfigurationUtils.java|   6 +-
 .../apache/flink/configuration/ReadableConfig.java |  11 ++
 .../configuration/RestartStrategyOptions.java  |  41 +++--
 .../StateChangelogOptionsInternal.java |  37 
 .../flink/api/common/ExecutionConfigTest.java  |  29 +--
 .../api/input/StreamOperatorContextBuilder.java|   9 +-
 .../state/api/runtime/SavepointEnvironment.java|   7 +-
 .../flink/state/api/SavepointWriterTest.java   |  11 +-
 .../tests/test_stream_execution_environment.py |   3 +-
 flink-python/pyflink/table/table_config.py |   7 +
 .../java/org/apache/flink/python/PythonConfig.java |   9 +
 .../process/ExternalPythonCoProcessOperator.java   |   1 +
 .../ExternalPythonKeyedCoProcessOperator.java  |   1 +
 .../ExternalPythonKeyedProcessOperator.java|   1 +
 .../process/ExternalPythonProcessOperator.java |   1 +
 .../python/AbstractStatelessFunctionOperator.java  |   1 +
 .../AbstractPythonStreamAggregateOperator.java |   1 +
 .../flink/runtime/checkpoint/Checkpoints.java  |  24 ++-
 .../DefaultExecutionGraphBuilder.java  |   4 +-
 .../RestartBackoffTimeStrategyFactoryLoader.java   |  71 
 .../apache/flink/runtime/jobgraph/JobGraph.java|  18 +-
 .../runtime/scheduler/DefaultSchedulerFactory.java |   1 +
 .../adaptive/AdaptiveSchedulerFactory.java |   1 +
 .../AdaptiveBatchSchedulerFactory.java |   1 +
 .../runtime/state/CheckpointStorageLoader.java |  36 ++--
 .../flink/runtime/state/StateBackendLoader.java|  42 +++--
 .../state/TaskExecutorLocalStateStoresManager.java |   5 +-
 .../util/config/memory/ManagedMemoryUtils.java |   7 +-
 .../flink/runtime/dispatcher/DispatcherTest.java   |   5 +-
 ...estartBackoffTimeStrategyFactoryLoaderTest.java | 114 ++--
 .../runtime/state/CheckpointStorageLoaderTest.java | 131 +-
 .../runtime/state/StateBackendLoadingTest.java |  75 +---
 .../util/config/memory/ManagedMemoryUtilsTest.java |   5 +
 .../ChangelogStateBackendLoadingTest.java  |  34 ++--
 .../state/RocksDBStateBackendFactoryTest.java  |   3 +-
 .../api/environment/CheckpointConfig.java  |  14 ++
 .../environment/StreamExecutionEnvironment.java| 195 ++---
 .../flink/streaming/api/graph/StreamConfig.java|  17 +-
 .../flink/streaming/api/graph/StreamGraph.java |  59 ++-
 .../streaming/api/graph/StreamGraphGenerator.java  |  80 ++---
 .../api/graph/StreamingJobGraphGenerator.java  |  12 +-
 .../api/operators/AbstractStreamOperator.java  |   1 +
 .../api/operators/AbstractStreamOperatorV2.java|   1 +
 .../api/operators/StreamingRuntimeContext.java |   5 +
 .../io/StreamMultipleInputProcessorFactory.java|   1 +
 .../runtime/io/StreamTwoInputProcessorFactory.java |   1 +
 .../runtime/tasks/OneInputStreamTask.java  |   1 +
 .../flink/streaming/runtime/tasks/StreamTask.java  |  18 +-
 ...ecutionEnvironmentComplexConfigurationTest.java | 124 +++--
 .../StreamExecutionEnvironmentTest.java|  26 +--
 .../api/graph/StreamingJobGraphGeneratorTest.java  |   4 +
 .../org/apache/flink/table/api/TableConfig.java|   9 +
 .../planner/plan/nodes/exec/ExecNodeConfig.java|  10 ++
 .../runtime/operators/TableStreamOperator.java |   1 +
 .../window/LocalSlicingWindowAggOperator.java  |   1 +
 .../operators/fusion/FusionStreamOperatorBase.java |   1 +
 .../BatchMultipleInputStreamOperator.java  |   1 +
 .../over/BufferDataOverWindowOperatorTest.java |   1 +
 .../CheckpointAfterAllTasksFinishedITCase.java |  10 +-
 .../checkpointing/RestoreUpgradedJobITCase.java|   1 +
 .../distributedcache

(flink) branch master updated (d2fbe464b1a -> 0153b903d4d)

2024-01-15 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from d2fbe464b1a [FLINK-34077][python] Limits some sphinxcontrib packages 
upper bounds (#24086)
 add ddc7171dd83 [FLINK-33576][core] Introduce new Flink conf file 
'config.yaml' supporting standard YAML syntax.
 add b6b2a0f7abe [FLINK-33576][core] Extend standard yaml parser for 
FLINK-specified List and Map pattern.
 add 0153b903d4d [FLINK-33576][core] Change the UT case to use new conf 
file.

No new revisions were added by this update.

Summary of changes:
 .../apache/flink/client/cli/CliFrontendITCase.java |   4 +-
 .../flink/client/cli/CliFrontendTestUtils.java |   9 +-
 .../testconfig/{flink-conf.yaml => config.yaml}|   0
 .../{flink-conf.yaml => config.yaml}   |   0
 .../{flink-conf.yaml => config.yaml}   |   0
 .../flink/configuration/ConfigurationUtils.java|  98 --
 .../flink/configuration/GlobalConfiguration.java   | 148 +++--
 .../flink/configuration/YamlParserUtils.java   |  19 ++-
 .../configuration/ConfigurationUtilsTest.java  |  26 
 .../configuration/GlobalConfigurationTest.java |  91 -
 .../flink/configuration/YamlParserUtilsTest.java   |   8 ++
 .../flink/tests/util/flink/FlinkDistribution.java  |   5 +-
 .../decorators/FlinkConfMountDecorator.java|   8 +-
 .../flink/kubernetes/KubernetesTestBase.java   |   3 +-
 .../clusterframework/BootstrapToolsTest.java   |  41 +-
 .../runtime/util/bash/FlinkConfigLoaderTest.java   |  48 +--
 .../apache/flink/table/client/SqlClientTest.java   |   4 +-
 .../flink-sql-client/src/test/resources/sql/set.q  |  20 +--
 .../apache/flink/table/gateway/SqlGatewayTest.java |   6 +-
 .../service/utils/SqlGatewayServiceExtension.java  |   4 +-
 .../flink-sql-gateway/src/test/resources/sql/set.q |   8 +-
 .../testframe/container/FlinkImageBuilder.java |   8 +-
 .../java/org/apache/flink/yarn/YarnTestBase.java   |   3 +
 .../apache/flink/yarn/YarnClusterDescriptor.java   |   9 +-
 .../yarn/entrypoint/YarnEntrypointUtilsTest.java   |   2 +-
 25 files changed, 461 insertions(+), 111 deletions(-)
 rename flink-clients/src/test/resources/testconfig/{flink-conf.yaml => 
config.yaml} (100%)
 rename 
flink-clients/src/test/resources/testconfigwithinvalidyarn/{flink-conf.yaml => 
config.yaml} (100%)
 rename flink-clients/src/test/resources/testconfigwithyarn/{flink-conf.yaml => 
config.yaml} (100%)



(flink) branch master updated: [FLINK-33968][runtime] Advance the calculation of num of subpartitions to the time of initializing execution job vertex

2024-01-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new b25dfaee807 [FLINK-33968][runtime] Advance the calculation of num of 
subpartitions to the time of initializing execution job vertex
b25dfaee807 is described below

commit b25dfaee80727d6662a5fd445fe51cc139a8b9eb
Author: Lijie Wang 
AuthorDate: Wed Dec 27 23:13:05 2023 +0800

[FLINK-33968][runtime] Advance the calculation of num of subpartitions to 
the time of initializing execution job vertex

This closes #24019.
---
 .../IntermediateResultPartition.java   | 44 --
 1 file changed, 24 insertions(+), 20 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
index 00e99674371..e132f9079f1 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
@@ -43,8 +43,8 @@ public class IntermediateResultPartition {
 
 private final EdgeManager edgeManager;
 
-/** Number of subpartitions. Initialized lazily and will not change once 
set. */
-private int numberOfSubpartitions = NUM_SUBPARTITIONS_UNKNOWN;
+/** Number of subpartitions for dynamic graph. */
+private final int numberOfSubpartitionsForDynamicGraph;
 
 /** Whether this partition has produced all data. */
 private boolean dataAllProduced = false;
@@ -64,6 +64,17 @@ public class IntermediateResultPartition {
 this.producer = producer;
 this.partitionId = new 
IntermediateResultPartitionID(totalResult.getId(), partitionNumber);
 this.edgeManager = edgeManager;
+
+if (!producer.getExecutionGraphAccessor().isDynamic()) {
+this.numberOfSubpartitionsForDynamicGraph = 
NUM_SUBPARTITIONS_UNKNOWN;
+} else {
+this.numberOfSubpartitionsForDynamicGraph =
+computeNumberOfSubpartitionsForDynamicGraph();
+checkState(
+numberOfSubpartitionsForDynamicGraph > 0,
+"Number of subpartitions is an unexpected value: "
++ numberOfSubpartitionsForDynamicGraph);
+}
 }
 
 public void markPartitionGroupReleasable(ConsumedPartitionGroup 
partitionGroup) {
@@ -114,17 +125,6 @@ public class IntermediateResultPartition {
 }
 
 public int getNumberOfSubpartitions() {
-if (numberOfSubpartitions == NUM_SUBPARTITIONS_UNKNOWN) {
-numberOfSubpartitions = computeNumberOfSubpartitions();
-checkState(
-numberOfSubpartitions > 0,
-"Number of subpartitions is an unexpected value: " + 
numberOfSubpartitions);
-}
-
-return numberOfSubpartitions;
-}
-
-private int computeNumberOfSubpartitions() {
 if (!getProducer().getExecutionGraphAccessor().isDynamic()) {
 List consumerVertexGroups = 
getConsumerVertexGroups();
 checkState(!consumerVertexGroups.isEmpty());
@@ -134,13 +134,17 @@ public class IntermediateResultPartition {
 // for non-dynamic graph.
 return consumerVertexGroups.get(0).size();
 } else {
-if (totalResult.isBroadcast()) {
-// for dynamic graph and broadcast result, we only produced 
one subpartition,
-// and all the downstream vertices should consume this 
subpartition.
-return 1;
-} else {
-return computeNumberOfMaxPossiblePartitionConsumers();
-}
+return numberOfSubpartitionsForDynamicGraph;
+}
+}
+
+private int computeNumberOfSubpartitionsForDynamicGraph() {
+if (totalResult.isBroadcast()) {
+// for dynamic graph and broadcast result, we only produced one 
subpartition,
+// and all the downstream vertices should consume this 
subpartition.
+return 1;
+} else {
+return computeNumberOfMaxPossiblePartitionConsumers();
 }
 }
 



(flink) branch master updated: [FLINK-33713][core] Deprecate RuntimeContext#getExecutionConfig

2023-12-11 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 3aa70df4e7d [FLINK-33713][core] Deprecate 
RuntimeContext#getExecutionConfig
3aa70df4e7d is described below

commit 3aa70df4e7da93ed32c26cfabdaeb606233419b1
Author: JunRuiLee 
AuthorDate: Wed Nov 8 11:48:45 2023 +0800

[FLINK-33713][core] Deprecate RuntimeContext#getExecutionConfig

This closes #23848.
---
 .../mapred/HadoopReduceCombineFunction.java|  3 +-
 .../mapred/HadoopReduceFunction.java   |  3 +-
 .../flink/api/common/functions/RuntimeContext.java | 33 +
 .../api/common/functions/SerializerFactory.java| 38 
 .../functions/util/AbstractRuntimeUDFContext.java  | 19 ++
 .../flink/api/common/state/StateDescriptor.java| 15 +++-
 .../flink/cep/operator/CepRuntimeContext.java  | 19 ++
 .../flink/cep/operator/CepRuntimeContextTest.java  | 12 ---
 .../flink/state/api/EvictingWindowReader.java  |  3 +-
 .../state/api/EvictingWindowSavepointReader.java   |  3 +-
 .../apache/flink/state/api/ExistingSavepoint.java  | 39 
 .../apache/flink/state/api/SavepointReader.java| 12 ---
 .../org/apache/flink/state/api/WindowReader.java   |  3 +-
 .../flink/state/api/WindowSavepointReader.java |  3 +-
 .../state/api/input/BroadcastStateInputFormat.java |  8 +++--
 .../state/api/input/KeyedStateInputFormat.java | 41 ++
 .../state/api/input/ListStateInputFormat.java  |  9 +++--
 .../state/api/input/OperatorStateInputFormat.java  | 25 +++--
 .../api/input/StreamOperatorContextBuilder.java|  9 +++--
 .../state/api/input/UnionStateInputFormat.java |  9 +++--
 .../api/input/operator/StateReaderOperator.java| 14 
 .../api/input/operator/WindowReaderOperator.java   |  4 +--
 .../api/output/BootstrapStreamTaskRunner.java  |  3 +-
 .../output/BoundedOneInputStreamTaskRunner.java|  5 ++-
 .../state/api/runtime/SavepointEnvironment.java| 13 +--
 .../state/api/runtime/SavepointRuntimeContext.java | 19 ++
 .../api/input/BroadcastStateInputFormatTest.java   |  4 ++-
 .../state/api/input/KeyedStateInputFormatTest.java | 22 
 .../state/api/input/ListStateInputFormatTest.java  | 13 +--
 .../input/StreamOperatorContextBuilderTest.java|  6 ++--
 .../state/api/input/UnionStateInputFormatTest.java |  4 ++-
 .../flink/state/api/input/WindowReaderTest.java| 16 ++---
 .../org/apache/flink/python/util/ProtoUtils.java   |  6 ++--
 .../AbstractPythonStreamAggregateOperator.java |  3 +-
 ...wPythonOverWindowAggregateFunctionOperator.java |  3 +-
 .../runtime/state/DefaultKeyedStateStore.java  | 18 +-
 .../scala/operators/ScalaAggregateOperator.java|  3 +-
 .../api/functions/async/RichAsyncFunction.java | 19 ++
 .../source/ContinuousFileReaderOperator.java   |  2 +-
 .../source/InputFormatSourceFunction.java  |  2 +-
 .../api/operators/StreamOperatorStateHandler.java  | 13 ++-
 .../streaming/api/operators/StreamSource.java  |  3 +-
 .../api/operators/StreamingRuntimeContext.java | 10 +++---
 .../runtime/operators/sink/SinkWriterOperator.java |  2 +-
 .../operators/windowing/WindowOperator.java| 11 +-
 .../api/functions/async/RichAsyncFunctionTest.java | 14 +---
 .../api/operators/StreamingRuntimeContextTest.java | 32 +++--
 .../flink/table/functions/FunctionContext.java |  7 +---
 .../org/apache/flink/test/operators/MapITCase.java | 22 ++--
 49 files changed, 461 insertions(+), 138 deletions(-)

diff --git 
a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
 
b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
index 4db71d76d7b..fc9981379f8 100644
--- 
a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
+++ 
b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
@@ -117,8 +117,7 @@ public final class HadoopReduceCombineFunction
 Class inKeyClass =
 (Class) TypeExtractor.getParameterType(Reducer.class, 
reducer.getClass(), 0);
 TypeSerializer keySerializer =
-TypeExtractor.getForClass(inKeyClass)
-
.createSerializer(getRuntimeContext().getExecutionConfig());
+
getRuntimeContext().createSerializer(TypeExtractor.getForClass(inKeyClass));
 this.valueIterator = new 
HadoopTupleUnwrappingIterator<>(keySerializer);
 this.combineCol

(flink) 01/03: [FLINK-33669][doc] Update the usage of configuring state backend in docs.

2023-12-05 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e27f8a3a0783d551457a2f424b01267bd1c8c2c2
Author: JunRuiLee 
AuthorDate: Mon Nov 20 20:17:22 2023 +0800

[FLINK-33669][doc] Update the usage of configuring state backend in docs.
---
 .../docs/deployment/filesystems/azure.md   |  7 ++-
 docs/content.zh/docs/deployment/filesystems/oss.md |  7 ++-
 docs/content.zh/docs/deployment/filesystems/s3.md  |  7 ++-
 .../datastream/fault-tolerance/checkpointing.md|  7 ++-
 .../datastream/fault-tolerance/state_backends.md   | 10 +++--
 docs/content.zh/docs/ops/state/checkpoints.md  |  6 ++-
 docs/content.zh/docs/ops/state/state_backends.md   | 50 +-
 .../datastream/fault-tolerance/state_backends.md   | 10 +++--
 docs/content/docs/ops/state/state_backends.md  | 50 +-
 9 files changed, 98 insertions(+), 56 deletions(-)

diff --git a/docs/content.zh/docs/deployment/filesystems/azure.md 
b/docs/content.zh/docs/deployment/filesystems/azure.md
index 42a0d09ee96..1482a302dac 100644
--- a/docs/content.zh/docs/deployment/filesystems/azure.md
+++ b/docs/content.zh/docs/deployment/filesystems/azure.md
@@ -63,8 +63,11 @@ 
env.readTextFile("wasb://@$.blob.core.window
 // 写入 Azure Blob 存储
 
stream.writeAsText("wasb://@$.blob.core.windows.net/");
 
-// 将 Azure Blob 存储用作 FsStatebackend
-env.setStateBackend(new 
FsStateBackend("wasb://@$.blob.core.windows.net/"));
+// 将 Azure Blob 存储用作 checkpoint storage
+Configuration config = new Configuration();
+config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
+config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
"wasb://@$.blob.core.windows.net/");
+env.configure(config);
 ```
 
 ## Shaded Hadoop Azure Blob 存储文件系统
diff --git a/docs/content.zh/docs/deployment/filesystems/oss.md 
b/docs/content.zh/docs/deployment/filesystems/oss.md
index e1ca862276c..5a210ad9954 100644
--- a/docs/content.zh/docs/deployment/filesystems/oss.md
+++ b/docs/content.zh/docs/deployment/filesystems/oss.md
@@ -48,8 +48,11 @@ env.readTextFile("oss:///");
 // 写入 OSS bucket
 stream.writeAsText("oss:///");
 
-// 将 OSS 用作 FsStatebackend
-env.setStateBackend(new FsStateBackend("oss:///"));
+// 将 OSS 用作 checkpoint storage
+Configuration config = new Configuration();
+config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
+config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
"oss:///");
+env.configure(config);
 ```
 
 ### Shaded Hadoop OSS 文件系统
diff --git a/docs/content.zh/docs/deployment/filesystems/s3.md 
b/docs/content.zh/docs/deployment/filesystems/s3.md
index 5bb316b038b..5f9a895c786 100644
--- a/docs/content.zh/docs/deployment/filesystems/s3.md
+++ b/docs/content.zh/docs/deployment/filesystems/s3.md
@@ -46,8 +46,11 @@ env.readTextFile("s3:///");
 // 写入 S3 bucket
 stream.writeAsText("s3:///");
 
-// 使用 S3 作为 FsStatebackend
-env.setStateBackend(new FsStateBackend("s3:///"));
+// 使用 S3 作为 checkpoint storage
+Configuration config = new Configuration();
+config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
+config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
"s3:///");
+env.configure(config);
 ```
 
 注意这些例子并*不详尽*,S3 同样可以用在其他场景,包括 [JobManager 高可用配置]({{< ref 
"docs/deployment/ha/overview" >}}) 或 [RocksDBStateBackend]({{< ref 
"docs/ops/state/state_backends" >}}#the-rocksdbstatebackend),以及所有 Flink 
需要使用文件系统 URI 的位置。
diff --git 
a/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md 
b/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md
index 17ade58ae72..5f10720e592 100644
--- a/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md
+++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/checkpointing.md
@@ -196,7 +196,12 @@ Flink 的 [checkpointing 机制]({{< ref 
"docs/learn-flink/fault_tolerance" >}})
 Checkpoint 存储在哪里取决于所配置的 **State Backend**(比如 JobManager memory、 file system、 
database)。
 
 默认情况下,状态是保持在 TaskManagers 的内存中,checkpoint 保存在 JobManager 的内存中。为了合适地持久化大体量状态,
-Flink 支持各种各样的途径去存储 checkpoint 状态到其他的 state backends 上。通过 
`StreamExecutionEnvironment.setStateBackend(…)` 来配置所选的 state backends。
+Flink 支持各种各样的途径去存储 checkpoint 状态到其他的 state backends 上。可以通过如下代码块来配置:
+```java
+Configuration config = new Configuration();
+config.set(StateBackendOptions.STATE_BACKEND, "hashmap");
+env.configure(config);
+```
 
 阅读 [state backends]({{< ref "docs/ops/state/state_backends" >}}) 来查看在 job 
范围和集群范围上可用的 state backends 与选项的更多细节。
 
diff --git 
a/docs/content.zh/docs/dev/datastream/fault-tolerance/state_backends.md 
b/docs/content.zh/docs/dev/datastream/fault-tolerance/state_backends.md
index 68821981731..9234fe78a89 100644
--- a/docs/content.zh/docs

(flink) branch master updated (18c03f2e6c5 -> 52d8d3583e5)

2023-12-05 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 18c03f2e6c5 [FLINK-28229][connectors] Deprecate 
StreamExecutionEnvironment#fromCollection()
 new e27f8a3a078 [FLINK-33669][doc] Update the usage of configuring state 
backend in docs.
 new d9bcb3b40ed [FLINK-33669][doc] Update the usage of configuring 
checkpoint storage in docs.
 new 52d8d3583e5 [FLINK-33669][doc] Update the usage of configuring restart 
strategy in docs.

The 3 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:
 .../docs/deployment/filesystems/azure.md   |  7 +-
 docs/content.zh/docs/deployment/filesystems/gcs.md | 11 ++-
 docs/content.zh/docs/deployment/filesystems/oss.md |  7 +-
 docs/content.zh/docs/deployment/filesystems/s3.md  |  7 +-
 .../datastream/fault-tolerance/checkpointing.md|  7 +-
 .../datastream/fault-tolerance/state_backends.md   | 10 ++-
 docs/content.zh/docs/ops/state/checkpoints.md  |  6 +-
 docs/content.zh/docs/ops/state/state_backends.md   | 78 +++
 .../docs/ops/state/task_failure_recovery.md| 76 +-
 docs/content/docs/deployment/filesystems/azure.md  |  5 +-
 docs/content/docs/deployment/filesystems/gcs.md|  5 +-
 docs/content/docs/deployment/filesystems/oss.md|  5 +-
 docs/content/docs/deployment/filesystems/s3.md |  5 +-
 .../datastream/fault-tolerance/checkpointing.md| 13 +++-
 .../datastream/fault-tolerance/state_backends.md   | 10 ++-
 docs/content/docs/ops/state/checkpoints.md | 12 ++-
 docs/content/docs/ops/state/state_backends.md  | 78 +++
 .../docs/ops/state/task_failure_recovery.md| 91 +++---
 18 files changed, 259 insertions(+), 174 deletions(-)



(flink) 03/03: [FLINK-33669][doc] Update the usage of configuring restart strategy in docs.

2023-12-05 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 52d8d3583e5c989da84126a8805ab335408c46c2
Author: JunRuiLee 
AuthorDate: Thu Nov 30 22:12:06 2023 +0800

[FLINK-33669][doc] Update the usage of configuring restart strategy in docs.

This closes #23847.
---
 .../docs/ops/state/task_failure_recovery.md| 76 +-
 .../docs/ops/state/task_failure_recovery.md| 91 +++---
 2 files changed, 84 insertions(+), 83 deletions(-)

diff --git a/docs/content.zh/docs/ops/state/task_failure_recovery.md 
b/docs/content.zh/docs/ops/state/task_failure_recovery.md
index 3a9dd02e423..234b29578c8 100644
--- a/docs/content.zh/docs/ops/state/task_failure_recovery.md
+++ b/docs/content.zh/docs/ops/state/task_failure_recovery.md
@@ -49,8 +49,6 @@ Flink 作业如果没有定义重启策略,则会遵循集群启动时加载
 {{< generated/restart_strategy_configuration >}}
 
 除了定义默认的重启策略以外,还可以为每个 Flink 作业单独定义重启策略。
-这个重启策略通过在程序中的 `StreamExecutionEnvironment` 对象上调用 `setRestartStrategy` 方法来设置。
-当然,对于 `StreamExecutionEnvironment` 也同样适用。
 
 下例展示了如何给我们的作业设置固定延时重启策略。
 如果发生故障,系统会重启作业 3 次,每两次连续的重启尝试之间等待 10 秒钟。
@@ -58,11 +56,11 @@ Flink 作业如果没有定义重启策略,则会遵循集群启动时加载
 {{< tabs "2b011473-9a34-4e7b-943b-be4a9071fe3c" >}}
 {{< tab "Java" >}}
 ```java
-StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
-  3, // 尝试重启的次数
-  Time.of(10, TimeUnit.SECONDS) // 延时
-));
+Configuration config = new Configuration();
+config.set(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay");
+config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 3); 
// 尝试重启的次数
+config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, 
Duration.ofSeconds(10)); // 延时
+StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(config);
 ```
 {{< /tab >}}
 {{< tab "Scala" >}}
@@ -76,11 +74,11 @@ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
 {{< /tab >}}
 {{< tab "Python" >}}
 ```python
-env = StreamExecutionEnvironment.get_execution_environment()
-env.set_restart_strategy(RestartStrategies.fixed_delay_restart(
-3,  # 尝试重启的次数
-1  # 延时(毫秒)
-))
+config = Configuration()
+config.set_string('restart-strategy.type', 'fixed-delay')
+config.set_string('restart-strategy.fixed-delay.attempts', '3') # 尝试重启的次数
+config.set_string('restart-strategy.fixed-delay.delay', '1 ms') # 延时
+env = StreamExecutionEnvironment.get_execution_environment(config)
 ```
 {{< /tab >}}
 {{< /tabs >}}
@@ -115,11 +113,11 @@ restart-strategy.fixed-delay.delay: 10 s
 {{< tabs "0877201b-96aa-4985-aebd-0780cf1d8e9e" >}}
 {{< tab "Java" >}}
 ```java
-StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
-  3, // 尝试重启的次数
-  Time.of(10, TimeUnit.SECONDS) // 延时
-));
+Configuration config = new Configuration();
+config.set(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay");
+config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 3); 
// 尝试重启次数
+config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, 
Duration.ofSeconds(10)); // 延时
+StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(config);
 ```
 {{< /tab >}}
 {{< tab "Scala" >}}
@@ -133,11 +131,11 @@ 
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(
 {{< /tab >}}
 {{< tab "Python" >}}
 ```python
-env = StreamExecutionEnvironment.get_execution_environment()
-env.set_restart_strategy(RestartStrategies.fixed_delay_restart(
-3,  # 尝试重启的次数
-1  # 延时(毫秒)
-))
+config = Configuration()
+config.set_string('restart-strategy.type', 'fixed-delay')
+config.set_string('restart-strategy.fixed-delay.attempts', '3') # 尝试重启的次数
+config.set_string('restart-strategy.fixed-delay.delay', '1 ms') # 延时
+env = StreamExecutionEnvironment.get_execution_environment(config)
 ```
 {{< /tab >}}
 {{< /tabs >}}
@@ -227,12 +225,12 @@ restart-strategy.failure-rate.delay: 10 s
 {{< tabs "f4fba671-e1a8-408d-9f3d-d679aa6473ea" >}}
 {{< tab "Java" >}}
 ```java
-StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-env.setRestartStrategy(RestartStrategies.failureRateRestart(
-  3, // 每个时间间隔的最大故障次数
-  Time.of(5, TimeUnit.MINUTES), // 测量故障率的时间间隔
-  Time.of(10, TimeUnit.SECONDS) // 延时
-));
+Configuration config = new Configuration();
+config.set(RestartStrategyOptions.RESTART_STRATEGY, "failure-rate");
+config.set(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL,
 3); // 每个时间间隔的最大故障次数
+config.set(Restar

(flink) 02/03: [FLINK-33669][doc] Update the usage of configuring checkpoint storage in docs.

2023-12-05 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit d9bcb3b40ed5cefadbbaf391dacaa0ecd7fc8243
Author: JunRuiLee 
AuthorDate: Tue Nov 21 10:06:10 2023 +0800

[FLINK-33669][doc] Update the usage of configuring checkpoint storage in 
docs.
---
 docs/content.zh/docs/deployment/filesystems/gcs.md | 11 +
 docs/content.zh/docs/ops/state/state_backends.md   | 28 +-
 docs/content/docs/deployment/filesystems/azure.md  |  5 +++-
 docs/content/docs/deployment/filesystems/gcs.md|  5 +++-
 docs/content/docs/deployment/filesystems/oss.md|  5 +++-
 docs/content/docs/deployment/filesystems/s3.md |  5 +++-
 .../datastream/fault-tolerance/checkpointing.md| 13 --
 docs/content/docs/ops/state/checkpoints.md | 12 +++---
 docs/content/docs/ops/state/state_backends.md  | 28 +-
 9 files changed, 77 insertions(+), 35 deletions(-)

diff --git a/docs/content.zh/docs/deployment/filesystems/gcs.md 
b/docs/content.zh/docs/deployment/filesystems/gcs.md
index f80e5b3af4a..47cd137db33 100644
--- a/docs/content.zh/docs/deployment/filesystems/gcs.md
+++ b/docs/content.zh/docs/deployment/filesystems/gcs.md
@@ -37,14 +37,17 @@ gs:///
 The endpoint can either be a single file or a directory, for example:
 
 ```java
-// Read from GCS bucket
+// 读取 GCS bucket
 env.readTextFile("gs:///");
 
-// Write to GCS bucket
+// 写入 GCS bucket
 stream.writeAsText("gs:///");
 
-// Use GCS as checkpoint storage
-env.getCheckpointConfig().setCheckpointStorage("gs:///");
+// 将 GCS 用作 checkpoint storage
+Configuration config = new Configuration();
+config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem");
+config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
"gs:///");
+env.configure(config);
 
 ```
 
diff --git a/docs/content.zh/docs/ops/state/state_backends.md 
b/docs/content.zh/docs/ops/state/state_backends.md
index bca2bc7af5b..eda37dada7e 100644
--- a/docs/content.zh/docs/ops/state/state_backends.md
+++ b/docs/content.zh/docs/ops/state/state_backends.md
@@ -516,8 +516,8 @@ env.getCheckpointConfig().setCheckpointStorage(new 
JobManagerCheckpointStorage)
 ```python
 config = Configuration()
 config.set_string('state.backend.type', 'hashmap')
+config.set_string('state.checkpoint-storage', 'jobmanager')
 env = StreamExecutionEnvironment.get_execution_environment(config)
-env.get_checkpoint_config().set_checkpoint_storage(JobManagerCheckpointStorage())
 ```
 {{< /tab >}}
 {{< /tabs>}}
@@ -550,8 +550,9 @@ env.configure(config);
 
 
 // Advanced FsStateBackend configurations, such as write buffer size
-// can be set by manually instantiating a FileSystemCheckpointStorage object.
-env.getCheckpointConfig().setCheckpointStorage(new 
FileSystemCheckpointStorage("file:///checkpoint-dir"));
+// can be set manually by using CheckpointingOptions.
+config.set(CheckpointingOptions.FS_WRITE_BUFFER_SIZE, 4 * 1024);
+env.configure(config);
 ```
 {{< /tab >}}
 {{< tab "Scala" >}}
@@ -570,13 +571,15 @@ env.getCheckpointConfig().setCheckpointStorage(new 
FileSystemCheckpointStorage("
 ```python
 config = Configuration()
 config.set_string('state.backend.type', 'hashmap')
+config.set_string('state.checkpoint-storage', 'filesystem')
+config.set_string('state.checkpoints.dir', 'file:///checkpoint-dir')
 env = StreamExecutionEnvironment.get_execution_environment(config)
-env.get_checkpoint_config().set_checkpoint_storage_dir("file:///checkpoint-dir")
 
 
 # Advanced FsStateBackend configurations, such as write buffer size
-# can be set by manually instantiating a FileSystemCheckpointStorage object.
-env.get_checkpoint_config().set_checkpoint_storage(FileSystemCheckpointStorage("file:///checkpoint-dir"))
+# can be set manually by using CheckpointingOptions.
+config.set_string('state.storage.fs.write-buffer-size', '4096');
+env.configure(config);
 ```
 {{< /tab >}}
 {{< /tabs>}}
@@ -610,8 +613,9 @@ env.configure(config);
 
 // If you manually passed FsStateBackend into the RocksDBStateBackend 
constructor
 // to specify advanced checkpointing configurations such as write buffer size,
-// you can achieve the same results by using manually instantiating a 
FileSystemCheckpointStorage object.
-env.getCheckpointConfig().setCheckpointStorage(new 
FileSystemCheckpointStorage("file:///checkpoint-dir"));
+// you can achieve the same results by using CheckpointingOptions.
+config.set(CheckpointingOptions.FS_WRITE_BUFFER_SIZE, 4 * 1024);
+env.configure(config);
 ```
 {{< /tab >}}
 {{< tab "Scala" >}}
@@ -631,14 +635,16 @@ env.getCheckpointConfig().setCheckpointStorage(new 
FileSystemCheckpointStorage("
 ```python
 config = Configuration()
 config.set_string(

(flink) 03/03: [FLINK-33581][python] Deprecate getter/setter methods related to state backend in python flink.

2023-12-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 7eaa5db30244b8b5d9cdc6ab0cb327e255d6fadc
Author: JunRuiLee 
AuthorDate: Mon Dec 4 11:58:50 2023 +0800

[FLINK-33581][python] Deprecate getter/setter methods related to state 
backend in python flink.

This closes #23864.
---
 flink-python/pyflink/datastream/stream_execution_environment.py | 7 +++
 1 file changed, 7 insertions(+)

diff --git a/flink-python/pyflink/datastream/stream_execution_environment.py 
b/flink-python/pyflink/datastream/stream_execution_environment.py
index 55543c67b80..d1b7ce41118 100644
--- a/flink-python/pyflink/datastream/stream_execution_environment.py
+++ b/flink-python/pyflink/datastream/stream_execution_environment.py
@@ -331,7 +331,14 @@ class StreamExecutionEnvironment(object):
 
 :param state_backend: The :class:`StateBackend`.
 :return: This object.
+
+.. note:: Deprecated since version 1.19: This method is deprecated and 
will be removed in
+  future FLINK major version. Use 
`stream_execution_environment.configure` method
+  instead to set the state backend.
 """
+warnings.warn("Deprecated since version 1.19: This method is 
deprecated and will be removed"
+  " in future FLINK major version. Use 
`stream_execution_environment.configure`"
+  " method instead to set the state backend.", 
DeprecationWarning)
 self._j_stream_execution_environment = \
 
self._j_stream_execution_environment.setStateBackend(state_backend._j_state_backend)
 return self



(flink) 02/03: [FLINK-33581][python] Deprecate getter/setter methods related to CheckpointStorage in python flink.

2023-12-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 4a7def9fe00387df66af8daf23603a6ea0848e03
Author: JunRuiLee 
AuthorDate: Mon Dec 4 11:56:54 2023 +0800

[FLINK-33581][python] Deprecate getter/setter methods related to 
CheckpointStorage in python flink.
---
 .../pyflink/datastream/checkpoint_config.py| 22 ++
 1 file changed, 22 insertions(+)

diff --git a/flink-python/pyflink/datastream/checkpoint_config.py 
b/flink-python/pyflink/datastream/checkpoint_config.py
index f03ecf26abb..232effd1e0b 100644
--- a/flink-python/pyflink/datastream/checkpoint_config.py
+++ b/flink-python/pyflink/datastream/checkpoint_config.py
@@ -15,6 +15,7 @@
 #  See the License for the specific language governing permissions and
 # limitations under the License.
 

+import warnings
 from enum import Enum
 from typing import Optional
 
@@ -440,7 +441,14 @@ class CheckpointConfig(object):
 HDFS, NFS Drivs, S3, and GCS, this storage policy supports large state 
size, in the
 magnitude of many terabytes while providing a highly available 
foundation for stateful
 applications. This checkpoint storage policy is recommended for most 
production deployments.
+
+.. note:: Deprecated since version 1.19: This method is deprecated and 
will be removed in
+  future FLINK major version. Use 
`stream_execution_environment.configure` method
+  instead to set the checkpoint storage.
 """
+warnings.warn("Deprecated since version 1.19: This method is 
deprecated and will be removed"
+  " in future FLINK major version. Use 
`stream_execution_environment.configure`"
+  " method instead to set the checkpoint storage.", 
DeprecationWarning)
 
self._j_checkpoint_config.setCheckpointStorage(storage._j_checkpoint_storage)
 return self
 
@@ -448,7 +456,14 @@ class CheckpointConfig(object):
 """
 Configures the application to write out checkpoint snapshots to the 
configured directory.
 See `FileSystemCheckpointStorage` for more details on checkpointing to 
a file system.
+
+.. note:: Deprecated since version 1.19: This method is deprecated and 
will be removed in
+  future FLINK major version. Use 
`stream_execution_environment.configure` method
+  instead to set the checkpoint storage.
 """
+warnings.warn("Deprecated since version 1.19: This method is 
deprecated and will be removed"
+  " in future FLINK major version. Use 
`stream_execution_environment.configure`"
+  " method instead to set the checkpoint storage.", 
DeprecationWarning)
 self._j_checkpoint_config.setCheckpointStorage(checkpoint_path)
 return self
 
@@ -456,7 +471,14 @@ class CheckpointConfig(object):
 """
 The checkpoint storage that has been configured for the Job, or None if
 none has been set.
+
+.. note:: Deprecated since version 1.19: This method is deprecated and 
will be removed in
+  future FLINK major version. It is recommended to find which 
checkpoint storage is
+  used by checkpoint storage ConfigOption.
 """
+warnings.warn("Deprecated since version 1.19: This method is 
deprecated and will be removed"
+  " in future FLINK major version. It is recommended to 
find which checkpoint"
+  " storage is used by checkpoint storage ConfigOption.", 
DeprecationWarning)
 j_storage = self._j_checkpoint_config.getCheckpointStorage()
 if j_storage is None:
 return None



(flink) branch master updated (53367b08900 -> 7eaa5db3024)

2023-12-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 53367b08900 [FLINK-33731][Scheduler] Remove the redundant flip1 package
 new 17b82a7662c [FLINK-33581][python] Deprecate getter/setter methods 
related in the RestartStrategy in python flink.
 new 4a7def9fe00 [FLINK-33581][python] Deprecate getter/setter methods 
related to CheckpointStorage in python flink.
 new 7eaa5db3024 [FLINK-33581][python] Deprecate getter/setter methods 
related to state backend in python flink.

The 3 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-python/pyflink/common/execution_config.py|  7 +++
 .../pyflink/datastream/checkpoint_config.py| 22 ++
 .../datastream/stream_execution_environment.py | 15 +++
 3 files changed, 44 insertions(+)



(flink) 01/03: [FLINK-33581][python] Deprecate getter/setter methods related in the RestartStrategy in python flink.

2023-12-04 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 17b82a7662c76dc9f41125782ba5232bb1a7eea4
Author: JunRuiLee 
AuthorDate: Mon Dec 4 11:52:55 2023 +0800

[FLINK-33581][python] Deprecate getter/setter methods related in the 
RestartStrategy in python flink.
---
 flink-python/pyflink/common/execution_config.py | 7 +++
 flink-python/pyflink/datastream/stream_execution_environment.py | 8 
 2 files changed, 15 insertions(+)

diff --git a/flink-python/pyflink/common/execution_config.py 
b/flink-python/pyflink/common/execution_config.py
index 4e7ceae37a4..fd42550c518 100644
--- a/flink-python/pyflink/common/execution_config.py
+++ b/flink-python/pyflink/common/execution_config.py
@@ -263,7 +263,14 @@ class ExecutionConfig(object):
 The restart strategy configurations are all created from 
:class:`RestartStrategies`.
 
 :param restart_strategy_configuration: Configuration defining the 
restart strategy to use.
+
+.. note:: Deprecated since version 1.19: This method is deprecated and 
will be removed in
+  future FLINK major version. Use 
`stream_execution_environment.configure` method
+  instead to set the restart strategy.
 """
+warnings.warn("Deprecated since version 1.19: This method is 
deprecated and will be removed"
+  " in future FLINK major version. Use 
`stream_execution_environment.configure`"
+  " method instead to set the restart strategy.", 
DeprecationWarning)
 self._j_execution_config.setRestartStrategy(
 restart_strategy_configuration._j_restart_strategy_configuration)
 return self
diff --git a/flink-python/pyflink/datastream/stream_execution_environment.py 
b/flink-python/pyflink/datastream/stream_execution_environment.py
index 1c3c3f43e8b..55543c67b80 100644
--- a/flink-python/pyflink/datastream/stream_execution_environment.py
+++ b/flink-python/pyflink/datastream/stream_execution_environment.py
@@ -17,6 +17,7 @@
 

 import os
 import tempfile
+import warnings
 
 from typing import List, Any, Optional, cast
 
@@ -424,7 +425,14 @@ class StreamExecutionEnvironment(object):
 
 :param restart_strategy_configuration: Restart strategy configuration 
to be set.
 :return:
+
+.. note:: Deprecated since version 1.19: This method is deprecated and 
will be removed in
+  future FLINK major version. Use 
`stream_execution_environment.configure` method
+  instead to set the restart strategy.
 """
+warnings.warn("Deprecated since version 1.19: This method is 
deprecated and will be removed"
+  " in future FLINK major version. Use 
`stream_execution_environment.configure`"
+  " method instead to set the restart strategy.", 
DeprecationWarning)
 self._j_stream_execution_environment.setRestartStrategy(
 restart_strategy_configuration._j_restart_strategy_configuration)
 



(flink) branch master updated (acb4217e1b5 -> d5b1afb7f67)

2023-11-29 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from acb4217e1b5 [hotfix][sequence-file][test] fix typo
 add 1a11197397f [hotfix] Migrate GlobalConfigurationTest to Junit5.
 add f39019e4317 [hotfix] Migrate ConfigurationTest to Junit5.
 add c3db0445c59 [hotfix] Migrate ConfigurationUtilsTest to Junit5.
 add 20339eac018 [hotfix] Migrate ReadableWritableConfigurationTest to 
Junit5 and AssertJ
 add d5b1afb7f67 [FLINK-33364][core] Introduce standard YAML for flink 
configuration.

No new revisions were added by this update.

Summary of changes:
 .../apache/flink/configuration/Configuration.java  |  60 +++-
 .../flink/configuration/ConfigurationUtils.java|  60 +++-
 .../configuration/DelegatingConfiguration.java |  17 +
 .../flink/configuration/GlobalConfiguration.java   |   6 +
 .../flink/configuration/YamlParserUtils.java   | 239 ++
 .../flink/configuration/ConfigurationTest.java | 354 -
 .../configuration/ConfigurationUtilsTest.java  |  75 +++--
 .../configuration/GlobalConfigurationTest.java | 117 +++
 .../ReadableWritableConfigurationTest.java | 228 +++--
 .../flink/configuration/YamlParserUtilsTest.java   | 148 +
 .../flink/tests/util/flink/FlinkDistribution.java  |   2 +-
 .../decorators/FlinkConfMountDecorator.java|   2 +-
 .../runtime/clusterframework/BootstrapTools.java   |   2 +-
 .../flink/table/catalog/FileCatalogStore.java  |   2 +-
 .../testframe/container/FlinkImageBuilder.java |   2 +-
 15 files changed, 968 insertions(+), 346 deletions(-)
 create mode 100644 
flink-core/src/main/java/org/apache/flink/configuration/YamlParserUtils.java
 create mode 100644 
flink-core/src/test/java/org/apache/flink/configuration/YamlParserUtilsTest.java



(flink) branch master updated (22028753985 -> dae2eb5b61f)

2023-11-27 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 22028753985 [FLINK-33643][runtime] Allow StreamExecutionEnvironment's 
executeAsync API to use default JobName (#23794)
 new 0e0099b4eb1 [FLINK-33581][core] Deprecate complex Java objects and 
their corresponding getter/setter methods related to restart strategies.
 new 139db3f4bc7 [FLINK-33581][core] Deprecate getter/setter methods 
related to checkpoint storage in the CheckpointConfig.
 new dae2eb5b61f [FLINK-33581][core] Deprecate getter/setter methods 
related to state backend in the StreamExecutionEnvironment.

The 3 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:
 .../5b9eed8a-5fb6-4373-98ac-3be2a71941b8   |  4 --
 .../apache/flink/api/common/ExecutionConfig.java   | 24 +++-
 .../common/restartstrategy/RestartStrategies.java  | 18 ++
 .../examples/statemachine/StateMachineExample.java | 24 +---
 .../api/environment/CheckpointConfig.java  | 68 ++
 .../environment/StreamExecutionEnvironment.java| 47 ++-
 6 files changed, 170 insertions(+), 15 deletions(-)



(flink) 01/03: [FLINK-33581][core] Deprecate complex Java objects and their corresponding getter/setter methods related to restart strategies.

2023-11-27 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0e0099b4eb1285929fec02326f661cba899eedcd
Author: JunRuiLee 
AuthorDate: Mon Nov 20 16:14:56 2023 +0800

[FLINK-33581][core] Deprecate complex Java objects and their corresponding 
getter/setter methods related to restart strategies.

This closes #23758.
---
 .../5b9eed8a-5fb6-4373-98ac-3be2a71941b8   |  4 
 .../apache/flink/api/common/ExecutionConfig.java   | 24 +-
 .../common/restartstrategy/RestartStrategies.java  | 18 
 .../environment/StreamExecutionEnvironment.java| 14 +
 4 files changed, 55 insertions(+), 5 deletions(-)

diff --git 
a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
 
b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
index cad03129799..7510d7d0190 100644
--- 
a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
+++ 
b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
@@ -1,5 +1,3 @@
-org.apache.flink.api.common.ExecutionConfig.getRestartStrategy(): Returned 
leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.common.ExecutionConfig.setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration):
 Argument leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.api.common.eventtime.TimestampAssignerSupplier.createTimestampAssigner(org.apache.flink.api.common.eventtime.TimestampAssignerSupplier$Context):
 Argument leaf type 
org.apache.flink.api.common.eventtime.TimestampAssignerSupplier$Context does 
not satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.createWatermarkGenerator(org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier$Context):
 Argument leaf type 
org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier$Context does 
not satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 org.apache.flink.api.common.operators.Operator.getMinResources(): Returned 
leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: 
reside outside of package 'org.apache.flink..' or reside in any package 
['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or 
annotated with @Deprecated
@@ -185,8 +183,6 @@ 
org.apache.flink.streaming.api.datastream.BroadcastStream.getTransformation(): R
 org.apache.flink.streaming.api.datastream.DataStream.getMinResources(): 
Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 org.apache.flink.streaming.api.datastream.DataStream.getPreferredResources(): 
Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.streaming.api.datastream.DataStream.transform(java.lang.String,
 org.apache.flink.api.common.typeinfo.TypeInformation, 
org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory): 
Argument leaf type 
org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
-org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getRestartStrategy():
 Returned leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink

(flink) 03/03: [FLINK-33581][core] Deprecate getter/setter methods related to state backend in the StreamExecutionEnvironment.

2023-11-27 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit dae2eb5b61f71b9453a73e4f0b3c69fd28f54ebf
Author: JunRuiLee 
AuthorDate: Mon Nov 20 17:05:00 2023 +0800

[FLINK-33581][core] Deprecate getter/setter methods related to state 
backend in the StreamExecutionEnvironment.
---
 .../examples/statemachine/StateMachineExample.java | 10 ---
 .../environment/StreamExecutionEnvironment.java| 33 --
 2 files changed, 37 insertions(+), 6 deletions(-)

diff --git 
a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
 
b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
index f905043b2ee..940b836d0bf 100644
--- 
a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
+++ 
b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
@@ -30,15 +30,14 @@ import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.configuration.CheckpointingOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateBackendOptions;
 import org.apache.flink.connector.datagen.source.DataGeneratorSource;
 import org.apache.flink.connector.datagen.source.GeneratorFunction;
 import org.apache.flink.connector.file.sink.FileSink;
 import org.apache.flink.connector.kafka.source.KafkaSource;
 import 
org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
 import 
org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
-import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import 
org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy;
@@ -89,13 +88,16 @@ public class StateMachineExample {
 final String stateBackend = params.get("backend", "memory");
 if ("hashmap".equals(stateBackend)) {
 final String checkpointDir = params.get("checkpoint-dir");
-env.setStateBackend(new HashMapStateBackend());
+configuration.set(StateBackendOptions.STATE_BACKEND, "hashmap");
 configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, 
"filesystem");
 configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
checkpointDir);
 } else if ("rocks".equals(stateBackend)) {
 final String checkpointDir = params.get("checkpoint-dir");
 boolean incrementalCheckpoints = 
params.getBoolean("incremental-checkpoints", false);
-env.setStateBackend(new 
EmbeddedRocksDBStateBackend(incrementalCheckpoints));
+configuration.set(
+StateBackendOptions.STATE_BACKEND,
+
"org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackendFactory");
+configuration.set(CheckpointingOptions.INCREMENTAL_CHECKPOINTS, 
incrementalCheckpoints);
 configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, 
"filesystem");
 configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
checkpointDir);
 }
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 52b5e30034f..9069b3a0d3c 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -202,8 +202,15 @@ public class StreamExecutionEnvironment implements 
AutoCloseable {
 
 private boolean isChainingOfOperatorsWithDifferentMaxParallelismEnabled = 
true;
 
-/** The state backend used for storing k/v state and state snapshots. */
-private StateBackend defaultStateBackend;
+/**
+ * The state backend used for storing k/v state and state snapshots.
+ *
+ * @deprecated The field is marked as deprecated because starting from 
Flink 1.19, the usage of
+ * all complex Java objects related to configuration, including their 
getter and setter
+ * methods, should be replaced by ConfigOption. I

(flink) 02/03: [FLINK-33581][core] Deprecate getter/setter methods related to checkpoint storage in the CheckpointConfig.

2023-11-27 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 139db3f4bc7faed4478393a91a063ad54d15a523
Author: JunRuiLee 
AuthorDate: Mon Nov 20 16:58:04 2023 +0800

[FLINK-33581][core] Deprecate getter/setter methods related to checkpoint 
storage in the CheckpointConfig.
---
 .../examples/statemachine/StateMachineExample.java | 14 +++--
 .../api/environment/CheckpointConfig.java  | 68 ++
 2 files changed, 78 insertions(+), 4 deletions(-)

diff --git 
a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
 
b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
index 82ec3674178..f905043b2ee 100644
--- 
a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
+++ 
b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java
@@ -27,6 +27,8 @@ import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import 
org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy;
 import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.connector.datagen.source.DataGeneratorSource;
 import org.apache.flink.connector.datagen.source.GeneratorFunction;
@@ -82,20 +84,24 @@ public class StateMachineExample {
 final ParameterTool params = ParameterTool.fromArgs(args);
 
 // create the environment to create streams and configure execution
-final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
-env.enableCheckpointing(2000L);
+Configuration configuration = new Configuration();
 
 final String stateBackend = params.get("backend", "memory");
 if ("hashmap".equals(stateBackend)) {
 final String checkpointDir = params.get("checkpoint-dir");
 env.setStateBackend(new HashMapStateBackend());
-env.getCheckpointConfig().setCheckpointStorage(checkpointDir);
+configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, 
"filesystem");
+configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
checkpointDir);
 } else if ("rocks".equals(stateBackend)) {
 final String checkpointDir = params.get("checkpoint-dir");
 boolean incrementalCheckpoints = 
params.getBoolean("incremental-checkpoints", false);
 env.setStateBackend(new 
EmbeddedRocksDBStateBackend(incrementalCheckpoints));
-env.getCheckpointConfig().setCheckpointStorage(checkpointDir);
+configuration.set(CheckpointingOptions.CHECKPOINT_STORAGE, 
"filesystem");
+configuration.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
checkpointDir);
 }
+final StreamExecutionEnvironment env =
+
StreamExecutionEnvironment.getExecutionEnvironment(configuration);
+env.enableCheckpointing(2000L);
 
 if (params.has("kafka-topic")) {
 // set up the Kafka reader
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
index b80ed10bb73..cf299080571 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
@@ -747,8 +747,23 @@ public class CheckpointConfig implements 
java.io.Serializable {
  * terabytes while providing a highly available foundation for stateful 
applications. This
  * checkpoint storage policy is recommended for most production 
deployments.
  *
+ * @deprecated The method is marked as deprecated because starting from 
Flink 1.19, the usage of
+ * all complex Java objects related to configuration, including their 
getter and setter
+ * methods, should be replaced by ConfigOption. In a future major 
version of Flink, this
+ * method will be removed entirely. It is recommended to switch to 
using the ConfigOptions
+ * provided for configuring checkpoint storage like the following code 
snippet:
+ * {@code
+ * Configuration config = new Configuration();
+ * config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "

svn commit: r65358 - /dev/flink/flink-shaded-16.2-rc1/ /release/flink/flink-shaded-16.2/

2023-11-17 Thread zhuzh
Author: zhuzh
Date: Fri Nov 17 08:18:44 2023
New Revision: 65358

Log:
Release Flink-shaded 16.2

Added:
release/flink/flink-shaded-16.2/
  - copied from r65357, dev/flink/flink-shaded-16.2-rc1/
Removed:
dev/flink/flink-shaded-16.2-rc1/



(flink) branch master updated: [FLINK-33080][runtime] Fix ‘state.checkpoint-storage’ not taking effect in job generation stage

2023-10-30 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 25697476095 [FLINK-33080][runtime] Fix ‘state.checkpoint-storage’ not 
taking effect in job generation stage
25697476095 is described below

commit 25697476095a5b9cf38dc3b61c684d0e912b1353
Author: JunRuiLee 
AuthorDate: Wed Sep 13 16:00:52 2023 +0800

[FLINK-33080][runtime] Fix ‘state.checkpoint-storage’ not taking effect in 
job generation stage

This closes #23408.
---
 .../client/program/StreamContextEnvironment.java   |  1 +
 .../program/StreamContextEnvironmentTest.java  | 30 ++
 .../environment/StreamExecutionEnvironment.java| 18 +
 .../StreamExecutionEnvironmentTest.java| 19 ++
 4 files changed, 68 insertions(+)

diff --git 
a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
 
b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
index 21ddca61694..07f47573666 100644
--- 
a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
+++ 
b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
@@ -303,6 +303,7 @@ public class StreamContextEnvironment extends 
StreamExecutionEnvironment {
 private void checkCheckpointConfig(Configuration clusterConfigMap, 
List errors) {
 CheckpointConfig expectedCheckpointConfig = new CheckpointConfig();
 expectedCheckpointConfig.configure(clusterConfigMap);
+configureCheckpointStorage(clusterConfigMap, expectedCheckpointConfig);
 checkConfigurationObject(
 expectedCheckpointConfig.toConfiguration(),
 checkpointCfg.toConfiguration(),
diff --git 
a/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java
 
b/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java
index 85f6279fc58..1af115a402a 100644
--- 
a/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java
+++ 
b/flink-clients/src/test/java/org/apache/flink/client/program/StreamContextEnvironmentTest.java
@@ -121,6 +121,36 @@ class StreamContextEnvironmentTest {
 CheckpointConfig.class.getSimpleName(), 
"setCheckpointStorage");
 }
 
+@ParameterizedTest
+@MethodSource("provideExecutors")
+void testDisallowCheckpointStorageByConfiguration(
+ThrowingConsumer executor) {
+final Configuration clusterConfig = new Configuration();
+
+Configuration jobConfig = new Configuration();
+String disallowedPath = "file:///flink/disallowed/modification";
+jobConfig.set(CheckpointingOptions.CHECKPOINT_STORAGE, "jobmanager");
+jobConfig.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
disallowedPath);
+final ClassLoader classLoader = 
Thread.currentThread().getContextClassLoader();
+final StreamContextEnvironment environment =
+new StreamContextEnvironment(
+new MockExecutorServiceLoader(),
+clusterConfig,
+jobConfig,
+classLoader,
+true,
+true,
+false,
+Collections.emptyList());
+
+environment.fromCollection(Collections.singleton(1)).sinkTo(new 
DiscardingSink<>());
+assertThatThrownBy(() -> executor.accept(environment))
+.isInstanceOf(MutatedConfigurationException.class)
+.hasMessageContainingAll(
+CheckpointingOptions.CHECKPOINT_STORAGE.key(),
+CheckpointingOptions.CHECKPOINTS_DIRECTORY.key());
+}
+
 @ParameterizedTest
 @MethodSource("provideExecutors")
 void testAllowCheckpointStorage(
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 891aabc52f6..343308edcf2 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -73,6 +73,8 @@ import 
org.apache.flink.core.execution.PipelineExecutorServiceLoader;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
 import org.apache.flink.runtime.scheduler.ClusterDatasetCorruptedException;
+impor

(flink) branch master updated: [FLINK-33084][runtime] Migrate globalJobParameter to configuration.

2023-10-30 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new b946ecc6683 [FLINK-33084][runtime] Migrate globalJobParameter to 
configuration.
b946ecc6683 is described below

commit b946ecc668342d48c2c0193ad4eff1897c75b68f
Author: JunRuiLee 
AuthorDate: Wed Sep 13 18:51:27 2023 +0800

[FLINK-33084][runtime] Migrate globalJobParameter to configuration.

This closes #23409.
---
 .../org/apache/flink/api/common/ExecutionConfig.java | 20 ++--
 1 file changed, 10 insertions(+), 10 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java 
b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index f3fae490cf7..22aee331513 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -154,8 +154,6 @@ public class ExecutionConfig implements Serializable, 
Archiveable parameters) {
+configuration.set(PipelineOptions.GLOBAL_JOB_PARAMETERS, parameters);
 }
 
 // 

@@ -985,7 +990,6 @@ public class ExecutionConfig implements Serializable, 
Archiveable

[flink] branch master updated (7299da4cf68 -> b93216f7855)

2023-08-18 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 7299da4cf68 [FLINK-32880][flink-runtime]Fulfill redundant taskmanagers 
periodically in FineGrainedSlotManager (#23230)
 add b93216f7855 [FLINK-32788][runtime] Support the handling of exception 
when slow task detector notify slow tasks

No new revisions were added by this update.

Summary of changes:
 .../ExecutionTimeBasedSlowTaskDetector.java| 26 ++-
 .../ExecutionTimeBasedSlowTaskDetectorTest.java| 52 +-
 2 files changed, 75 insertions(+), 3 deletions(-)



[flink] branch release-1.17 updated: [FLINK-32876][runtime] Prevent ExecutionTimeBasedSlowTaskDetector from identifying tasks in CREATED state as slow tasks.

2023-08-18 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.17 by this push:
 new 38b9c280128 [FLINK-32876][runtime] Prevent 
ExecutionTimeBasedSlowTaskDetector from identifying tasks in CREATED state as 
slow tasks.
38b9c280128 is described below

commit 38b9c280128981b3e809df1f963bdaf8c0491804
Author: JunRuiLee 
AuthorDate: Fri Aug 18 10:42:25 2023 +0800

[FLINK-32876][runtime] Prevent ExecutionTimeBasedSlowTaskDetector from 
identifying tasks in CREATED state as slow tasks.

This closes #23222.
---
 .../ExecutionTimeBasedSlowTaskDetector.java | 12 +++-
 .../ExecutionTimeBasedSlowTaskDetectorTest.java | 21 +
 2 files changed, 32 insertions(+), 1 deletion(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
index 34cb5b47b36..f6d08548a0a 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
@@ -214,7 +214,17 @@ public class ExecutionTimeBasedSlowTaskDetector implements 
SlowTaskDetector {
 ExecutionTimeWithInputBytes baseline,
 long currentTimeMillis) {
 return executions.stream()
-.filter(e -> !e.getState().isTerminal() && e.getState() != 
ExecutionState.CANCELING)
+.filter(
+// We will filter out tasks that are in the CREATED 
state, as we do not
+// allow speculative execution for them because they 
have not been
+// scheduled.
+// However, for tasks that are already in the 
SCHEDULED state, we allow
+// speculative execution to provide the capability of 
parallel execution
+// running.
+e ->
+!e.getState().isTerminal()
+&& e.getState() != 
ExecutionState.CANCELING
+&& e.getState() != 
ExecutionState.CREATED)
 .filter(
 e -> {
 ExecutionTimeWithInputBytes timeWithBytes =
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
index b11f86c80d4..1714d79edbf 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
@@ -76,6 +76,27 @@ class ExecutionTimeBasedSlowTaskDetectorTest {
 assertThat(slowTasks).hasSize(parallelism);
 }
 
+@Test
+void testAllTasksInCreatedAndNoSlowTasks() throws Exception {
+final int parallelism = 3;
+final JobVertex jobVertex = createNoOpVertex(parallelism);
+final JobGraph jobGraph = 
JobGraphTestUtils.streamingJobGraph(jobVertex);
+
+// all tasks are in the CREATED state, which is not classified as slow 
tasks.
+final ExecutionGraph executionGraph =
+SchedulerTestingUtils.createScheduler(
+jobGraph,
+
ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+EXECUTOR_RESOURCE.getExecutor())
+.getExecutionGraph();
+
+final ExecutionTimeBasedSlowTaskDetector slowTaskDetector = 
createSlowTaskDetector(0, 1, 0);
+final Map> slowTasks 
=
+slowTaskDetector.findSlowTasks(executionGraph);
+
+assertThat(slowTasks.size()).isZero();
+}
+
 @Test
 void testFinishedTaskNotExceedRatio() throws Exception {
 final int parallelism = 3;



[flink] branch master updated: [FLINK-32876][runtime] Prevent ExecutionTimeBasedSlowTaskDetector from identifying tasks in CREATED state as slow tasks.

2023-08-18 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 4bce9c09ca4 [FLINK-32876][runtime] Prevent 
ExecutionTimeBasedSlowTaskDetector from identifying tasks in CREATED state as 
slow tasks.
4bce9c09ca4 is described below

commit 4bce9c09ca4fdf3ad8bf95ba5cf4ca361acea156
Author: JunRuiLee 
AuthorDate: Fri Aug 18 10:42:25 2023 +0800

[FLINK-32876][runtime] Prevent ExecutionTimeBasedSlowTaskDetector from 
identifying tasks in CREATED state as slow tasks.

This closes #23222.
---
 .../ExecutionTimeBasedSlowTaskDetector.java | 12 +++-
 .../ExecutionTimeBasedSlowTaskDetectorTest.java | 21 +
 2 files changed, 32 insertions(+), 1 deletion(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
index 34cb5b47b36..f6d08548a0a 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetector.java
@@ -214,7 +214,17 @@ public class ExecutionTimeBasedSlowTaskDetector implements 
SlowTaskDetector {
 ExecutionTimeWithInputBytes baseline,
 long currentTimeMillis) {
 return executions.stream()
-.filter(e -> !e.getState().isTerminal() && e.getState() != 
ExecutionState.CANCELING)
+.filter(
+// We will filter out tasks that are in the CREATED 
state, as we do not
+// allow speculative execution for them because they 
have not been
+// scheduled.
+// However, for tasks that are already in the 
SCHEDULED state, we allow
+// speculative execution to provide the capability of 
parallel execution
+// running.
+e ->
+!e.getState().isTerminal()
+&& e.getState() != 
ExecutionState.CANCELING
+&& e.getState() != 
ExecutionState.CREATED)
 .filter(
 e -> {
 ExecutionTimeWithInputBytes timeWithBytes =
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
index b11f86c80d4..1714d79edbf 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/slowtaskdetector/ExecutionTimeBasedSlowTaskDetectorTest.java
@@ -76,6 +76,27 @@ class ExecutionTimeBasedSlowTaskDetectorTest {
 assertThat(slowTasks).hasSize(parallelism);
 }
 
+@Test
+void testAllTasksInCreatedAndNoSlowTasks() throws Exception {
+final int parallelism = 3;
+final JobVertex jobVertex = createNoOpVertex(parallelism);
+final JobGraph jobGraph = 
JobGraphTestUtils.streamingJobGraph(jobVertex);
+
+// all tasks are in the CREATED state, which is not classified as slow 
tasks.
+final ExecutionGraph executionGraph =
+SchedulerTestingUtils.createScheduler(
+jobGraph,
+
ComponentMainThreadExecutorServiceAdapter.forMainThread(),
+EXECUTOR_RESOURCE.getExecutor())
+.getExecutionGraph();
+
+final ExecutionTimeBasedSlowTaskDetector slowTaskDetector = 
createSlowTaskDetector(0, 1, 0);
+final Map> slowTasks 
=
+slowTaskDetector.findSlowTasks(executionGraph);
+
+assertThat(slowTasks.size()).isZero();
+}
+
 @Test
 void testFinishedTaskNotExceedRatio() throws Exception {
 final int parallelism = 3;



[flink] branch master updated: [FLINK-32768][runtime] Change ExecutionSlotAllocator#allocateSlotsFor return type to allow the order of execution and assignments to be inconsistent

2023-08-09 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 6806f30fe23 [FLINK-32768][runtime] Change 
ExecutionSlotAllocator#allocateSlotsFor return type to allow the order of 
execution and assignments to be inconsistent
6806f30fe23 is described below

commit 6806f30fe234790f25b5ca83e6d2910286a76aeb
Author: JunRuiLee 
AuthorDate: Wed Aug 9 13:49:23 2023 +0800

[FLINK-32768][runtime] Change ExecutionSlotAllocator#allocateSlotsFor 
return type to allow the order of execution and assignments to be inconsistent

This closes #23165.
---
 .../scheduler/DefaultExecutionDeployer.java| 17 +++---
 .../runtime/scheduler/ExecutionSlotAllocator.java  |  6 +-
 .../scheduler/SimpleExecutionSlotAllocator.java| 18 +++---
 .../SlotSharingExecutionSlotAllocator.java | 15 +++--
 .../SimpleExecutionSlotAllocatorTest.java  |  2 +-
 .../SlotSharingExecutionSlotAllocatorTest.java | 67 +++---
 .../scheduler/TestExecutionSlotAllocator.java  | 19 +++---
 7 files changed, 87 insertions(+), 57 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployer.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployer.java
index 99b3bcd04e5..9589981dcf9 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployer.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployer.java
@@ -94,12 +94,12 @@ public class DefaultExecutionDeployer implements 
ExecutionDeployer {
 
 transitionToScheduled(executionsToDeploy);
 
-final List executionSlotAssignments =
+final Map 
executionSlotAssignmentMap =
 allocateSlotsFor(executionsToDeploy);
 
 final List deploymentHandles =
 createDeploymentHandles(
-executionsToDeploy, requiredVersionByVertex, 
executionSlotAssignments);
+executionsToDeploy, requiredVersionByVertex, 
executionSlotAssignmentMap);
 
 waitForAllSlotsAndDeploy(deploymentHandles);
 }
@@ -118,7 +118,7 @@ public class DefaultExecutionDeployer implements 
ExecutionDeployer {
 executionsToDeploy.forEach(e -> 
e.transitionState(ExecutionState.SCHEDULED));
 }
 
-private List allocateSlotsFor(
+private Map allocateSlotsFor(
 final List executionsToDeploy) {
 final List executionAttemptIds =
 executionsToDeploy.stream()
@@ -130,14 +130,13 @@ public class DefaultExecutionDeployer implements 
ExecutionDeployer {
 private List createDeploymentHandles(
 final List executionsToDeploy,
 final Map 
requiredVersionByVertex,
-final List executionSlotAssignments) {
-
+final Map 
executionSlotAssignmentMap) {
+checkState(executionsToDeploy.size() == 
executionSlotAssignmentMap.size());
 final List deploymentHandles =
 new ArrayList<>(executionsToDeploy.size());
-for (int i = 0; i < executionsToDeploy.size(); i++) {
-final Execution execution = executionsToDeploy.get(i);
-final ExecutionSlotAssignment assignment = 
executionSlotAssignments.get(i);
-
checkState(execution.getAttemptId().equals(assignment.getExecutionAttemptId()));
+for (final Execution execution : executionsToDeploy) {
+final ExecutionSlotAssignment assignment =
+
checkNotNull(executionSlotAssignmentMap.get(execution.getAttemptId()));
 
 final ExecutionVertexID executionVertexId = 
execution.getVertex().getID();
 final ExecutionDeploymentHandle deploymentHandle =
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocator.java
index 526ebae99a4..569ec5fb5c8 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocator.java
@@ -22,6 +22,7 @@ import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 
 import java.util.List;
+import java.util.Map;
 
 /** Component responsible for assigning slots to a collection of {@link 
Execution}. */
 public interface ExecutionSlotAllocator {
@@ -30,9 +31,10 @@ public interface ExecutionSlotAllocator {
  * Allocate slots for the given executions.
  *
  * @param executionAttemptIds executions to allocate slots for
- * @return List of slot assignments to the executions
+ * @return Map of slot assignments to th

[flink] branch master updated: [FLINK-32654][configuration] Deprecate ExecutionConfig#canEqual(obj)

2023-07-24 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 5cda33d87be [FLINK-32654][configuration] Deprecate 
ExecutionConfig#canEqual(obj)
5cda33d87be is described below

commit 5cda33d87be1a39e63f24d3c564c5a23ecb699d0
Author: Zhu Zhu 
AuthorDate: Mon Jul 24 16:00:13 2023 +0800

[FLINK-32654][configuration] Deprecate ExecutionConfig#canEqual(obj)

This closes #23061.
---
 .../main/java/org/apache/flink/api/common/ExecutionConfig.java   | 9 +++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java 
b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index e09cd74a8a8..19f82fdfe74 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -941,8 +941,7 @@ public class ExecutionConfig implements Serializable, 
Archiveable

[flink] branch master updated: [FLINK-32586][coordination] Enable input locality in SimpleExecutionSlotAllocator.

2023-07-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new e732edb41a4 [FLINK-32586][coordination] Enable input locality in 
SimpleExecutionSlotAllocator.
e732edb41a4 is described below

commit e732edb41a423f19d5eefc397ddbfacadaf0179e
Author: sunxia 
AuthorDate: Tue Jul 18 17:11:01 2023 +0800

[FLINK-32586][coordination] Enable input locality in 
SimpleExecutionSlotAllocator.

This closes #23009.
---
 .../executiongraph/SpeculativeExecutionVertex.java |  6 
 .../scheduler/SimpleExecutionSlotAllocator.java| 18 +--
 .../SimpleExecutionSlotAllocatorTest.java  | 36 ++
 3 files changed, 52 insertions(+), 8 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java
index dbd5f84a461..8179d541210 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SpeculativeExecutionVertex.java
@@ -308,10 +308,4 @@ public class SpeculativeExecutionVertex extends 
ExecutionVertex {
 throw new UnsupportedOperationException(
 "Method is not supported in SpeculativeExecutionVertex.");
 }
-
-@Override
-public CompletableFuture 
getCurrentTaskManagerLocationFuture() {
-throw new UnsupportedOperationException(
-"Method is not supported in SpeculativeExecutionVertex.");
-}
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SimpleExecutionSlotAllocator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SimpleExecutionSlotAllocator.java
index e5d6d8ad1e7..07251ef42cf 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SimpleExecutionSlotAllocator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SimpleExecutionSlotAllocator.java
@@ -28,11 +28,14 @@ import 
org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot;
 import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotProvider;
 import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlotRequest;
 import org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.util.DualKeyLinkedMap;
 import org.apache.flink.util.FlinkException;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -41,7 +44,7 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * A simple implementation of {@link ExecutionSlotAllocator}. No support for 
slot sharing,
- * co-location, state/input locality, nor local recovery.
+ * co-location, nor local recovery.
  */
 public class SimpleExecutionSlotAllocator implements ExecutionSlotAllocator {
 private final PhysicalSlotProvider slotProvider;
@@ -50,6 +53,8 @@ public class SimpleExecutionSlotAllocator implements 
ExecutionSlotAllocator {
 
 private final Function 
resourceProfileRetriever;
 
+private final SyncPreferredLocationsRetriever preferredLocationsRetriever;
+
 private final DualKeyLinkedMap<
 ExecutionAttemptID, SlotRequestId, 
CompletableFuture>
 requestedPhysicalSlots;
@@ -57,10 +62,12 @@ public class SimpleExecutionSlotAllocator implements 
ExecutionSlotAllocator {
 SimpleExecutionSlotAllocator(
 PhysicalSlotProvider slotProvider,
 Function 
resourceProfileRetriever,
+SyncPreferredLocationsRetriever preferredLocationsRetriever,
 boolean slotWillBeOccupiedIndefinitely) {
 this.slotProvider = checkNotNull(slotProvider);
 this.slotWillBeOccupiedIndefinitely = slotWillBeOccupiedIndefinitely;
 this.resourceProfileRetriever = checkNotNull(resourceProfileRetriever);
+this.preferredLocationsRetriever = 
checkNotNull(preferredLocationsRetriever);
 this.requestedPhysicalSlots = new DualKeyLinkedMap<>();
 }
 
@@ -78,11 +85,14 @@ public class SimpleExecutionSlotAllocator implements 
ExecutionSlotAllocator {
 }
 final SlotRequestId slotRequestId = new SlotRequestId();
 final ResourceProfile resourceProfile = 
resourceProfileRetriever.apply(executionAttemptId);
+Collection preferredLocations =
+preferredLocationsRetriever.getPreferredLocations(
+executionAttemptId.getExecutionVertexId(), 
Collect

[flink] branch master updated (4586bcbe662 -> b462d0ec4d1)

2023-06-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 4586bcbe662 [FLINK-27240][table] Support ADD PARTITION statement for 
partitioned table (#22711)
 new 7e42493b021 [hotfix] Enrich hybridPartitionDataConsumeConstraint when 
create AdaptiveBatchScheduler in scheduler benchmark utils.
 new b462d0ec4d1 [FLINK-32288][runtime] Improve the scheduling performance 
of AdaptiveBatchScheduler

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:
 .../AllFinishedInputConsumableDecider.java |  5 +++--
 .../strategy/DefaultInputConsumableDecider.java| 13 +
 .../scheduler/strategy/InputConsumableDecider.java | 10 +-
 .../PartialFinishedInputConsumableDecider.java |  5 +++--
 .../strategy/VertexwiseSchedulingStrategy.java |  5 +
 .../benchmark/SchedulerBenchmarkUtils.java |  2 ++
 .../DefaultInputConsumableDeciderTest.java | 22 ++
 .../strategy/TestingInputConsumableDecider.java|  6 ++
 8 files changed, 63 insertions(+), 5 deletions(-)



[flink] 02/02: [FLINK-32288][runtime] Improve the scheduling performance of AdaptiveBatchScheduler

2023-06-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit b462d0ec4d1d421a369e45f8dca33284b5be6bc2
Author: sunxia 
AuthorDate: Fri Jun 16 10:14:21 2023 +0800

[FLINK-32288][runtime] Improve the scheduling performance of 
AdaptiveBatchScheduler

This close #22798.
---
 .../AllFinishedInputConsumableDecider.java |  5 +++--
 .../strategy/DefaultInputConsumableDecider.java| 13 +
 .../scheduler/strategy/InputConsumableDecider.java | 10 +-
 .../PartialFinishedInputConsumableDecider.java |  5 +++--
 .../strategy/VertexwiseSchedulingStrategy.java |  5 +
 .../DefaultInputConsumableDeciderTest.java | 22 ++
 .../strategy/TestingInputConsumableDecider.java|  6 ++
 7 files changed, 61 insertions(+), 5 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/AllFinishedInputConsumableDecider.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/AllFinishedInputConsumableDecider.java
index 6c23757a1a7..f8cbb260488 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/AllFinishedInputConsumableDecider.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/AllFinishedInputConsumableDecider.java
@@ -37,14 +37,15 @@ public class AllFinishedInputConsumableDecider implements 
InputConsumableDecider
 executionVertex.getConsumedPartitionGroups()) {
 
 if (!consumableStatusCache.computeIfAbsent(
-consumedPartitionGroup, 
this::isConsumedPartitionGroupConsumable)) {
+consumedPartitionGroup, 
this::isConsumableBasedOnFinishedProducers)) {
 return false;
 }
 }
 return true;
 }
 
-private boolean isConsumedPartitionGroupConsumable(
+@Override
+public boolean isConsumableBasedOnFinishedProducers(
 final ConsumedPartitionGroup consumedPartitionGroup) {
 return consumedPartitionGroup.getNumberOfUnfinishedPartitions() == 0;
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/DefaultInputConsumableDecider.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/DefaultInputConsumableDecider.java
index 93db09c94b6..ccd354b0d0d 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/DefaultInputConsumableDecider.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/DefaultInputConsumableDecider.java
@@ -65,6 +65,19 @@ public class DefaultInputConsumableDecider implements 
InputConsumableDecider {
 return true;
 }
 
+@Override
+public boolean isConsumableBasedOnFinishedProducers(
+final ConsumedPartitionGroup consumedPartitionGroup) {
+if 
(consumedPartitionGroup.getResultPartitionType().canBePipelinedConsumed()) {
+// For canBePipelined consumed partition group, whether it is 
consumable does not depend
+// on task finish. To optimize performance and avoid unnecessary 
computation, we simply
+// return false.
+return false;
+} else {
+return consumedPartitionGroup.areAllPartitionsFinished();
+}
+}
+
 private boolean isConsumedPartitionGroupConsumable(
 final ConsumedPartitionGroup consumedPartitionGroup,
 final Set verticesToSchedule) {
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputConsumableDecider.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputConsumableDecider.java
index e34cb06bc4e..1d19dd2cf62 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputConsumableDecider.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputConsumableDecider.java
@@ -24,7 +24,7 @@ import java.util.function.Function;
 
 /**
  * {@link InputConsumableDecider} is responsible for determining whether the 
input of an
- * executionVertex is consumable.
+ * executionVertex or a consumed partition group is consumable.
  */
 public interface InputConsumableDecider {
 /**
@@ -41,6 +41,14 @@ public interface InputConsumableDecider {
 Set verticesToSchedule,
 Map consumableStatusCache);
 
+/**
+ * Determining whether the consumed partition group is consumable based on 
finished producers.
+ *
+ * @param consumedPartitionGroup to be determined whether it is consumable.
+ */
+boolean isConsumableBasedOnFinishedProducers(
+final ConsumedPartitionGroup consumedPartitionGroup);
+
 /** Factory for {@link InputConsumableDecider}. */
 interface Factory {
 InputConsumableDecider createInstance(
diff

[flink] 01/02: [hotfix] Enrich hybridPartitionDataConsumeConstraint when create AdaptiveBatchScheduler in scheduler benchmark utils.

2023-06-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 7e42493b021e3e012dd429e89588a54d8b151dfd
Author: sunxia 
AuthorDate: Thu Jun 15 18:12:45 2023 +0800

[hotfix] Enrich hybridPartitionDataConsumeConstraint when create 
AdaptiveBatchScheduler in scheduler benchmark utils.
---
 .../flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java  | 2 ++
 1 file changed, 2 insertions(+)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java
index 2368e781ec4..a60b3fd584b 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java
@@ -135,6 +135,8 @@ public class SchedulerBenchmarkUtils {
 return schedulerBuilder
 .setVertexParallelismAndInputInfosDecider(
 
createCustomParallelismDecider(jobConfiguration.getParallelism()))
+.setHybridPartitionDataConsumeConstraint(
+
jobConfiguration.getHybridPartitionDataConsumeConstraint())
 .setInputConsumableDeciderFactory(
 loadInputConsumableDeciderFactory(
 
jobConfiguration.getHybridPartitionDataConsumeConstraint()))



[flink-benchmarks] branch master updated: [FLINK-30480][runtime] Add benchmarks for adaptive batch scheduler.

2023-06-12 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-benchmarks.git


The following commit(s) were added to refs/heads/master by this push:
 new 2c03dca  [FLINK-30480][runtime] Add benchmarks for adaptive batch 
scheduler.
2c03dca is described below

commit 2c03dca3c8e8b4aabac1ff803cb363cbd36c74d1
Author: sunxia 
AuthorDate: Fri Jun 9 14:55:53 2023 +0800

[FLINK-30480][runtime] Add benchmarks for adaptive batch scheduler.
---
 .../SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/src/main/java/org/apache/flink/scheduler/benchmark/scheduling/SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java
 
b/src/main/java/org/apache/flink/scheduler/benchmark/scheduling/SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java
index 7b05575..182776b 100644
--- 
a/src/main/java/org/apache/flink/scheduler/benchmark/scheduling/SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java
+++ 
b/src/main/java/org/apache/flink/scheduler/benchmark/scheduling/SchedulingDownstreamTasksInBatchJobBenchmarkExecutor.java
@@ -35,7 +35,7 @@ import org.openjdk.jmh.runner.RunnerException;
 public class SchedulingDownstreamTasksInBatchJobBenchmarkExecutor
 extends SchedulerBenchmarkExecutorBase {
 
-@Param({"BATCH"})
+@Param({"BATCH", "BATCH_HYBRID_DEFAULT", "BATCH_HYBRID_PARTIAL_FINISHED", 
"BATCH_HYBRID_ALL_FINISHED"})
 private JobConfiguration jobConfiguration;
 
 private SchedulingDownstreamTasksInBatchJobBenchmark benchmark;



[flink] branch master updated (31e17bd13ef -> 27285d67127)

2023-06-09 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 31e17bd13ef [FLINK-31790][connectors/filesystem] Respect the 
PartitionCommitPolicy for batch Filesystem mode (#22387)
 add dfa9e09d84e [hotfix] Migrate scheduler benchmark related tests under 
flink-runtime module to junit5.
 add 27285d67127 [FLINK-30480][runtime] Add benchmarks for adaptive batch 
scheduler.

No new revisions were added by this update.

Summary of changes:
 .../adaptivebatch/AdaptiveBatchScheduler.java  |  3 +-
 .../scheduler/benchmark/JobConfiguration.java  | 82 +-
 .../benchmark/SchedulerBenchmarkUtils.java | 60 ++--
 ...yingDownstreamTasksInBatchJobBenchmarkTest.java |  7 +-
 .../DeployingTasksInStreamingJobBenchmarkTest.java |  7 +-
 .../e2e/CreateSchedulerBenchmarkTest.java  |  9 ++-
 ...leGlobalFailureAndRestartAllTasksBenchmark.java | 31 
 .../e2e/SchedulerEndToEndBenchmarkBase.java| 21 --
 .../e2e/SchedulingAndDeployingBenchmarkTest.java   |  9 ++-
 .../RegionToRestartInBatchJobBenchmarkTest.java|  7 +-
 ...RegionToRestartInStreamingJobBenchmarkTest.java |  7 +-
 .../PartitionReleaseInBatchJobBenchmarkTest.java   |  7 +-
 .../InitSchedulingStrategyBenchmarkTest.java   |  9 ++-
 .../scheduling/SchedulingBenchmarkBase.java|  9 +--
 ...hedulingDownstreamTasksInBatchJobBenchmark.java | 15 ++--
 ...lingDownstreamTasksInBatchJobBenchmarkTest.java | 25 ---
 .../topology/BuildExecutionGraphBenchmarkTest.java |  9 ++-
 17 files changed, 235 insertions(+), 82 deletions(-)



[flink] branch master updated: [FLINK-31868] Fix DefaultInputSplitAssigner javadoc for class

2023-04-21 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new f240a5110a0 [FLINK-31868] Fix DefaultInputSplitAssigner javadoc for 
class
f240a5110a0 is described below

commit f240a5110a0d28473b534cf377d287d5d072a93e
Author: Peter Vary 
AuthorDate: Thu Apr 20 13:44:10 2023 +0200

[FLINK-31868] Fix DefaultInputSplitAssigner javadoc for class

This closes #22437.
---
 .../java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
index bdabc69ff8b..948e54ae0bf 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
@@ -32,8 +32,7 @@ import java.util.List;
 
 /**
  * This is the default implementation of the {@link InputSplitAssigner} 
interface. The default input
- * split assigner simply returns all input splits of an input vertex in the 
order they were
- * originally computed.
+ * split assigner simply returns all input splits of an input vertex in an 
undefined order.
  */
 @Internal
 public class DefaultInputSplitAssigner implements InputSplitAssigner {



[flink] branch master updated (e107783d91f -> bc859f6ffa9)

2023-04-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from e107783d91f [FLINK-31859][build] Update cyclonedx-maven-plugin from 
2.7.3 to 2.7.7
 new c20910ad991 [hotfix][test] Migrate 
FailoverStrategyCheckpointCoordinatorTest to Junit5 and AssertJ.
 new bc859f6ffa9 [FLINK-31853][runtime] Remove use of 
ExecutionAttemptMappingProvider

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:
 .../runtime/checkpoint/CheckpointCoordinator.java  | 13 +---
 .../runtime/checkpoint/CheckpointStatsTracker.java |  8 +--
 .../ExecutionAttemptMappingProvider.java   | 82 --
 .../executiongraph/DefaultExecutionGraph.java  |  2 -
 .../CheckpointCoordinatorMasterHooksTest.java  |  1 -
 .../CheckpointCoordinatorTestingUtils.java |  1 -
 .../FailoverStrategyCheckpointCoordinatorTest.java | 43 ++--
 7 files changed, 26 insertions(+), 124 deletions(-)
 delete mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ExecutionAttemptMappingProvider.java



[flink] 02/02: [FLINK-31853][runtime] Remove use of ExecutionAttemptMappingProvider

2023-04-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit bc859f6ffa9724459f7db1a7a4d3826a9ae67151
Author: JunRuiLee 
AuthorDate: Wed Apr 19 20:35:16 2023 +0800

[FLINK-31853][runtime] Remove use of ExecutionAttemptMappingProvider

This closes #22429.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 13 +---
 .../runtime/checkpoint/CheckpointStatsTracker.java |  8 +--
 .../ExecutionAttemptMappingProvider.java   | 82 --
 .../executiongraph/DefaultExecutionGraph.java  |  2 -
 .../CheckpointCoordinatorMasterHooksTest.java  |  1 -
 .../CheckpointCoordinatorTestingUtils.java |  1 -
 .../FailoverStrategyCheckpointCoordinatorTest.java |  1 -
 7 files changed, 6 insertions(+), 102 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index c7f84e73bc1..10195483ba0 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -222,8 +222,6 @@ public class CheckpointCoordinator {
 
 private final CheckpointPlanCalculator checkpointPlanCalculator;
 
-private final ExecutionAttemptMappingProvider attemptMappingProvider;
-
 private boolean baseLocationsForCheckpointInitialized = false;
 
 private boolean forceFullSnapshot;
@@ -242,7 +240,6 @@ public class CheckpointCoordinator {
 ScheduledExecutor timer,
 CheckpointFailureManager failureManager,
 CheckpointPlanCalculator checkpointPlanCalculator,
-ExecutionAttemptMappingProvider attemptMappingProvider,
 CheckpointStatsTracker statsTracker) {
 
 this(
@@ -257,7 +254,6 @@ public class CheckpointCoordinator {
 timer,
 failureManager,
 checkpointPlanCalculator,
-attemptMappingProvider,
 SystemClock.getInstance(),
 statsTracker,
 VertexFinishedStateChecker::new);
@@ -276,7 +272,6 @@ public class CheckpointCoordinator {
 ScheduledExecutor timer,
 CheckpointFailureManager failureManager,
 CheckpointPlanCalculator checkpointPlanCalculator,
-ExecutionAttemptMappingProvider attemptMappingProvider,
 Clock clock,
 CheckpointStatsTracker statsTracker,
 BiFunction<
@@ -314,7 +309,6 @@ public class CheckpointCoordinator {
 this.checkpointsCleaner = checkNotNull(checkpointsCleaner);
 this.failureManager = checkNotNull(failureManager);
 this.checkpointPlanCalculator = checkNotNull(checkpointPlanCalculator);
-this.attemptMappingProvider = checkNotNull(attemptMappingProvider);
 this.clock = checkNotNull(clock);
 this.isExactlyOnceMode = chkConfig.isExactlyOnce();
 this.unalignedCheckpointsEnabled = 
chkConfig.isUnalignedCheckpointsEnabled();
@@ -2070,11 +2064,8 @@ public class CheckpointCoordinator {
 }
 }
 
-public void reportStats(long id, ExecutionAttemptID attemptId, 
CheckpointMetrics metrics)
-throws CheckpointException {
-attemptMappingProvider
-.getVertex(attemptId)
-.ifPresent(ev -> statsTracker.reportIncompleteStats(id, ev, 
metrics));
+public void reportStats(long id, ExecutionAttemptID attemptId, 
CheckpointMetrics metrics) {
+statsTracker.reportIncompleteStats(id, attemptId, metrics);
 }
 
 // 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
index 52f00fa2d0c..0542ae4e01b 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java
@@ -22,7 +22,7 @@ import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.metrics.Gauge;
 import org.apache.flink.metrics.Metric;
 import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 import javax.annotation.Nullable;
@@ -258,16 +258,16 @@ public class CheckpointStatsTracker {
 }
 
 public void reportIncompleteStats(
-long checkpointId, ExecutionVertex vertex, CheckpointMetrics 
metrics) {
+long checkpointId, ExecutionAttemptID attemptId, CheckpointM

[flink] 01/02: [hotfix][test] Migrate FailoverStrategyCheckpointCoordinatorTest to Junit5 and AssertJ.

2023-04-19 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c20910ad991171501c6cac5934b173672f586941
Author: JunRuiLee 
AuthorDate: Wed Apr 19 21:05:52 2023 +0800

[hotfix][test] Migrate FailoverStrategyCheckpointCoordinatorTest to Junit5 
and AssertJ.
---
 .../FailoverStrategyCheckpointCoordinatorTest.java | 42 +++---
 1 file changed, 20 insertions(+), 22 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java
index 45c980c193f..3235250c86b 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java
@@ -25,34 +25,32 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import 
org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.testutils.TestingUtils;
-import org.apache.flink.testutils.executor.TestExecutorResource;
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.testutils.executor.TestExecutorExtension;
 import org.apache.flink.util.concurrent.Executors;
 import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
 
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
 
 import java.util.Collections;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadLocalRandom;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.mock;
 
 /** Tests for actions of {@link CheckpointCoordinator} on task failures. */
-public class FailoverStrategyCheckpointCoordinatorTest extends TestLogger {
+class FailoverStrategyCheckpointCoordinatorTest {
 
-@ClassRule
-public static final TestExecutorResource 
EXECUTOR_RESOURCE =
-TestingUtils.defaultExecutorResource();
+@RegisterExtension
+static final TestExecutorExtension 
EXECUTOR_RESOURCE =
+TestingUtils.defaultExecutorExtension();
 
 private ManuallyTriggeredScheduledExecutor manualThreadExecutor;
 
-@Before
-public void setUp() {
+@BeforeEach
+void setUp() {
 manualThreadExecutor = new ManuallyTriggeredScheduledExecutor();
 }
 
@@ -61,7 +59,7 @@ public class FailoverStrategyCheckpointCoordinatorTest 
extends TestLogger {
  * on job failover could handle the {@code currentPeriodicTrigger} null 
case well.
  */
 @Test
-public void testAbortPendingCheckpointsWithTriggerValidation() throws 
Exception {
+void testAbortPendingCheckpointsWithTriggerValidation() throws Exception {
 final int maxConcurrentCheckpoints = 
ThreadLocalRandom.current().nextInt(10) + 1;
 ExecutionGraph graph =
 new 
CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder()
@@ -108,18 +106,18 @@ public class FailoverStrategyCheckpointCoordinatorTest 
extends TestLogger {
 
.transitionState(ExecutionState.RUNNING));
 
 checkpointCoordinator.startCheckpointScheduler();
-assertTrue(checkpointCoordinator.isCurrentPeriodicTriggerAvailable());
+
assertThat(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()).isTrue();
 // only trigger the periodic scheduling
 // we can't trigger all scheduled task, because there is also a 
cancellation scheduled
 manualThreadExecutor.triggerPeriodicScheduledTasks();
 manualThreadExecutor.triggerAll();
-assertEquals(1, checkpointCoordinator.getNumberOfPendingCheckpoints());
+
assertThat(checkpointCoordinator.getNumberOfPendingCheckpoints()).isOne();
 
 for (int i = 1; i < maxConcurrentCheckpoints; i++) {
 checkpointCoordinator.triggerCheckpoint(false);
 manualThreadExecutor.triggerAll();
-assertEquals(i + 1, 
checkpointCoordinator.getNumberOfPendingCheckpoints());
-
assertTrue(checkpointCoordinator.isCurrentPeriodicTriggerAvailable());
+
assertThat(checkpointCoordinator.getNumberOfPendingCheckpoints()).isEqualTo(i + 
1);
+
assertThat(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()).isTrue();
 }
 
 // as we only support limited concurrent checkpoints, after checkpoint 
triggered more than
@@ -127,13 +125,13 @@ public cl

[flink] branch release-1.17 updated: [FLINK-31144][coordination] Ignore the input locations of a ConsumePartitionGroup if it has too many consumers

2023-03-10 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.17 by this push:
 new 5b90c075bf9 [FLINK-31144][coordination] Ignore the input locations of 
a ConsumePartitionGroup if it has too many consumers
5b90c075bf9 is described below

commit 5b90c075bf9b987de2fa4a02e0cf63602152eba1
Author: JunRuiLee 
AuthorDate: Wed Mar 8 10:57:30 2023 +0800

[FLINK-31144][coordination] Ignore the input locations of a 
ConsumePartitionGroup if it has too many consumers

This closes #22098.
---
 .../DefaultPreferredLocationsRetriever.java|  8 +++---
 .../DefaultPreferredLocationsRetrieverTest.java| 30 +++---
 2 files changed, 12 insertions(+), 26 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetriever.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetriever.java
index f3b71366682..d5f0f8dae65 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetriever.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetriever.java
@@ -92,11 +92,9 @@ public class DefaultPreferredLocationsRetriever implements 
PreferredLocationsRet
 
inputsLocationsRetriever.getConsumedPartitionGroups(executionVertexId);
 for (ConsumedPartitionGroup consumedPartitionGroup : 
consumedPartitionGroups) {
 // Ignore the location of a consumed partition group if it has too 
many distinct
-// consumers compared to the consumed partition group size. This 
is to avoid tasks
-// unevenly distributed on nodes when running batch jobs or 
running jobs in
-// session/standalone mode.
-if ((double) consumedPartitionGroup.getConsumerVertexGroup().size()
-/ consumedPartitionGroup.size()
+// consumers. This is to avoid tasks unevenly distributed on nodes 
when running batch
+// jobs or running jobs in session/standalone mode.
+if (consumedPartitionGroup.getConsumerVertexGroup().size()
 > MAX_DISTINCT_CONSUMERS_TO_CONSIDER) {
 continue;
 }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetrieverTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetrieverTest.java
index 6fda52dc173..7409e297af3 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetrieverTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultPreferredLocationsRetrieverTest.java
@@ -71,26 +71,14 @@ class DefaultPreferredLocationsRetrieverTest {
 
 @Test
 void testInputLocations() {
-{
-final List producerLocations =
-Collections.singletonList(new LocalTaskManagerLocation());
-testInputLocationsInternal(
-1,
-MAX_DISTINCT_CONSUMERS_TO_CONSIDER,
-producerLocations,
-producerLocations,
-Collections.emptySet());
-}
-{
-final List producerLocations =
-Arrays.asList(new LocalTaskManagerLocation(), new 
LocalTaskManagerLocation());
-testInputLocationsInternal(
-2,
-MAX_DISTINCT_CONSUMERS_TO_CONSIDER * 2,
-producerLocations,
-producerLocations,
-Collections.emptySet());
-}
+final List producerLocations =
+Collections.singletonList(new LocalTaskManagerLocation());
+testInputLocationsInternal(
+1,
+MAX_DISTINCT_CONSUMERS_TO_CONSIDER,
+producerLocations,
+producerLocations,
+Collections.emptySet());
 }
 
 @Test
@@ -101,7 +89,7 @@ class DefaultPreferredLocationsRetrieverTest {
 @Test
 void testInputLocationsIgnoresEdgeOfTooManyConsumers() {
 testNoPreferredInputLocationsInternal(1, 
MAX_DISTINCT_CONSUMERS_TO_CONSIDER + 1);
-testNoPreferredInputLocationsInternal(2, 
MAX_DISTINCT_CONSUMERS_TO_CONSIDER * 2 + 1);
+testNoPreferredInputLocationsInternal(2, 
MAX_DISTINCT_CONSUMERS_TO_CONSIDER + 1);
 }
 
 @Test



[flink] branch master updated (84d000cf47b -> 6d79f6fc52b)

2023-03-10 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 84d000cf47b [FLINK-31383] Add support for documenting 
additionProperties of the REST API payloads.
 add 6d79f6fc52b [FLINK-31144][coordination] Ignore the input locations of 
a ConsumePartitionGroup if it has too many consumers

No new revisions were added by this update.

Summary of changes:
 .../DefaultPreferredLocationsRetriever.java|  8 +++---
 .../DefaultPreferredLocationsRetrieverTest.java| 30 +++---
 2 files changed, 12 insertions(+), 26 deletions(-)



[flink] branch release-1.16 updated (2f0df8076cf -> 5b9398ae067)

2023-02-28 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


from 2f0df8076cf [FLINK-29816][streaming] Fix the bug that StreamTask 
doesn't handle exception during restoring
 add 5b9398ae067 [FLINK-29859][e2e] Running TPC-DS with adaptive batch 
scheduler supports custom errors check.

No new revisions were added by this update.

Summary of changes:
 flink-end-to-end-tests/test-scripts/common.sh | 78 +++
 flink-end-to-end-tests/test-scripts/test_tpcds.sh |  8 ++-
 2 files changed, 59 insertions(+), 27 deletions(-)



[flink] branch release-1.17 updated: [FLINK-29859][e2e] Running TPC-DS with adaptive batch scheduler supports custom errors check.

2023-02-28 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.17 by this push:
 new 3de6b337cce [FLINK-29859][e2e] Running TPC-DS with adaptive batch 
scheduler supports custom errors check.
3de6b337cce is described below

commit 3de6b337cceda5b4b81ac8d34e099e3f1c802df9
Author: JunRuiLee 
AuthorDate: Mon Feb 27 16:42:45 2023 +0800

[FLINK-29859][e2e] Running TPC-DS with adaptive batch scheduler supports 
custom errors check.

This closes #22029.
---
 flink-end-to-end-tests/test-scripts/common.sh | 79 +++
 flink-end-to-end-tests/test-scripts/test_tpcds.sh |  8 ++-
 2 files changed, 60 insertions(+), 27 deletions(-)

diff --git a/flink-end-to-end-tests/test-scripts/common.sh 
b/flink-end-to-end-tests/test-scripts/common.sh
index 44f0ed4c5df..1bef68bbefb 100644
--- a/flink-end-to-end-tests/test-scripts/common.sh
+++ b/flink-end-to-end-tests/test-scripts/common.sh
@@ -355,33 +355,60 @@ function wait_for_number_of_running_tms {
 }
 
 function check_logs_for_errors {
+  internal_check_logs_for_errors
+}
+
+# check logs for errors, the arguments are the additional allowed errors
+function internal_check_logs_for_errors {
   echo "Checking for errors..."
-  error_count=$(grep -rv "GroupCoordinatorNotAvailableException" 
$FLINK_LOG_DIR \
-  | grep -v "RetriableCommitFailedException" \
-  | grep -v "NoAvailableBrokersException" \
-  | grep -v "Async Kafka commit failed" \
-  | grep -v "DisconnectException" \
-  | grep -v "Cannot connect to ResourceManager right now" \
-  | grep -v "AskTimeoutException" \
-  | grep -v "Error while loading kafka-version.properties" \
-  | grep -v "WARN  akka.remote.transport.netty.NettyTransport" \
-  | grep -v "WARN  org.jboss.netty.channel.DefaultChannelPipeline" \
-  | grep -v "jvm-exit-on-fatal-error" \
-  | grep -v 'INFO.*AWSErrorCode' \
-  | grep -v "RejectedExecutionException" \
-  | grep -v "An exception was thrown by an exception handler" \
-  | grep -v "java.lang.NoClassDefFoundError: 
org/apache/hadoop/yarn/exceptions/YarnException" \
-  | grep -v "java.lang.NoClassDefFoundError: 
org/apache/hadoop/conf/Configuration" \
-  | grep -v 
"org.apache.commons.beanutils.FluentPropertyBeanIntrospector.*Error when 
creating 
PropertyDescriptor.*org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)!
 Ignoring this property." \
-  | grep -v "Error while loading kafka-version.properties :null" \
-  | grep -v "[Terror] modules" \
-  | grep -v "HeapDumpOnOutOfMemoryError" \
-  | grep -v "error_prone_annotations" \
-  | grep -v "Error sending fetch request" \
-  | grep -v "WARN  akka.remote.ReliableDeliverySupervisor" \
-  | grep -v "Options.*error_*" \
-  | grep -v "not packaged with this application" \
-  | grep -ic "error" || true)
+
+  local additional_allowed_errors=()
+  local index=0
+  for error in "$@"; do
+additional_allowed_errors[index]="$error"
+index=$index+1
+  done
+
+  local default_allowed_errors=("GroupCoordinatorNotAvailableException" \
+  "RetriableCommitFailedException" \
+  "NoAvailableBrokersException" \
+  "Async Kafka commit failed" \
+  "DisconnectException" \
+  "Cannot connect to ResourceManager right now" \
+  "AskTimeoutException" \
+  "Error while loading kafka-version.properties" \
+  "WARN  akka.remote.transport.netty.NettyTransport" \
+  "WARN  org.jboss.netty.channel.DefaultChannelPipeline" \
+  "jvm-exit-on-fatal-error" \
+  'INFO.*AWSErrorCode' \
+  "RejectedExecutionException" \
+  "An exception was thrown by an exception handler" \
+  "java.lang.NoClassDefFoundError: 
org/apache/hadoop/yarn/exceptions/YarnException" \
+  "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \
+  "org.apache.commons.beanutils.FluentPropertyBeanIntrospector.*Error when 
creating 
PropertyDescriptor.*org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)!
 Ignoring this property." \
+  "Error while loading kafka-version.properties :null" \
+  "[Terror] modules" \
+  "HeapDumpOnOutOfMemoryError" \
+  "error_prone_annotations" \
+  "Error sending fetch request" \
+  "WARN  akka.remote.ReliableDeliverySupervisor" \
+  "Options.*error_*" 

[flink] branch master updated: [FLINK-29859][e2e] Running TPC-DS with adaptive batch scheduler supports custom errors check.

2023-02-28 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new 28e75620db7 [FLINK-29859][e2e] Running TPC-DS with adaptive batch 
scheduler supports custom errors check.
28e75620db7 is described below

commit 28e75620db7b0794ff1c19f8928ce7d33516ba64
Author: JunRuiLee 
AuthorDate: Mon Feb 27 16:42:45 2023 +0800

[FLINK-29859][e2e] Running TPC-DS with adaptive batch scheduler supports 
custom errors check.

This closes #22029.
---
 flink-end-to-end-tests/test-scripts/common.sh | 79 +++
 flink-end-to-end-tests/test-scripts/test_tpcds.sh |  8 ++-
 2 files changed, 60 insertions(+), 27 deletions(-)

diff --git a/flink-end-to-end-tests/test-scripts/common.sh 
b/flink-end-to-end-tests/test-scripts/common.sh
index 44f0ed4c5df..1bef68bbefb 100644
--- a/flink-end-to-end-tests/test-scripts/common.sh
+++ b/flink-end-to-end-tests/test-scripts/common.sh
@@ -355,33 +355,60 @@ function wait_for_number_of_running_tms {
 }
 
 function check_logs_for_errors {
+  internal_check_logs_for_errors
+}
+
+# check logs for errors, the arguments are the additional allowed errors
+function internal_check_logs_for_errors {
   echo "Checking for errors..."
-  error_count=$(grep -rv "GroupCoordinatorNotAvailableException" 
$FLINK_LOG_DIR \
-  | grep -v "RetriableCommitFailedException" \
-  | grep -v "NoAvailableBrokersException" \
-  | grep -v "Async Kafka commit failed" \
-  | grep -v "DisconnectException" \
-  | grep -v "Cannot connect to ResourceManager right now" \
-  | grep -v "AskTimeoutException" \
-  | grep -v "Error while loading kafka-version.properties" \
-  | grep -v "WARN  akka.remote.transport.netty.NettyTransport" \
-  | grep -v "WARN  org.jboss.netty.channel.DefaultChannelPipeline" \
-  | grep -v "jvm-exit-on-fatal-error" \
-  | grep -v 'INFO.*AWSErrorCode' \
-  | grep -v "RejectedExecutionException" \
-  | grep -v "An exception was thrown by an exception handler" \
-  | grep -v "java.lang.NoClassDefFoundError: 
org/apache/hadoop/yarn/exceptions/YarnException" \
-  | grep -v "java.lang.NoClassDefFoundError: 
org/apache/hadoop/conf/Configuration" \
-  | grep -v 
"org.apache.commons.beanutils.FluentPropertyBeanIntrospector.*Error when 
creating 
PropertyDescriptor.*org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)!
 Ignoring this property." \
-  | grep -v "Error while loading kafka-version.properties :null" \
-  | grep -v "[Terror] modules" \
-  | grep -v "HeapDumpOnOutOfMemoryError" \
-  | grep -v "error_prone_annotations" \
-  | grep -v "Error sending fetch request" \
-  | grep -v "WARN  akka.remote.ReliableDeliverySupervisor" \
-  | grep -v "Options.*error_*" \
-  | grep -v "not packaged with this application" \
-  | grep -ic "error" || true)
+
+  local additional_allowed_errors=()
+  local index=0
+  for error in "$@"; do
+additional_allowed_errors[index]="$error"
+index=$index+1
+  done
+
+  local default_allowed_errors=("GroupCoordinatorNotAvailableException" \
+  "RetriableCommitFailedException" \
+  "NoAvailableBrokersException" \
+  "Async Kafka commit failed" \
+  "DisconnectException" \
+  "Cannot connect to ResourceManager right now" \
+  "AskTimeoutException" \
+  "Error while loading kafka-version.properties" \
+  "WARN  akka.remote.transport.netty.NettyTransport" \
+  "WARN  org.jboss.netty.channel.DefaultChannelPipeline" \
+  "jvm-exit-on-fatal-error" \
+  'INFO.*AWSErrorCode' \
+  "RejectedExecutionException" \
+  "An exception was thrown by an exception handler" \
+  "java.lang.NoClassDefFoundError: 
org/apache/hadoop/yarn/exceptions/YarnException" \
+  "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \
+  "org.apache.commons.beanutils.FluentPropertyBeanIntrospector.*Error when 
creating 
PropertyDescriptor.*org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)!
 Ignoring this property." \
+  "Error while loading kafka-version.properties :null" \
+  "[Terror] modules" \
+  "HeapDumpOnOutOfMemoryError" \
+  "error_prone_annotations" \
+  "Error sending fetch request" \
+  "WARN  akka.remote.ReliableDeliverySupervisor" \
+  "Options.*error_*" \
+  "not 

[flink] branch release-1.16 updated (cd7b0495bcd -> 4c815914002)

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


from cd7b0495bcd [FLINK-31183][Connector/Kinesis] Fix bug where EFO 
Consumer can fail to stop gracefully during stop-with-savepoint
 add b60178b72dd [hotfix] Migrate DefaultSchedulerTest to Junit5 and 
AssertJ.
 add ca60b10e299 [hotfix] Migrate PendingCheckpointTest to Junit5 and 
AssertJ.
 add 4c815914002 [FLINK-31077][runtime] Mark pending checkpoint 
onCompletionPromise complete only after the completed checkpoint is added to 
the checkpoint store.

No new revisions were added by this update.

Summary of changes:
 .../runtime/checkpoint/CheckpointCoordinator.java  |  15 +-
 .../runtime/checkpoint/PendingCheckpoint.java  |   7 +-
 .../runtime/checkpoint/PendingCheckpointTest.java  | 160 +++
 .../runtime/scheduler/DefaultSchedulerTest.java| 527 -
 .../ExceptionHistoryEntryTestingUtils.java |  90 
 5 files changed, 469 insertions(+), 330 deletions(-)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryTestingUtils.java



[flink] 03/03: [FLINK-31077][runtime] Mark pending checkpoint onCompletionPromise complete only after the completed checkpoint is added to the checkpoint store.

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit dca819556fb9b675852df99ada45e0f22262cb28
Author: JunRuiLee 
AuthorDate: Thu Feb 16 13:47:29 2023 +0800

[FLINK-31077][runtime] Mark pending checkpoint onCompletionPromise complete 
only after the completed checkpoint is added to the checkpoint store.

This closes #21943.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 15 +++--
 .../runtime/checkpoint/PendingCheckpoint.java  |  7 +-
 .../runtime/checkpoint/PendingCheckpointTest.java  |  2 +-
 .../runtime/scheduler/DefaultSchedulerTest.java| 74 ++
 4 files changed, 90 insertions(+), 8 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index e485b461557..c7f84e73bc1 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -1314,14 +1314,18 @@ public class CheckpointCoordinator {
 if (!props.isSavepoint()) {
 lastSubsumed =
 addCompletedCheckpointToStoreAndSubsumeOldest(
-checkpointId,
-completedCheckpoint,
-
pendingCheckpoint.getCheckpointPlan().getTasksToCommitTo());
+checkpointId, completedCheckpoint, 
pendingCheckpoint);
 } else {
 lastSubsumed = null;
 }
 
+
pendingCheckpoint.getCompletionFuture().complete(completedCheckpoint);
 reportCompletedCheckpoint(completedCheckpoint);
+} catch (Exception exception) {
+// For robustness reasons, we need catch exception and try marking 
the checkpoint
+// completed.
+
pendingCheckpoint.getCompletionFuture().completeExceptionally(exception);
+throw exception;
 } finally {
 pendingCheckpoints.remove(checkpointId);
 scheduleTriggerRequest();
@@ -1442,8 +1446,10 @@ public class CheckpointCoordinator {
 private CompletedCheckpoint addCompletedCheckpointToStoreAndSubsumeOldest(
 long checkpointId,
 CompletedCheckpoint completedCheckpoint,
-List tasksToAbort)
+PendingCheckpoint pendingCheckpoint)
 throws CheckpointException {
+List tasksToAbort =
+pendingCheckpoint.getCheckpointPlan().getTasksToCommitTo();
 try {
 final CompletedCheckpoint subsumedCheckpoint =
 completedCheckpointStore.addCheckpointAndSubsumeOldestOne(
@@ -1453,6 +1459,7 @@ public class CheckpointCoordinator {
 this.forceFullSnapshot = false;
 return subsumedCheckpoint;
 } catch (Exception exception) {
+
pendingCheckpoint.getCompletionFuture().completeExceptionally(exception);
 if (exception instanceof PossibleInconsistentStateException) {
 LOG.warn(
 "An error occurred while writing checkpoint {} to the 
underlying metadata"
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 cbe096a5481..a02d0d1d29e 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
@@ -107,7 +107,10 @@ public class PendingCheckpoint implements Checkpoint {
 /** The checkpoint properties. */
 private final CheckpointProperties props;
 
-/** The promise to fulfill once the checkpoint has been completed. */
+/**
+ * The promise to fulfill once the checkpoint has been completed. Note 
that it will be completed
+ * only after the checkpoint is successfully added to 
CompletedCheckpointStore.
+ */
 private final CompletableFuture onCompletionPromise;
 
 @Nullable private final PendingCheckpointStats pendingCheckpointStats;
@@ -340,8 +343,6 @@ public class PendingCheckpoint implements Checkpoint {
 finalizedLocation,
 toCompletedCheckpointStats(finalizedLocation));
 
-onCompletionPromise.complete(completed);
-
 // mark this pending checkpoint as disposed, but do NOT drop 
the state
 dispose(false, checkpointsCleaner, postCleanup, executor);
 
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
 
b/flink-runtime/src/tes

[flink] 01/03: [hotfix] Migrate DefaultSchedulerTest to Junit5 and AssertJ.

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit 6f30f5b3cde5bb2b3a20bc9699950694e84278a1
Author: JunRuiLee 
AuthorDate: Wed Feb 15 18:30:03 2023 +0800

[hotfix] Migrate DefaultSchedulerTest to Junit5 and AssertJ.
---
 .../runtime/scheduler/DefaultSchedulerTest.java| 453 ++---
 .../ExceptionHistoryEntryTestingUtils.java |  90 
 2 files changed, 308 insertions(+), 235 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
index 59122521cb2..49ba055732b 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
@@ -75,7 +75,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
 import org.apache.flink.runtime.metrics.util.TestingMetricRegistry;
 import org.apache.flink.runtime.scheduler.adaptive.AdaptiveSchedulerTest;
-import 
org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntryMatcher;
+import 
org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntryTestingUtils;
 import 
org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry;
 import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
 import 
org.apache.flink.runtime.scheduler.strategy.PipelinedRegionSchedulingStrategy;
@@ -99,16 +99,9 @@ import org.apache.flink.util.concurrent.ScheduledExecutor;
 
 import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
 
-import org.hamcrest.collection.IsEmptyIterable;
-import org.hamcrest.collection.IsIterableContainingInOrder;
-import org.hamcrest.collection.IsIterableWithSize;
-import org.hamcrest.core.Is;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 
 import java.time.Duration;
@@ -141,28 +134,14 @@ import static 
org.apache.flink.runtime.scheduler.SchedulerTestingUtils.enableChe
 import static 
org.apache.flink.runtime.scheduler.SchedulerTestingUtils.getCheckpointCoordinator;
 import static org.apache.flink.util.ExceptionUtils.findThrowable;
 import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.hasSize;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link DefaultScheduler}. */
 public class DefaultSchedulerTest extends TestLogger {
 
 private static final int TIMEOUT_MS = 1000;
 
-@ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new 
TemporaryFolder();
-
 private final ManuallyTriggeredScheduledExecutor taskRestartExecutor =
 new ManuallyTriggeredScheduledExecutor();
 
@@ -188,8 +167,8 @@ public class DefaultSchedulerTest extends TestLogger {
 
 private Time timeout;
 
-@Before
-public void setUp() throws Exception {
+@BeforeEach
+void setUp() {
 executor = Executors.newSingleThreadExecutor();
 scheduledExecutorService = new DirectScheduledExecutorService();
 
@@ -211,8 +190,8 @@ public class DefaultSchedulerTest extends TestLogger {
 timeout = Time.seconds(60);
 }
 
-@After
-public void tearDown() throws Exception {
+@AfterEach
+void tearDown() {
 if (scheduledExecutorService != null) {
 ExecutorUtils.gracefulShutdown(
 TIMEOUT_MS, TimeUnit.MILLISECONDS, 
scheduledExecutorService);
@@ -224,7 +203,7 @@ public class DefaultSchedulerTest extends TestLogger {
 }
 
 @Test
-public void startScheduling() {
+void startScheduling() {
 final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
 final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph);
 
@@ -234,11 +213,11 @@ public class

[flink] branch release-1.17 updated (5ee98b544d9 -> dca819556fb)

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


from 5ee98b544d9 [FLINK-30396][table-planner] Fix incorrect hint 
propagation into sub-query of lookup join hint
 new 6f30f5b3cde [hotfix] Migrate DefaultSchedulerTest to Junit5 and 
AssertJ.
 new 57175e81c80 [hotfix] Migrate PendingCheckpointTest to Junit5 and 
AssertJ.
 new dca819556fb [FLINK-31077][runtime] Mark pending checkpoint 
onCompletionPromise complete only after the completed checkpoint is added to 
the checkpoint store.

The 3 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:
 .../runtime/checkpoint/CheckpointCoordinator.java  |  15 +-
 .../runtime/checkpoint/PendingCheckpoint.java  |   7 +-
 .../runtime/checkpoint/PendingCheckpointTest.java  | 160 +++
 .../runtime/scheduler/DefaultSchedulerTest.java| 527 -
 .../ExceptionHistoryEntryTestingUtils.java |  90 
 5 files changed, 469 insertions(+), 330 deletions(-)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryTestingUtils.java



[flink] 02/03: [hotfix] Migrate PendingCheckpointTest to Junit5 and AssertJ.

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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

commit 57175e81c80134bac53ac7beaf2d9b8b14ecddde
Author: JunRuiLee 
AuthorDate: Wed Feb 15 18:30:44 2023 +0800

[hotfix] Migrate PendingCheckpointTest to Junit5 and AssertJ.
---
 .../runtime/checkpoint/PendingCheckpointTest.java  | 160 ++---
 1 file changed, 72 insertions(+), 88 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
index 49a4da447d1..4724ccd765f 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
@@ -39,13 +39,11 @@ import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.TestingStreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
 import org.apache.flink.util.concurrent.Executors;
 
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 import org.mockito.Mockito;
 
 import javax.annotation.Nullable;
@@ -66,13 +64,8 @@ import java.util.concurrent.ScheduledFuture;
 
 import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createExecutionAttemptId;
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doNothing;
@@ -82,7 +75,7 @@ import static org.mockito.Mockito.verify;
 import static org.powermock.api.mockito.PowerMockito.when;
 
 /** Tests for the {@link PendingCheckpoint}. */
-public class PendingCheckpointTest {
+class PendingCheckpointTest {
 
 private static final List ACK_TASKS = new ArrayList<>();
 private static final List TASKS_TO_COMMIT = new 
ArrayList<>();
@@ -111,9 +104,7 @@ public class PendingCheckpointTest {
 TASKS_TO_COMMIT.add(vertex);
 }
 
-@Rule public final TemporaryFolder tmpFolder = new TemporaryFolder();
-
-@Rule public final ExpectedException expectedException = 
ExpectedException.none();
+@TempDir private java.nio.file.Path tmpFolder;
 
 /** Tests that pending checkpoints can be subsumed iff they are forced. */
 @Test
@@ -129,15 +120,13 @@ public class PendingCheckpointTest {
 false,
 false,
 false);
-PendingCheckpoint pending = createPendingCheckpoint(forced);
-assertFalse(pending.canBeSubsumed());
+final PendingCheckpoint pending = createPendingCheckpoint(forced);
+assertThat(pending.canBeSubsumed()).isFalse();
 
-try {
-abort(pending, CheckpointFailureReason.CHECKPOINT_SUBSUMED);
-fail("Did not throw expected Exception");
-} catch (IllegalStateException ignored) {
-// Expected
-}
+assertThatThrownBy(
+() -> abort(pending, 
CheckpointFailureReason.CHECKPOINT_SUBSUMED),
+"Did not throw expected Exception")
+.isInstanceOf(IllegalStateException.class);
 
 // Non-forced checkpoints can be subsumed
 CheckpointProperties subsumed =
@@ -150,8 +139,7 @@ public class PendingCheckpointTest {
 false,
 false,
 false);
-pending = createPendingCheckpoint(subsumed);
-assertFalse(pending.canBeSubsumed());
+
assertThat(createPendingCheckpoint(subsumed).canBeSubsumed()).isFalse();
 }
 
 @Test
@@ -160,14 +148,12 @@ public class PendingCheckpointTest {
 CheckpointProperties forced =
 CheckpointProperties.forSyncSavepoint(true, false, 
SavepointFormatType.CANONICAL);
 PendingCheckpoint pending = createPendingCheckpoint(forced);
-assertFalse(pending.canBeSubsumed());
+assertThat(pending.canBeSubsumed()).isFalse();
 
- 

[flink] 01/03: [hotfix] Migrate DefaultSchedulerTest to Junit5 and AssertJ.

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0351f88cea26dc57ca7aeb78564a8e2acffc505a
Author: JunRuiLee 
AuthorDate: Wed Feb 15 18:30:03 2023 +0800

[hotfix] Migrate DefaultSchedulerTest to Junit5 and AssertJ.
---
 .../runtime/scheduler/DefaultSchedulerTest.java| 453 ++---
 .../ExceptionHistoryEntryTestingUtils.java |  90 
 2 files changed, 308 insertions(+), 235 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
index 59122521cb2..49ba055732b 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
@@ -75,7 +75,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry;
 import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
 import org.apache.flink.runtime.metrics.util.TestingMetricRegistry;
 import org.apache.flink.runtime.scheduler.adaptive.AdaptiveSchedulerTest;
-import 
org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntryMatcher;
+import 
org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntryTestingUtils;
 import 
org.apache.flink.runtime.scheduler.exceptionhistory.RootExceptionHistoryEntry;
 import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
 import 
org.apache.flink.runtime.scheduler.strategy.PipelinedRegionSchedulingStrategy;
@@ -99,16 +99,9 @@ import org.apache.flink.util.concurrent.ScheduledExecutor;
 
 import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
 
-import org.hamcrest.collection.IsEmptyIterable;
-import org.hamcrest.collection.IsIterableContainingInOrder;
-import org.hamcrest.collection.IsIterableWithSize;
-import org.hamcrest.core.Is;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 
 import java.time.Duration;
@@ -141,28 +134,14 @@ import static 
org.apache.flink.runtime.scheduler.SchedulerTestingUtils.enableChe
 import static 
org.apache.flink.runtime.scheduler.SchedulerTestingUtils.getCheckpointCoordinator;
 import static org.apache.flink.util.ExceptionUtils.findThrowable;
 import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage;
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.hasSize;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link DefaultScheduler}. */
 public class DefaultSchedulerTest extends TestLogger {
 
 private static final int TIMEOUT_MS = 1000;
 
-@ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new 
TemporaryFolder();
-
 private final ManuallyTriggeredScheduledExecutor taskRestartExecutor =
 new ManuallyTriggeredScheduledExecutor();
 
@@ -188,8 +167,8 @@ public class DefaultSchedulerTest extends TestLogger {
 
 private Time timeout;
 
-@Before
-public void setUp() throws Exception {
+@BeforeEach
+void setUp() {
 executor = Executors.newSingleThreadExecutor();
 scheduledExecutorService = new DirectScheduledExecutorService();
 
@@ -211,8 +190,8 @@ public class DefaultSchedulerTest extends TestLogger {
 timeout = Time.seconds(60);
 }
 
-@After
-public void tearDown() throws Exception {
+@AfterEach
+void tearDown() {
 if (scheduledExecutorService != null) {
 ExecutorUtils.gracefulShutdown(
 TIMEOUT_MS, TimeUnit.MILLISECONDS, 
scheduledExecutorService);
@@ -224,7 +203,7 @@ public class DefaultSchedulerTest extends TestLogger {
 }
 
 @Test
-public void startScheduling() {
+void startScheduling() {
 final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
 final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph);
 
@@ -234,11 +213,11 @@ public class

[flink] branch master updated (2e0efe4e072 -> eb17ec3f05d)

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


from 2e0efe4e072 [FLINK-30277][python]Allow PYTHONPATH of Python Worker 
configurable
 new 0351f88cea2 [hotfix] Migrate DefaultSchedulerTest to Junit5 and 
AssertJ.
 new c9f1e89b485 [hotfix] Migrate PendingCheckpointTest to Junit5 and 
AssertJ.
 new eb17ec3f05d [FLINK-31077][runtime] Mark pending checkpoint 
onCompletionPromise complete only after the completed checkpoint is added to 
the checkpoint store.

The 3 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:
 .../runtime/checkpoint/CheckpointCoordinator.java  |  15 +-
 .../runtime/checkpoint/PendingCheckpoint.java  |   7 +-
 .../runtime/checkpoint/PendingCheckpointTest.java  | 160 +++
 .../runtime/scheduler/DefaultSchedulerTest.java| 527 -
 .../ExceptionHistoryEntryTestingUtils.java |  90 
 5 files changed, 469 insertions(+), 330 deletions(-)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/ExceptionHistoryEntryTestingUtils.java



[flink] 03/03: [FLINK-31077][runtime] Mark pending checkpoint onCompletionPromise complete only after the completed checkpoint is added to the checkpoint store.

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit eb17ec3f05d4bd512bc70ee79296d0b884894eaf
Author: JunRuiLee 
AuthorDate: Thu Feb 16 13:47:29 2023 +0800

[FLINK-31077][runtime] Mark pending checkpoint onCompletionPromise complete 
only after the completed checkpoint is added to the checkpoint store.

This closes #21943.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 15 +++--
 .../runtime/checkpoint/PendingCheckpoint.java  |  7 +-
 .../runtime/checkpoint/PendingCheckpointTest.java  |  2 +-
 .../runtime/scheduler/DefaultSchedulerTest.java| 74 ++
 4 files changed, 90 insertions(+), 8 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index e485b461557..c7f84e73bc1 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -1314,14 +1314,18 @@ public class CheckpointCoordinator {
 if (!props.isSavepoint()) {
 lastSubsumed =
 addCompletedCheckpointToStoreAndSubsumeOldest(
-checkpointId,
-completedCheckpoint,
-
pendingCheckpoint.getCheckpointPlan().getTasksToCommitTo());
+checkpointId, completedCheckpoint, 
pendingCheckpoint);
 } else {
 lastSubsumed = null;
 }
 
+
pendingCheckpoint.getCompletionFuture().complete(completedCheckpoint);
 reportCompletedCheckpoint(completedCheckpoint);
+} catch (Exception exception) {
+// For robustness reasons, we need catch exception and try marking 
the checkpoint
+// completed.
+
pendingCheckpoint.getCompletionFuture().completeExceptionally(exception);
+throw exception;
 } finally {
 pendingCheckpoints.remove(checkpointId);
 scheduleTriggerRequest();
@@ -1442,8 +1446,10 @@ public class CheckpointCoordinator {
 private CompletedCheckpoint addCompletedCheckpointToStoreAndSubsumeOldest(
 long checkpointId,
 CompletedCheckpoint completedCheckpoint,
-List tasksToAbort)
+PendingCheckpoint pendingCheckpoint)
 throws CheckpointException {
+List tasksToAbort =
+pendingCheckpoint.getCheckpointPlan().getTasksToCommitTo();
 try {
 final CompletedCheckpoint subsumedCheckpoint =
 completedCheckpointStore.addCheckpointAndSubsumeOldestOne(
@@ -1453,6 +1459,7 @@ public class CheckpointCoordinator {
 this.forceFullSnapshot = false;
 return subsumedCheckpoint;
 } catch (Exception exception) {
+
pendingCheckpoint.getCompletionFuture().completeExceptionally(exception);
 if (exception instanceof PossibleInconsistentStateException) {
 LOG.warn(
 "An error occurred while writing checkpoint {} to the 
underlying metadata"
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 cbe096a5481..a02d0d1d29e 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
@@ -107,7 +107,10 @@ public class PendingCheckpoint implements Checkpoint {
 /** The checkpoint properties. */
 private final CheckpointProperties props;
 
-/** The promise to fulfill once the checkpoint has been completed. */
+/**
+ * The promise to fulfill once the checkpoint has been completed. Note 
that it will be completed
+ * only after the checkpoint is successfully added to 
CompletedCheckpointStore.
+ */
 private final CompletableFuture onCompletionPromise;
 
 @Nullable private final PendingCheckpointStats pendingCheckpointStats;
@@ -340,8 +343,6 @@ public class PendingCheckpoint implements Checkpoint {
 finalizedLocation,
 toCompletedCheckpointStats(finalizedLocation));
 
-onCompletionPromise.complete(completed);
-
 // mark this pending checkpoint as disposed, but do NOT drop 
the state
 dispose(false, checkpointsCleaner, postCleanup, executor);
 
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
 
b/flink-runtime/src/test/java/

[flink] 02/03: [hotfix] Migrate PendingCheckpointTest to Junit5 and AssertJ.

2023-02-26 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c9f1e89b48569aec092048b9db01cc2335c929d5
Author: JunRuiLee 
AuthorDate: Wed Feb 15 18:30:44 2023 +0800

[hotfix] Migrate PendingCheckpointTest to Junit5 and AssertJ.
---
 .../runtime/checkpoint/PendingCheckpointTest.java  | 160 ++---
 1 file changed, 72 insertions(+), 88 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
index 49a4da447d1..4724ccd765f 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
@@ -39,13 +39,11 @@ import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.TestingStreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+import org.apache.flink.testutils.junit.utils.TempDirUtils;
 import org.apache.flink.util.concurrent.Executors;
 
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 import org.mockito.Mockito;
 
 import javax.annotation.Nullable;
@@ -66,13 +64,8 @@ import java.util.concurrent.ScheduledFuture;
 
 import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createExecutionAttemptId;
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doNothing;
@@ -82,7 +75,7 @@ import static org.mockito.Mockito.verify;
 import static org.powermock.api.mockito.PowerMockito.when;
 
 /** Tests for the {@link PendingCheckpoint}. */
-public class PendingCheckpointTest {
+class PendingCheckpointTest {
 
 private static final List ACK_TASKS = new ArrayList<>();
 private static final List TASKS_TO_COMMIT = new 
ArrayList<>();
@@ -111,9 +104,7 @@ public class PendingCheckpointTest {
 TASKS_TO_COMMIT.add(vertex);
 }
 
-@Rule public final TemporaryFolder tmpFolder = new TemporaryFolder();
-
-@Rule public final ExpectedException expectedException = 
ExpectedException.none();
+@TempDir private java.nio.file.Path tmpFolder;
 
 /** Tests that pending checkpoints can be subsumed iff they are forced. */
 @Test
@@ -129,15 +120,13 @@ public class PendingCheckpointTest {
 false,
 false,
 false);
-PendingCheckpoint pending = createPendingCheckpoint(forced);
-assertFalse(pending.canBeSubsumed());
+final PendingCheckpoint pending = createPendingCheckpoint(forced);
+assertThat(pending.canBeSubsumed()).isFalse();
 
-try {
-abort(pending, CheckpointFailureReason.CHECKPOINT_SUBSUMED);
-fail("Did not throw expected Exception");
-} catch (IllegalStateException ignored) {
-// Expected
-}
+assertThatThrownBy(
+() -> abort(pending, 
CheckpointFailureReason.CHECKPOINT_SUBSUMED),
+"Did not throw expected Exception")
+.isInstanceOf(IllegalStateException.class);
 
 // Non-forced checkpoints can be subsumed
 CheckpointProperties subsumed =
@@ -150,8 +139,7 @@ public class PendingCheckpointTest {
 false,
 false,
 false);
-pending = createPendingCheckpoint(subsumed);
-assertFalse(pending.canBeSubsumed());
+
assertThat(createPendingCheckpoint(subsumed).canBeSubsumed()).isFalse();
 }
 
 @Test
@@ -160,14 +148,12 @@ public class PendingCheckpointTest {
 CheckpointProperties forced =
 CheckpointProperties.forSyncSavepoint(true, false, 
SavepointFormatType.CANONICAL);
 PendingCheckpoint pending = createPendingCheckpoint(forced);
-assertFalse(pending.canBeSubsumed());
+assertThat(pending.canBeSubsumed()).isFalse();
 
- 

[flink] branch release-1.17 updated: [FLINK-31123][connector/files] Add IT case for FileSink speculative execution

2023-02-23 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.17 by this push:
 new e0773803fa8 [FLINK-31123][connector/files] Add IT case for FileSink 
speculative execution
e0773803fa8 is described below

commit e0773803fa8b1b6a62ba02ffc778bbdbdd188b19
Author: frankeshi 
AuthorDate: Wed Feb 15 20:23:03 2023 +0800

[FLINK-31123][connector/files] Add IT case for FileSink speculative 
execution

This closes #21968.
---
 .../file/sink/FileSinkSpeculativeITCase.java   | 206 +
 1 file changed, 206 insertions(+)

diff --git 
a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkSpeculativeITCase.java
 
b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkSpeculativeITCase.java
new file mode 100644
index 000..948cfc3fe07
--- /dev/null
+++ 
b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkSpeculativeITCase.java
@@ -0,0 +1,206 @@
+/*
+ * 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.connector.file.sink;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.SlowTaskDetectorOptions;
+import org.apache.flink.connector.file.sink.utils.IntegerFileSinkTestDataUtils;
+import 
org.apache.flink.connector.file.sink.utils.PartSizeAndCheckpointRollingPolicy;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests {@link FileSink} with speculative scheduling enabled. */
+class FileSinkSpeculativeITCase {
+
+@RegisterExtension
+private static final MiniClusterExtension miniClusterExtension =
+new MiniClusterExtension(
+new MiniClusterResourceConfiguration.Builder()
+.setNumberTaskManagers(1)
+.setNumberSlotsPerTaskManager(4)
+.setConfiguration(configure())
+.build());
+
+@TempDir private Path tmpDir;
+
+private static final int NUM_SOURCES = 4;
+
+private static final int NUM_SINKS = 3;
+
+private static final int NUM_RECORDS = 1;
+
+private static final int NUM_BUCKETS = 4;
+
+private static final AtomicInteger slowTaskCounter = new AtomicInteger(1);
+
+@BeforeEach
+void setUp() {
+slowTaskCounter.set(1);
+}
+
+@Test
+void testFileSinkSpeculative() throws Exception {
+String path = tmpDir.toString();
+executeJobWithSlowSink(path);
+IntegerFileSinkTestDataUtils.checkIntegerSequenceSinkOutput(
+path, NUM_RECORDS, NUM_

[flink] branch master updated: [FLINK-31123][connector/files] Add IT case for FileSink speculative execution

2023-02-23 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
 new f244945aa7c [FLINK-31123][connector/files] Add IT case for FileSink 
speculative execution
f244945aa7c is described below

commit f244945aa7cf9d3fb5d2fd2fe49446dd759f9aef
Author: frankeshi 
AuthorDate: Wed Feb 15 20:23:03 2023 +0800

[FLINK-31123][connector/files] Add IT case for FileSink speculative 
execution

This closes #21968.
---
 .../file/sink/FileSinkSpeculativeITCase.java   | 206 +
 1 file changed, 206 insertions(+)

diff --git 
a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkSpeculativeITCase.java
 
b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkSpeculativeITCase.java
new file mode 100644
index 000..948cfc3fe07
--- /dev/null
+++ 
b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkSpeculativeITCase.java
@@ -0,0 +1,206 @@
+/*
+ * 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.connector.file.sink;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.configuration.SlowTaskDetectorOptions;
+import org.apache.flink.connector.file.sink.utils.IntegerFileSinkTestDataUtils;
+import 
org.apache.flink.connector.file.sink.utils.PartSizeAndCheckpointRollingPolicy;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests {@link FileSink} with speculative scheduling enabled. */
+class FileSinkSpeculativeITCase {
+
+@RegisterExtension
+private static final MiniClusterExtension miniClusterExtension =
+new MiniClusterExtension(
+new MiniClusterResourceConfiguration.Builder()
+.setNumberTaskManagers(1)
+.setNumberSlotsPerTaskManager(4)
+.setConfiguration(configure())
+.build());
+
+@TempDir private Path tmpDir;
+
+private static final int NUM_SOURCES = 4;
+
+private static final int NUM_SINKS = 3;
+
+private static final int NUM_RECORDS = 1;
+
+private static final int NUM_BUCKETS = 4;
+
+private static final AtomicInteger slowTaskCounter = new AtomicInteger(1);
+
+@BeforeEach
+void setUp() {
+slowTaskCounter.set(1);
+}
+
+@Test
+void testFileSinkSpeculative() throws Exception {
+String path = tmpDir.toString();
+executeJobWithSlowSink(path);
+IntegerFileSinkTestDataUtils.checkIntegerSequenceSinkOutput(
+path, NUM_RECORDS, NUM_BUCKETS, NUM_SOURCES);
+}

[flink] branch release-1.17 updated: [FLINK-31124][Connectors/Hive] Add IT case for HiveTableSink speculative execution

2023-02-22 Thread zhuzh
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.17 by this push:
 new c94a0799016 [FLINK-31124][Connectors/Hive] Add IT case for 
HiveTableSink speculative execution
c94a0799016 is described below

commit c94a0799016ae7f8d6e348930124936bc71a61aa
Author: shuiqiangchen 
AuthorDate: Sat Feb 18 21:24:58 2023 +0800

[FLINK-31124][Connectors/Hive] Add IT case for HiveTableSink speculative 
execution

This closes #21962.
---
 .../hive/HiveTableSpeculativeSinkITCase.java   | 190 +
 1 file changed, 190 insertions(+)

diff --git 
a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSpeculativeSinkITCase.java
 
b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSpeculativeSinkITCase.java
new file mode 100644
index 000..c4a1ca36cd2
--- /dev/null
+++ 
b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSpeculativeSinkITCase.java
@@ -0,0 +1,190 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.BatchExecutionOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.SlowTaskDetectorOptions;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.SqlDialect;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import 
org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.table.catalog.hive.HiveCatalog;
+import org.apache.flink.table.catalog.hive.HiveTestUtils;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.operations.ModifyOperation;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static 
org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests {@link HiveTableSink} with {@link
+ * org.apache.flink.connector.file.table.FileSystemOutputFormat} and 
speculative execution enabled.
+ */
+class HiveTableSpeculativeSinkITCase {
+
+private static final int PARALLELISM = 3;
+
+@RegisterExtension
+private static final MiniClusterExtension MINI_CLUSTER =
+new MiniClusterExtension(
+new MiniClusterResourceConfiguration.Builder()
+.setConfiguration(configure(new Configuration()))
+.setNumberTaskManagers(1)
+.setNumberSlotsPerTaskManager(PARALLELISM)
+.build());
+
+private HiveCatalog hiveCatalog;
+
+@BeforeEach
+void createCatalog() {
+hiveCatalog = HiveTestUtils.createHiveCatalog();
+hiveCatalog.open();
+}
+
+@AfterEach
+void closeCatalog() {
+if (hiveCatalog != null) {
+hiveCatalog.close();
+}

  1   2   3   4   5   6   >