[flink] branch master updated: [FLINK-33128] Add converter.open() method on TestValuesRuntimeFunctions
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 f31770fcf57 [FLINK-33128] Add converter.open() method on TestValuesRuntimeFunctions f31770fcf57 is described below commit f31770fcf5769052f1ac32a6529de979eaf339a4 Author: Jerome Gagnon AuthorDate: Fri Sep 22 14:35:19 2023 -0400 [FLINK-33128] Add converter.open() method on TestValuesRuntimeFunctions This closes #23453. --- .../table/planner/factories/TestValuesRuntimeFunctions.java | 13 +++-- 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java index a381d573c41..3ab136d451f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.connector.RuntimeConverter; import org.apache.flink.table.connector.sink.DynamicTableSink.DataStructureConverter; import org.apache.flink.table.connector.source.LookupTableSource; import org.apache.flink.table.data.GenericRowData; @@ -631,11 +632,11 @@ final class TestValuesRuntimeFunctions { public void open(FunctionContext context) throws Exception { RESOURCE_COUNTER.incrementAndGet(); isOpenCalled = true; +ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); if (projectable) { -projection = -generatedProjection.newInstance( - Thread.currentThread().getContextClassLoader()); +projection = generatedProjection.newInstance(classLoader); } +converter.open(RuntimeConverter.Context.create(classLoader)); rowSerializer = InternalSerializers.create(producedRowType); indexDataByKey(); } @@ -725,11 +726,11 @@ final class TestValuesRuntimeFunctions { @Override public void open(FunctionContext context) throws Exception { RESOURCE_COUNTER.incrementAndGet(); +ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); if (projectable) { -projection = -generatedProjection.newInstance( - Thread.currentThread().getContextClassLoader()); +projection = generatedProjection.newInstance(classLoader); } +converter.open(RuntimeConverter.Context.create(classLoader)); rowSerializer = InternalSerializers.create(producedRowType); isOpenCalled = true; // generate unordered result for async lookup
[flink] branch master updated: [FLINK-31721][core] Move JobStatusHook to flink-core module
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 fa94fb5a027 [FLINK-31721][core] Move JobStatusHook to flink-core module fa94fb5a027 is described below commit fa94fb5a0271590c89edf6f3c0d4f274cf995e7d Author: Tartarus0zm AuthorDate: Fri Jun 16 10:43:57 2023 +0800 [FLINK-31721][core] Move JobStatusHook to flink-core module This closes #22344. --- .../src/main/java/org/apache/flink/core/execution}/JobStatusHook.java | 2 +- .../org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java | 1 + .../src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java | 2 +- .../org/apache/flink/runtime/executiongraph/TestingJobStatusHook.java | 1 + .../java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java | 2 +- .../src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java | 2 +- 6 files changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusHook.java b/flink-core/src/main/java/org/apache/flink/core/execution/JobStatusHook.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusHook.java rename to flink-core/src/main/java/org/apache/flink/core/execution/JobStatusHook.java index 231cbab87cd..5a6467a66d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusHook.java +++ b/flink-core/src/main/java/org/apache/flink/core/execution/JobStatusHook.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.executiongraph; +package org.apache.flink.core.execution; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.JobID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java index 6d7f325a590..8c64183915d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraph.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobStatusHook; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.JobException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 14e49c89228..a8a1b9cafe6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -24,9 +24,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.StateChangelogOptionsInternal; +import org.apache.flink.core.execution.JobStatusHook; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.executiongraph.JobStatusHook; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingJobStatusHook.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingJobStatusHook.java index bad1af69002..4720ee62372 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingJobStatusHook.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingJobStatusHook.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.executiongraph; import org.apache.flink.api.common.JobID; +import org.apache.flink.core.execution.JobStatusHook; import java.util.function.BiConsumer; import java.util.function.Consumer; 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 5e60f34cdd8..1635cb41d79 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
[flink] branch native deleted (was f0d96eb3f94)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch native in repository https://gitbox.apache.org/repos/asf/flink.git was f0d96eb3f94 Tmp save This change permanently discards the following revisions: discard f0d96eb3f94 Tmp save discard 88b997806bb Tmp save
[flink] branch native created (now f0d96eb3f94)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch native in repository https://gitbox.apache.org/repos/asf/flink.git at f0d96eb3f94 Tmp save This branch includes the following new commits: new 88b997806bb Tmp save new f0d96eb3f94 Tmp save 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.
[flink] 02/02: Tmp save
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch native in repository https://gitbox.apache.org/repos/asf/flink.git commit f0d96eb3f94bb41dd3dd8e35d0d6aaab7ca69670 Author: Yun Gao AuthorDate: Sun May 28 10:28:32 2023 +0800 Tmp save --- .../natives/StreamExecColumnToRowExchange.java | 8 +- .../NativeRowColumnTranslationProcessor.java | 348 - .../stream/StreamExecGlobalWindowAggregate.java| 219 - .../exec/stream/StreamExecGroupAggregate.java | 171 +- .../StreamExecIncrementalGroupAggregate.java | 181 +-- .../plan/nodes/exec/stream/StreamExecJoin.java | 115 ++- .../exec/stream/StreamExecLocalGroupAggregate.java | 177 ++- .../stream/StreamExecLocalWindowAggregate.java | 231 +- .../exec/stream/StreamExecWindowAggregate.java | 231 +- .../AbstractExecNodeExactlyOnceVisitor.java| 2 +- 10 files changed, 1567 insertions(+), 116 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/natives/StreamExecColumnToRowExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/natives/StreamExecColumnToRowExchange.java index 3e4f253d786..a0ae1e2802e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/natives/StreamExecColumnToRowExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/natives/StreamExecColumnToRowExchange.java @@ -30,11 +30,7 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; -import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; -import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; -import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; -import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; +import org.apache.flink.table.planner.plan.nodes.exec.*; import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExchange; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecNode; @@ -57,7 +53,7 @@ import static org.apache.flink.runtime.state.KeyGroupRangeAssignment.DEFAULT_LOW minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) public class StreamExecColumnToRowExchange extends CommonExecExchange -implements StreamExecNode { +implements StreamExecNode, NativeSupportedExec { public static final String EXCHANGE_TRANSFORMATION = "exchange"; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/NativeRowColumnTranslationProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/NativeRowColumnTranslationProcessor.java index 40f8b6e6f9f..e6f21b360d4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/NativeRowColumnTranslationProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/NativeRowColumnTranslationProcessor.java @@ -19,26 +19,348 @@ package org.apache.flink.table.planner.plan.nodes.exec.processor; import org.apache.flink.configuration.ReadableConfig; - import org.apache.flink.streaming.api.transformations.StreamExchangeMode; -import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; +import org.apache.flink.table.planner.plan.nodes.exec.*; +import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExchange; +import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecUnion; +import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExchange; +import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecUnion; +import org.apache.flink.table.planner.plan.nodes.exec.natives.StreamExecColumnToColumnExchange; +import org.apache.flink.table.planner.plan.nodes.exec.natives.StreamExecColumnToRowExchange; +import org.apache.flink.table.planner.plan.nodes.exec.natives.StreamExecRowToColumnExchange; +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange; +import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion; +import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor; +import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.planner.plan.nodes.exec.ExecNod
[flink] branch release-1.16 updated (ab2ba9612a6 -> 389389fabd0)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from ab2ba9612a6 [FLINK-31763][runtime] Convert requested buffers to overdraft buffers when pool size is decreased add 389389fabd0 [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0. No new revisions were added by this update. Summary of changes: .../org/apache/flink/core/memory/DataInputDeserializer.java| 2 +- .../main/java/org/apache/flink/core/memory/DataInputView.java | 7 ++- flink-core/src/main/java/org/apache/flink/types/Record.java| 2 +- .../apache/flink/core/memory/DataInputDeserializerTest.java| 10 ++ 4 files changed, 18 insertions(+), 3 deletions(-)
[flink] branch release-1.17 updated: [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 729043df687 [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0. 729043df687 is described below commit 729043df687a96711d3591fcdf5e8e712cd21b87 Author: 沈嘉琦 AuthorDate: Mon Apr 3 19:43:14 2023 +0800 [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0. This closes #22335. --- .../org/apache/flink/core/memory/DataInputDeserializer.java| 2 +- .../main/java/org/apache/flink/core/memory/DataInputView.java | 7 ++- flink-core/src/main/java/org/apache/flink/types/Record.java| 2 +- .../apache/flink/core/memory/DataInputDeserializerTest.java| 10 ++ 4 files changed, 18 insertions(+), 3 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java index 9ece9f00135..051bad2650e 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java @@ -378,7 +378,7 @@ public class DataInputDeserializer implements DataInputView, java.io.Serializabl } if (this.position >= this.end) { -return -1; +return len == 0 ? 0 : -1; } else { int toRead = Math.min(this.end - this.position, len); System.arraycopy(this.buffer, this.position, b, off, toRead); diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java index 43f34f5383f..58d5c634aed 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java @@ -44,7 +44,12 @@ public interface DataInputView extends DataInput { /** * Reads up to {@code len} bytes of memory and stores it into {@code b} starting at offset - * {@code off}. It returns the number of read bytes or -1 if there is no more data left. + * {@code off}. + * + * If len is zero, then no bytes are read and 0 is returned; + * otherwise, there is an attempt to read at least one byte. If there is no more data left, the + * value -1 is returned; otherwise, at least one byte is read and stored into + * b. * * @param b byte array to store the data to * @param off offset into byte array diff --git a/flink-core/src/main/java/org/apache/flink/types/Record.java b/flink-core/src/main/java/org/apache/flink/types/Record.java index 59e1568fc8f..a7b6f989b7d 100644 --- a/flink-core/src/main/java/org/apache/flink/types/Record.java +++ b/flink-core/src/main/java/org/apache/flink/types/Record.java @@ -1584,7 +1584,7 @@ public final class Record implements Value, CopyableValue { } if (this.position >= this.end) { -return -1; +return len == 0 ? 0 : -1; } else { int toRead = Math.min(this.end - this.position, len); System.arraycopy(this.memory, this.position, b, off, toRead); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java index d8ebd5968cb..e8707f1eb70 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java @@ -54,4 +54,14 @@ public class DataInputDeserializerTest { } Assert.assertEquals(0, dis.available()); } + +@Test +public void testReadWithLenZero() throws IOException { +byte[] bytes = new byte[0]; +DataInputDeserializer dis = new DataInputDeserializer(bytes, 0, bytes.length); +Assert.assertEquals(0, dis.available()); + +byte[] bytesForRead = new byte[0]; +Assert.assertEquals(0, dis.read(bytesForRead, 0, 0)); // do not throw when read with len 0 +} }
[flink] branch master updated: [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 d9e9d1ca741 [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0. d9e9d1ca741 is described below commit d9e9d1ca741c84a18dbfac0ed44bd38cb5e11f3f Author: 沈嘉琦 AuthorDate: Mon Apr 3 19:43:14 2023 +0800 [FLINK-31708][API/Type Serialization System] Make DataInputView#read(byte[], int, int) return 0 if len is 0. This closes #22335. --- .../org/apache/flink/core/memory/DataInputDeserializer.java| 2 +- .../main/java/org/apache/flink/core/memory/DataInputView.java | 7 ++- flink-core/src/main/java/org/apache/flink/types/Record.java| 2 +- .../apache/flink/core/memory/DataInputDeserializerTest.java| 10 ++ 4 files changed, 18 insertions(+), 3 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java index 9ece9f00135..051bad2650e 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputDeserializer.java @@ -378,7 +378,7 @@ public class DataInputDeserializer implements DataInputView, java.io.Serializabl } if (this.position >= this.end) { -return -1; +return len == 0 ? 0 : -1; } else { int toRead = Math.min(this.end - this.position, len); System.arraycopy(this.buffer, this.position, b, off, toRead); diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java index 43f34f5383f..58d5c634aed 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputView.java @@ -44,7 +44,12 @@ public interface DataInputView extends DataInput { /** * Reads up to {@code len} bytes of memory and stores it into {@code b} starting at offset - * {@code off}. It returns the number of read bytes or -1 if there is no more data left. + * {@code off}. + * + * If len is zero, then no bytes are read and 0 is returned; + * otherwise, there is an attempt to read at least one byte. If there is no more data left, the + * value -1 is returned; otherwise, at least one byte is read and stored into + * b. * * @param b byte array to store the data to * @param off offset into byte array diff --git a/flink-core/src/main/java/org/apache/flink/types/Record.java b/flink-core/src/main/java/org/apache/flink/types/Record.java index 59e1568fc8f..a7b6f989b7d 100644 --- a/flink-core/src/main/java/org/apache/flink/types/Record.java +++ b/flink-core/src/main/java/org/apache/flink/types/Record.java @@ -1584,7 +1584,7 @@ public final class Record implements Value, CopyableValue { } if (this.position >= this.end) { -return -1; +return len == 0 ? 0 : -1; } else { int toRead = Math.min(this.end - this.position, len); System.arraycopy(this.memory, this.position, b, off, toRead); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java index d8ebd5968cb..e8707f1eb70 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/DataInputDeserializerTest.java @@ -54,4 +54,14 @@ public class DataInputDeserializerTest { } Assert.assertEquals(0, dis.available()); } + +@Test +public void testReadWithLenZero() throws IOException { +byte[] bytes = new byte[0]; +DataInputDeserializer dis = new DataInputDeserializer(bytes, 0, bytes.length); +Assert.assertEquals(0, dis.available()); + +byte[] bytesForRead = new byte[0]; +Assert.assertEquals(0, dis.read(bytesForRead, 0, 0)); // do not throw when read with len 0 +} }
svn commit: r60428 - /release/flink/KEYS
Author: gaoyunhaii Date: Fri Mar 3 07:51:20 2023 New Revision: 60428 Log: Add keys for Qingsheng Ren Modified: release/flink/KEYS Modified: release/flink/KEYS == --- release/flink/KEYS (original) +++ release/flink/KEYS Fri Mar 3 07:51:20 2023 @@ -3110,3 +3110,63 @@ qc2bR9xFuJZ441z1PRIRLThyUzD7dZdqr4+aVa3m Bkg2cPr95iP358z2YkTvm8W7PN0j6hUeC+0j7KuwkQg= =cLO/ -END PGP PUBLIC KEY BLOCK- + +pub rsa4096 2023-03-03 [SC] + A1BD477F79D036D2C30CA7DBCA8AEEC2F6EB040B +uid [ultimate] Qingsheng Ren (CODE SIGNING KEY) +sig 3CA8AEEC2F6EB040B 2023-03-03 Qingsheng Ren (CODE SIGNING KEY) +sub rsa4096 2023-03-03 [E] +sig CA8AEEC2F6EB040B 2023-03-03 Qingsheng Ren (CODE SIGNING KEY) + +-BEGIN PGP PUBLIC KEY BLOCK- + +mQINBGQBlckBEAC+0SVEPKeLUh5onsw7lU5+gik78prvuk7MYVso9nBcmltqxkfZ +6AwobwM1sfoPKZRmVcgCzxdqOZKrYkWj1MGTa3S3iQy4v2TzjxatKFCmWl48KfTh +6V5WyW2+jZ2xZALDCqYkAwfbIAxYFzd998sl4DKyun84AH2PbZxkHwV+c4A4Yh+e +pBN1mrAo2yODq65YmO6ybT60WkgGshTjzvNy4lxKN95nesydARIcAmmrfdB93Ei7 +B+I2bfo7O51fiuOGLe002f5BTQPWJjiqyESX9uG2cQ7QY19YeOBV7UFQbbMy/UOM +tRjJZNcHRRaMIiNBJCSRR1cTYZE28JFvKf4Impu00fSXfyXtNy6jVOWIyzreXWP6 +ZcFu6EtYuPtIzdslZgjj77vwHion0BaMyidQjuQ40zJddYqPgdofEyTa1saqT+3T +ecV0DLn0nbHWtjoHZBVNIuj7xWiTWiuy+zJwlptGkzZOU/BFG/uJJJ5+01P8PbaC +RBQ2Pa4Ci7HXXAUKiag1Atxthp04WS9JMPyieYCVkXoGETNQSIQzlHeykZlS8RiP +pqIq79596Pb/E1pFjPZKEedsmCfoRabHp/ttaTbLqyd7k0g3qNO4nW0m+LVIs8e3 +VF0M33hRgCMUZBqJHLTlRmKQMaU8NyDIv1j76LVt+kvRXW21dA2gkkyGewARAQAB +tDNRaW5nc2hlbmcgUmVuIChDT0RFIFNJR05JTkcgS0VZKSA8cmVucXNAYXBhY2hl +Lm9yZz6JAlEEEwEIADsWIQShvUd/edA20sMMp9vKiu7C9usECwUCZAGVyQIbAwUL +CQgHAgIiAgYVCgkICwIEFgIDAQIeBwIXgAAKCRDKiu7C9usEC5uQD/4smIFXCoNQ +Fkm2c41tiYzOGosc9dXT8C8YcOF77faUoJCwL5q1kSNvrz4beKZhwGS0gPY+zWNq +4RNqrhzcZdSQaC2lTPTrmKayqrfLr09zbmJBpPfwoPlADSsSQgdsBHG7WR59Y+jk +9AHkEH5BrCUBvYyp0IVqrGQ7E5IS+jsze9mNrNkQFupTUgih1wJCp/QBRCta6ueZ +OKvv7r2bVlkTnuySL8PwLcZOp/wrqKFhiDroI8nNPErsKWGph/LUgAYpFjRWzXdk +rd5R3aTT/jal6NVJOJ6VUzfh8rfH9TKVz7yZHIpm8wnmxjIbZmQYjs65nX1kVCFK +XLu4Om6NRrsy+fk/5bzMVw1iNIGlsqAL2BxRJtJJGFAx8pRshSkjpLoGBzXbbgUq +d9qd2vH4xzBXIrgQzxu2U2RKk+w4WbZ/UQWg3Avl0XOTKHXGbtTMarNPiQAGs5LN +MlrhqVweTsjxzIhg+U7j0HOGJKb5R3253ERXXH9GCmbmRco8qNTVPu9Fk6WNbRD0 +grE7rksJ8w7Z/r7HMe6DS4lqNExOEIUiWzvqvDoLtOdb+mqjQzOJpcFsFCA99+OU +zLjB2wn2Zzf/wdDHkft9d/ivAVuklSx0uIeYQxL26YczjTZsgLyXIRNgZ7NQAyJr +20W/j+g6fJQuA15xGsb2w0faizhAARYTh7kCDQRkAZXJARAArd2Kxmm2XLwvEO9c +x0CccnJSKlmS4Y5Upx0spQkuMgvSBiBL0jgvvJ7Dx6LECnocPFA7zcw7uCDRvWtr +wgPbrDCmvxOEQzH34Qcru8EbRAb0UBHdgbjYDXqQKRbDhOXVmy5w8JC4k4CSlXC1 +h3mkzA6sYBiiyOBZNkyELGStijGZvWpUqzrb4BsMvtZx2RBusAh+tV36aP2N/uMJ +TbEgKRRpRfTS2ZdM2RlN+3pKxetPAuE4D6qHQQXWbAk6FMRNeQZ1RoW9JL48m3m0 +hZdpuujkI9saU/eobNcFu8Yy3/hdhs6JGfxrbQ8DMSZbaK5T4P05kKsK1aoeSDWv +HBgyoSIFsCimZxfGymdosQf6OwgVgyUJXtybQmkocxj+DrGXgi+jbBvfaN3GrBdp +DJL+sbhiK4WYXz0xJWXcB6p8HqHvhlgA+N1sQOYOwwehycv911LCuPJ/KwNxEEte +snKYiArdkTbZmRsY0R19oZLLnnLRUlJ8xwSx0EzMcbFPhXbYizngdkqFOBhKedcF +uPRjRiJKPtC/E6d/SlaaGZjkOh8zQgszMc1vm+kRdR53Jit7WIaONOHVUOVevVge +HGS9I6u9hoYEj9A0QPHrSHdinxk15CGiDk4Pc66J8GpYIOKB0wH0Y5Y9/mclbxPa +jZA7bfnNKH6Dn6shrosCZqbMpMMAEQEAAYkCNgQYAQgAIBYhBKG9R3950DbSwwyn +28qK7sL26wQLBQJkAZXJAhsMAAoJEMqK7sL26wQLN5AP/idv9jHxCmqVUZs6L8vc +ToKJVCT/cdYt3BTjuth9jeDD/OOlQzuiGy6sCHV/FioVJSQ47AmSTGjivfV1yawP +GeinurrB9Ow1bs+oBED3qyXiuYE4j1+eT+El8yUf+eUAmh3st/mHYWcATYyXbjxp +/eCYZ+Ab2DEgYonkdEGhmvFA3YHqMMhShvzb1C6ejZksRmu2QoCNtRug0NIXJCxJ +vezFGRZ0Pgde3nTObJ/dFA1XFLtyxDLywLjTyCbqZybR0hjjFgvioXk3g0Qm0E1x +x3Q5RxPI+HG9JGEr8vKTZV/RnKvPI5HwEG6R4MINTqm5dwGo0kz22pYK6Gd9/QRP +JQO/ZiTHAgxpLQPajQjeV42xx5W/tNB/0ZtwWKzvXZJmOcSzK79bbUgnMfaPZunw +MmDMMuKANZ7zrRVMeClUmANrDr/uV8h2rrdxZQmvtW0lNwCSWLn+vEgxc1lBQjmI +Q4cT7LF8iEZOv0vOg6nUWtlvWHZdz22WZYhcv5ioT1PclwdV5THnpqS8TkDxW0/J +w4Ky0IoazCnErrv/3D2ncU9amgi2BAr0IBFVaG3Auaqnk9MjSQLCGf1J2IB/Zxgw +6+uJTT17tte8Z5HOa1q4H78IHneBTt3Su7gHZJdEV0dy2A6UP6BtPGlJKi8RiLx9 +1ptChDC6Ruxjb9hN2mMg/32U +=1R3d +-END PGP PUBLIC KEY BLOCK-
[flink] branch release-1.16 updated: [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new 8a47420adfe [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress. 8a47420adfe is described below commit 8a47420adfe9af7f1c303874c7a32dab3229ea66 Author: Weihua Hu AuthorDate: Thu Nov 10 17:19:27 2022 +0800 [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress. If JobManager and TaskManager run in same host, the previous logic will use LOOPBACK interface for TaskManager, thus other TaskManagers in different host can not connect with the TaskManager. This closes #21422. --- .../org/apache/flink/runtime/net/ConnectionUtils.java | 19 +-- 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java index 84849c97b36..9f61caedbe0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java @@ -58,7 +58,6 @@ public class ConnectionUtils { * state failed to determine the address. */ private enum AddressDetectionState { -LOOPBACK(100), /** Connect from interface returned by InetAddress.getLocalHost(). * */ LOCAL_HOST(200), /** Detect own IP address based on the target IP address. Look for common prefix */ @@ -116,7 +115,6 @@ public class ConnectionUtils { final List strategies = Collections.unmodifiableList( Arrays.asList( -AddressDetectionState.LOOPBACK, AddressDetectionState.LOCAL_HOST, AddressDetectionState.ADDRESS, AddressDetectionState.FAST_CONNECT, @@ -227,18 +225,6 @@ public class ConnectionUtils { private static InetAddress findAddressUsingStrategy( AddressDetectionState strategy, InetSocketAddress targetAddress, boolean logging) throws IOException { -if (strategy == AddressDetectionState.LOOPBACK) { -InetAddress loopback = InetAddress.getLoopbackAddress(); - -if (tryToConnect(loopback, targetAddress, strategy.getTimeout(), logging)) { -LOG.debug( -"Using InetAddress.getLoopbackAddress() immediately for connecting address"); -return loopback; -} else { -return null; -} -} - // try LOCAL_HOST strategy independent of the network interfaces if (strategy == AddressDetectionState.LOCAL_HOST) { InetAddress localhostName; @@ -446,7 +432,7 @@ public class ConnectionUtils { } if (targetAddress != null) { -AddressDetectionState strategy = AddressDetectionState.LOOPBACK; +AddressDetectionState strategy = AddressDetectionState.LOCAL_HOST; boolean logging = elapsedTimeMillis >= startLoggingAfter.toMillis(); if (logging) { @@ -463,9 +449,6 @@ public class ConnectionUtils { // pick the next strategy switch (strategy) { -case LOOPBACK: -strategy = AddressDetectionState.LOCAL_HOST; -break; case LOCAL_HOST: strategy = AddressDetectionState.ADDRESS; break;
[flink] branch release-1.15 updated: [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.15 by this push: new 84e8806af16 [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress. 84e8806af16 is described below commit 84e8806af16f81a4295ca1f9b0c711c210884b1d Author: Weihua Hu AuthorDate: Thu Nov 10 17:19:27 2022 +0800 [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress. If JobManager and TaskManager run in same host, the previous logic will use LOOPBACK interface for TaskManager, thus other TaskManagers in different host can not connect with the TaskManager. This closes #21423. --- .../org/apache/flink/runtime/net/ConnectionUtils.java | 19 +-- 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java index 84849c97b36..9f61caedbe0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java @@ -58,7 +58,6 @@ public class ConnectionUtils { * state failed to determine the address. */ private enum AddressDetectionState { -LOOPBACK(100), /** Connect from interface returned by InetAddress.getLocalHost(). * */ LOCAL_HOST(200), /** Detect own IP address based on the target IP address. Look for common prefix */ @@ -116,7 +115,6 @@ public class ConnectionUtils { final List strategies = Collections.unmodifiableList( Arrays.asList( -AddressDetectionState.LOOPBACK, AddressDetectionState.LOCAL_HOST, AddressDetectionState.ADDRESS, AddressDetectionState.FAST_CONNECT, @@ -227,18 +225,6 @@ public class ConnectionUtils { private static InetAddress findAddressUsingStrategy( AddressDetectionState strategy, InetSocketAddress targetAddress, boolean logging) throws IOException { -if (strategy == AddressDetectionState.LOOPBACK) { -InetAddress loopback = InetAddress.getLoopbackAddress(); - -if (tryToConnect(loopback, targetAddress, strategy.getTimeout(), logging)) { -LOG.debug( -"Using InetAddress.getLoopbackAddress() immediately for connecting address"); -return loopback; -} else { -return null; -} -} - // try LOCAL_HOST strategy independent of the network interfaces if (strategy == AddressDetectionState.LOCAL_HOST) { InetAddress localhostName; @@ -446,7 +432,7 @@ public class ConnectionUtils { } if (targetAddress != null) { -AddressDetectionState strategy = AddressDetectionState.LOOPBACK; +AddressDetectionState strategy = AddressDetectionState.LOCAL_HOST; boolean logging = elapsedTimeMillis >= startLoggingAfter.toMillis(); if (logging) { @@ -463,9 +449,6 @@ public class ConnectionUtils { // pick the next strategy switch (strategy) { -case LOOPBACK: -strategy = AddressDetectionState.LOCAL_HOST; -break; case LOCAL_HOST: strategy = AddressDetectionState.ADDRESS; break;
[flink] branch master updated: [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 fc5e8bacef3 [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress. fc5e8bacef3 is described below commit fc5e8bacef34119579defca6256476482da523f9 Author: Weihua Hu AuthorDate: Thu Nov 10 17:19:27 2022 +0800 [FLINK-27341][runtime] Remove LOOPBACK from TaskManager findConnectingAddress. If JobManager and TaskManager run in same host, the previous logic will use LOOPBACK interface for TaskManager, thus other TaskManagers in different host can not connect with the TaskManager. This closes #21319. --- .../org/apache/flink/runtime/net/ConnectionUtils.java | 19 +-- 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java index 84849c97b36..9f61caedbe0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java @@ -58,7 +58,6 @@ public class ConnectionUtils { * state failed to determine the address. */ private enum AddressDetectionState { -LOOPBACK(100), /** Connect from interface returned by InetAddress.getLocalHost(). * */ LOCAL_HOST(200), /** Detect own IP address based on the target IP address. Look for common prefix */ @@ -116,7 +115,6 @@ public class ConnectionUtils { final List strategies = Collections.unmodifiableList( Arrays.asList( -AddressDetectionState.LOOPBACK, AddressDetectionState.LOCAL_HOST, AddressDetectionState.ADDRESS, AddressDetectionState.FAST_CONNECT, @@ -227,18 +225,6 @@ public class ConnectionUtils { private static InetAddress findAddressUsingStrategy( AddressDetectionState strategy, InetSocketAddress targetAddress, boolean logging) throws IOException { -if (strategy == AddressDetectionState.LOOPBACK) { -InetAddress loopback = InetAddress.getLoopbackAddress(); - -if (tryToConnect(loopback, targetAddress, strategy.getTimeout(), logging)) { -LOG.debug( -"Using InetAddress.getLoopbackAddress() immediately for connecting address"); -return loopback; -} else { -return null; -} -} - // try LOCAL_HOST strategy independent of the network interfaces if (strategy == AddressDetectionState.LOCAL_HOST) { InetAddress localhostName; @@ -446,7 +432,7 @@ public class ConnectionUtils { } if (targetAddress != null) { -AddressDetectionState strategy = AddressDetectionState.LOOPBACK; +AddressDetectionState strategy = AddressDetectionState.LOCAL_HOST; boolean logging = elapsedTimeMillis >= startLoggingAfter.toMillis(); if (logging) { @@ -463,9 +449,6 @@ public class ConnectionUtils { // pick the next strategy switch (strategy) { -case LOOPBACK: -strategy = AddressDetectionState.LOCAL_HOST; -break; case LOCAL_HOST: strategy = AddressDetectionState.ADDRESS; break;
[flink] branch release-1.15 updated (6b565055beb -> 28a877b8880)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git from 6b565055beb [FLINK-28695][network] Fix the bug of old netty client isn't closed when netty server closes channel and no input channel add 28a877b8880 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest No new revisions were added by this update. Summary of changes: .../io/network/netty/NettyBufferPoolTest.java | 58 -- 1 file changed, 44 insertions(+), 14 deletions(-)
[flink] branch release-1.16 updated: [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new d8ff321 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest d8ff321 is described below commit d8ff3216101fd31dbcdc4a725d2e7ead4113 Author: Yun Gao AuthorDate: Mon Dec 7 18:06:34 2020 +0800 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest --- .../io/network/netty/NettyBufferPoolTest.java | 58 -- 1 file changed, 44 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java index 3797d025e77..0aaa11ab64c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java @@ -18,36 +18,61 @@ package org.apache.flink.runtime.io.network.netty; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + +import org.junit.After; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** Tests for the {@link NettyBufferPool} wrapper. */ public class NettyBufferPoolTest { +private final List needReleasing = new ArrayList<>(); + +@After +public void tearDown() { +try { +// Release all of the buffers. +for (ByteBuf buf : needReleasing) { +buf.release(); +} + +// Checks in a separate loop in case we have sliced buffers. +for (ByteBuf buf : needReleasing) { +assertEquals(0, buf.refCnt()); +} +} finally { +needReleasing.clear(); +} +} + @Test public void testNoHeapAllocations() throws Exception { -NettyBufferPool nettyBufferPool = new NettyBufferPool(1); +final NettyBufferPool nettyBufferPool = new NettyBufferPool(1); // Buffers should prefer to be direct -assertTrue(nettyBufferPool.buffer().isDirect()); -assertTrue(nettyBufferPool.buffer(128).isDirect()); -assertTrue(nettyBufferPool.buffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer(128, 256)).isDirect()); // IO buffers should prefer to be direct -assertTrue(nettyBufferPool.ioBuffer().isDirect()); -assertTrue(nettyBufferPool.ioBuffer(128).isDirect()); -assertTrue(nettyBufferPool.ioBuffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer(128, 256)).isDirect()); // Currently we fakes the heap buffer allocation with direct buffers -assertTrue(nettyBufferPool.heapBuffer().isDirect()); -assertTrue(nettyBufferPool.heapBuffer(128).isDirect()); -assertTrue(nettyBufferPool.heapBuffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer(128, 256)).isDirect()); // Composite buffers allocates the corresponding type of buffers when extending its capacity - assertTrue(nettyBufferPool.compositeHeapBuffer().capacity(1024).isDirect()); - assertTrue(nettyBufferPool.compositeHeapBuffer(10).capacity(1024).isDirect()); + assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer()).capacity(1024).isDirect()); + assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer(10)).capacity(1024).isDirect()); // Is direct buffer pooled! assertTrue(nettyBufferPool.isDirectBufferPooled()); @@ -60,16 +85,21 @@ public class NettyBufferPoolTest { { // Single large buffer allocates one chunk -nettyBufferPool.directBuffer(chunkSize - 64); +releaseLater(nettyBufferPool.directBuffer(chunkSize - 64)); long allocated = nettyBufferPool.getNumberOfAllocatedBytes().get(); assertEquals(chunkSize, allocated); } { // Allocate a little more (one more chunk required) -nettyBufferPool.directBuffer(128); +releaseLater(nettyBufferPool.directBuffer(128)); long all
[flink] branch master updated: [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 88a161101bb [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest 88a161101bb is described below commit 88a161101bb33b4c088325788bd11d41f9369355 Author: Yun Gao AuthorDate: Mon Dec 7 18:06:34 2020 +0800 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest This closes #14319. --- .../io/network/netty/NettyBufferPoolTest.java | 58 -- 1 file changed, 44 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java index 3797d025e77..0aaa11ab64c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java @@ -18,36 +18,61 @@ package org.apache.flink.runtime.io.network.netty; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + +import org.junit.After; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** Tests for the {@link NettyBufferPool} wrapper. */ public class NettyBufferPoolTest { +private final List needReleasing = new ArrayList<>(); + +@After +public void tearDown() { +try { +// Release all of the buffers. +for (ByteBuf buf : needReleasing) { +buf.release(); +} + +// Checks in a separate loop in case we have sliced buffers. +for (ByteBuf buf : needReleasing) { +assertEquals(0, buf.refCnt()); +} +} finally { +needReleasing.clear(); +} +} + @Test public void testNoHeapAllocations() throws Exception { -NettyBufferPool nettyBufferPool = new NettyBufferPool(1); +final NettyBufferPool nettyBufferPool = new NettyBufferPool(1); // Buffers should prefer to be direct -assertTrue(nettyBufferPool.buffer().isDirect()); -assertTrue(nettyBufferPool.buffer(128).isDirect()); -assertTrue(nettyBufferPool.buffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer(128, 256)).isDirect()); // IO buffers should prefer to be direct -assertTrue(nettyBufferPool.ioBuffer().isDirect()); -assertTrue(nettyBufferPool.ioBuffer(128).isDirect()); -assertTrue(nettyBufferPool.ioBuffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer(128, 256)).isDirect()); // Currently we fakes the heap buffer allocation with direct buffers -assertTrue(nettyBufferPool.heapBuffer().isDirect()); -assertTrue(nettyBufferPool.heapBuffer(128).isDirect()); -assertTrue(nettyBufferPool.heapBuffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer(128, 256)).isDirect()); // Composite buffers allocates the corresponding type of buffers when extending its capacity - assertTrue(nettyBufferPool.compositeHeapBuffer().capacity(1024).isDirect()); - assertTrue(nettyBufferPool.compositeHeapBuffer(10).capacity(1024).isDirect()); + assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer()).capacity(1024).isDirect()); + assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer(10)).capacity(1024).isDirect()); // Is direct buffer pooled! assertTrue(nettyBufferPool.isDirectBufferPooled()); @@ -60,16 +85,21 @@ public class NettyBufferPoolTest { { // Single large buffer allocates one chunk -nettyBufferPool.directBuffer(chunkSize - 64); +releaseLater(nettyBufferPool.directBuffer(chunkSize - 64)); long allocated = nettyBufferPool.getNumberOfAllocatedBytes().get(); assertEquals(chunkSize, allocated); } { // Allocate a little more (one more chunk required) -nettyBufferPool.directBuffer(128); +releaseLater(nettyBufferPool.directBuffer(128)); long all
[flink] 01/01: [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.16-fix_netty_test_leak in repository https://gitbox.apache.org/repos/asf/flink.git commit 9810cf0885920922a43801f6289273896fc8612c Author: Yun Gao AuthorDate: Mon Dec 7 18:06:34 2020 +0800 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest --- .../io/network/netty/NettyBufferPoolTest.java | 58 -- 1 file changed, 44 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java index 3797d025e77..0aaa11ab64c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java @@ -18,36 +18,61 @@ package org.apache.flink.runtime.io.network.netty; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + +import org.junit.After; import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** Tests for the {@link NettyBufferPool} wrapper. */ public class NettyBufferPoolTest { +private final List needReleasing = new ArrayList<>(); + +@After +public void tearDown() { +try { +// Release all of the buffers. +for (ByteBuf buf : needReleasing) { +buf.release(); +} + +// Checks in a separate loop in case we have sliced buffers. +for (ByteBuf buf : needReleasing) { +assertEquals(0, buf.refCnt()); +} +} finally { +needReleasing.clear(); +} +} + @Test public void testNoHeapAllocations() throws Exception { -NettyBufferPool nettyBufferPool = new NettyBufferPool(1); +final NettyBufferPool nettyBufferPool = new NettyBufferPool(1); // Buffers should prefer to be direct -assertTrue(nettyBufferPool.buffer().isDirect()); -assertTrue(nettyBufferPool.buffer(128).isDirect()); -assertTrue(nettyBufferPool.buffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.buffer(128, 256)).isDirect()); // IO buffers should prefer to be direct -assertTrue(nettyBufferPool.ioBuffer().isDirect()); -assertTrue(nettyBufferPool.ioBuffer(128).isDirect()); -assertTrue(nettyBufferPool.ioBuffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.ioBuffer(128, 256)).isDirect()); // Currently we fakes the heap buffer allocation with direct buffers -assertTrue(nettyBufferPool.heapBuffer().isDirect()); -assertTrue(nettyBufferPool.heapBuffer(128).isDirect()); -assertTrue(nettyBufferPool.heapBuffer(128, 256).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer()).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer(128)).isDirect()); +assertTrue(releaseLater(nettyBufferPool.heapBuffer(128, 256)).isDirect()); // Composite buffers allocates the corresponding type of buffers when extending its capacity - assertTrue(nettyBufferPool.compositeHeapBuffer().capacity(1024).isDirect()); - assertTrue(nettyBufferPool.compositeHeapBuffer(10).capacity(1024).isDirect()); + assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer()).capacity(1024).isDirect()); + assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer(10)).capacity(1024).isDirect()); // Is direct buffer pooled! assertTrue(nettyBufferPool.isDirectBufferPooled()); @@ -60,16 +85,21 @@ public class NettyBufferPoolTest { { // Single large buffer allocates one chunk -nettyBufferPool.directBuffer(chunkSize - 64); +releaseLater(nettyBufferPool.directBuffer(chunkSize - 64)); long allocated = nettyBufferPool.getNumberOfAllocatedBytes().get(); assertEquals(chunkSize, allocated); } { // Allocate a little more (one more chunk required) -nettyBufferPool.directBuffer(128); +releaseLater(nettyBufferPool.directBuffer(128)); long allocated = nettyBufferPool.getNumberOfAllocatedBytes().get(); assertEquals(2 * chunkSize, allocated); } } + +private ByteBuf releaseLater(ByteBuf buf) { +needReleasing.a
[flink] branch 1.16-fix_netty_test_leak created (now 9810cf08859)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch 1.16-fix_netty_test_leak in repository https://gitbox.apache.org/repos/asf/flink.git at 9810cf08859 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest This branch includes the following new commits: new 9810cf08859 [FLINK-16582][runtime][test] Fix the buffer leaks in NettyBufferPoolTest The 1 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.
[flink] branch master updated (d745f5b3f7a -> a56af3b5702)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from d745f5b3f7a [FLINK-29234] JobMasterServiceLeadershipRunner#closeAsync has a lock with an excessive range add a56af3b5702 [FLINK-29557] Fix the SinkOperator with OutputFormatFunction is not recognized as InputFormatOperator No new revisions were added by this update. Summary of changes: .../api/operators/AbstractUdfStreamOperator.java | 3 +- .../api/operators/OutputFormatOperatorFactory.java | 2 +- .../api/operators/SimpleOperatorFactory.java | 12 +++-- .../SimpleOutputFormatOperatorFactory.java | 20 +++- .../api/operators/UserFunctionProvider.java| 28 ++ .../runtime/operators/sink/SinkOperatorTest.java | 59 ++ 6 files changed, 107 insertions(+), 17 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/UserFunctionProvider.java create mode 100644 flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/sink/SinkOperatorTest.java
[flink] branch master updated: [FLINK-29498][datastream] Add Scala Async Retry Strategies and ResultPredicates Helper Classes
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 d426489c9e5 [FLINK-29498][datastream] Add Scala Async Retry Strategies and ResultPredicates Helper Classes d426489c9e5 is described below commit d426489c9e5c634e2eec8fde6c71356700b7d4b2 Author: Eric Xiao AuthorDate: Sun Oct 16 18:12:27 2022 -0400 [FLINK-29498][datastream] Add Scala Async Retry Strategies and ResultPredicates Helper Classes This closes #21077. --- .../docs/dev/datastream/operators/asyncio.md | 10 +- .../docs/dev/datastream/operators/asyncio.md | 10 +- .../api/operators/async/AsyncWaitOperator.java | 2 +- .../api/scala/async/AsyncRetryStrategies.scala | 130 + .../api/scala/async/RetryPredicates.scala | 38 ++ .../api/scala/AsyncDataStreamITCase.scala | 42 ++- 6 files changed, 194 insertions(+), 38 deletions(-) diff --git a/docs/content.zh/docs/dev/datastream/operators/asyncio.md b/docs/content.zh/docs/dev/datastream/operators/asyncio.md index 6dbddab5824..0caa5e8ce3b 100644 --- a/docs/content.zh/docs/dev/datastream/operators/asyncio.md +++ b/docs/content.zh/docs/dev/datastream/operators/asyncio.md @@ -125,8 +125,8 @@ DataStream> resultStream = // 通过工具类创建一个异步重试策略, 或用户实现自定义的策略 AsyncRetryStrategy asyncRetryStrategy = new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms - .retryIfResult(RetryPredicates.EMPTY_RESULT_PREDICATE) - .retryIfException(RetryPredicates.HAS_EXCEPTION_PREDICATE) + .ifResult(RetryPredicates.EMPTY_RESULT_PREDICATE) + .ifException(RetryPredicates.HAS_EXCEPTION_PREDICATE) .build(); // 应用异步 I/O 转换操作并启用重试 @@ -170,7 +170,11 @@ val resultStream: DataStream[(String, String)] = // 或 应用异步 I/O 转换操作并启用重试 // 创建一个异步重试策略 -val asyncRetryStrategy: AsyncRetryStrategy[OUT] = ... +val asyncRetryStrategy: AsyncRetryStrategy[String] = + new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms +.ifResult(RetryPredicates.EMPTY_RESULT_PREDICATE) +.ifException(RetryPredicates.HAS_EXCEPTION_PREDICATE) +.build(); // 应用异步 I/O 转换操作并启用重试 val resultStream: DataStream[(String, String)] = diff --git a/docs/content/docs/dev/datastream/operators/asyncio.md b/docs/content/docs/dev/datastream/operators/asyncio.md index a6d218fff55..631c83eaa1b 100644 --- a/docs/content/docs/dev/datastream/operators/asyncio.md +++ b/docs/content/docs/dev/datastream/operators/asyncio.md @@ -140,8 +140,8 @@ DataStream> resultStream = // create an async retry strategy via utility class or a user defined strategy AsyncRetryStrategy asyncRetryStrategy = new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms - .retryIfResult(RetryPredicates.EMPTY_RESULT_PREDICATE) - .retryIfException(RetryPredicates.HAS_EXCEPTION_PREDICATE) + .ifResult(RetryPredicates.EMPTY_RESULT_PREDICATE) + .ifException(RetryPredicates.HAS_EXCEPTION_PREDICATE) .build(); // apply the async I/O transformation with retry @@ -185,7 +185,11 @@ val resultStream: DataStream[(String, String)] = // apply the async I/O transformation with retry // create an AsyncRetryStrategy -val asyncRetryStrategy: AsyncRetryStrategy[OUT] = ... +val asyncRetryStrategy: AsyncRetryStrategy[String] = + new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms +.ifResult(RetryPredicates.EMPTY_RESULT_PREDICATE) +.ifException(RetryPredicates.HAS_EXCEPTION_PREDICATE) +.build(); // apply the async I/O transformation with retry val resultStream: DataStream[(String, String)] = diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java index 0d88943b21e..7d2685f7828 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java @@ -430,7 +430,7 @@ public class AsyncWaitOperator /** * A guard similar to ResultHandler#complete to prevent repeated complete calls from * ill-written AsyncFunction. This flag indicates a retry is in-flight, new retry will be - * rejected if it is ture, and it will be reset to false after the retry fired. + * rejected if it is true, and it will be reset to false after the retry fired. */ private final AtomicBoolean retryAwaiting
[flink] branch release-1.16 updated: [FLINK-29348][table] DynamicFilteringEvent can be stored in the CoordinatorStore if it's received before the listener source coordinator is started.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new b30a502c53e [FLINK-29348][table] DynamicFilteringEvent can be stored in the CoordinatorStore if it's received before the listener source coordinator is started. b30a502c53e is described below commit b30a502c53eaca95630eddd03022871f17fdf299 Author: Gen Luo AuthorDate: Tue Sep 27 15:49:29 2022 +0800 [FLINK-29348][table] DynamicFilteringEvent can be stored in the CoordinatorStore if it's received before the listener source coordinator is started. This closes #20906. --- .../flink/connectors/hive/HiveDialectITCase.java | 135 .../hive/HiveDynamicPartitionPruningITCase.java| 234 + .../flink/table/catalog/hive/HiveTestUtils.java| 19 ++ .../operators/coordination/CoordinatorStore.java | 4 + .../coordination/CoordinatorStoreImpl.java | 5 + .../source/coordinator/SourceCoordinator.java | 37 +++- ...cFilteringDataCollectorOperatorCoordinator.java | 58 +++-- 7 files changed, 334 insertions(+), 158 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java index 81487871c78..32548dd731a 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.connectors.hive; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveTable; import org.apache.flink.table.HiveVersionTestUtil; import org.apache.flink.table.api.SqlDialect; @@ -26,7 +25,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -37,7 +35,6 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.table.catalog.hive.HiveTestUtils; -import org.apache.flink.table.catalog.stats.CatalogTableStatistics; import org.apache.flink.table.delegation.ExtendedOperationExecutor; import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.functions.hive.HiveGenericUDTFTest; @@ -1287,138 +1284,6 @@ public class HiveDialectITCase { } } -@Test -public void testDynamicPartitionPruning() throws Exception { -// src table -tableEnv.executeSql("create table dim (x int,y string,z int)"); -tableEnv.executeSql("insert into dim values (1,'a',1),(2,'b',1),(3,'c',2)").await(); - -// partitioned dest table -tableEnv.executeSql("create table fact (a int, b bigint, c string) partitioned by (p int)"); -tableEnv.executeSql( -"insert into fact partition (p=1) values (10,100,'aaa'),(11,101,'bbb'),(12,102,'ccc') ") -.await(); -tableEnv.executeSql( -"insert into fact partition (p=2) values (20,200,'aaa'),(21,201,'bbb'),(22,202,'ccc') ") -.await(); -tableEnv.executeSql( -"insert into fact partition (p=3) values (30,300,'aaa'),(31,301,'bbb'),(32,302,'ccc') ") -.await(); - -System.out.println( -tableEnv.explainSql( -"select a, b, c, p, x, y from fact, dim where x = p and z = 1 order by a")); - -tableEnv.getConfig().set(TaskManagerOptions.NUM_TASK_SLOTS, 4); - tableEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); - tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); - -String sql = "select a, b, c, p, x, y from fact, dim where x = p and z = 1 order by a"; -String sqlSwapFactDim = -"select a, b, c, p, x, y from dim, fact where x = p and z = 1 order by a"; - -String expected = -"[+I[10, 100, aaa, 1, 1, a], +I[11, 101, bbb, 1, 1, a], +I[12, 102, ccc, 1, 1, a], " -+ "+I[20, 200, aaa,
[flink] branch master updated: [FLINK-29348][table] DynamicFilteringEvent can be stored in the CoordinatorStore if it's received before the listener source coordinator is started.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 7bae0ebb637 [FLINK-29348][table] DynamicFilteringEvent can be stored in the CoordinatorStore if it's received before the listener source coordinator is started. 7bae0ebb637 is described below commit 7bae0ebb6379c175522bd903838bb3737fc6c65d Author: Gen Luo AuthorDate: Tue Sep 27 15:49:29 2022 +0800 [FLINK-29348][table] DynamicFilteringEvent can be stored in the CoordinatorStore if it's received before the listener source coordinator is started. This closes #20906. --- .../flink/connectors/hive/HiveDialectITCase.java | 135 .../hive/HiveDynamicPartitionPruningITCase.java| 234 + .../flink/table/catalog/hive/HiveTestUtils.java| 19 ++ .../operators/coordination/CoordinatorStore.java | 4 + .../coordination/CoordinatorStoreImpl.java | 5 + .../source/coordinator/SourceCoordinator.java | 37 +++- ...cFilteringDataCollectorOperatorCoordinator.java | 58 +++-- 7 files changed, 334 insertions(+), 158 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java index 81487871c78..32548dd731a 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.connectors.hive; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveTable; import org.apache.flink.table.HiveVersionTestUtil; import org.apache.flink.table.api.SqlDialect; @@ -26,7 +25,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -37,7 +35,6 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.table.catalog.hive.HiveTestUtils; -import org.apache.flink.table.catalog.stats.CatalogTableStatistics; import org.apache.flink.table.delegation.ExtendedOperationExecutor; import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.functions.hive.HiveGenericUDTFTest; @@ -1287,138 +1284,6 @@ public class HiveDialectITCase { } } -@Test -public void testDynamicPartitionPruning() throws Exception { -// src table -tableEnv.executeSql("create table dim (x int,y string,z int)"); -tableEnv.executeSql("insert into dim values (1,'a',1),(2,'b',1),(3,'c',2)").await(); - -// partitioned dest table -tableEnv.executeSql("create table fact (a int, b bigint, c string) partitioned by (p int)"); -tableEnv.executeSql( -"insert into fact partition (p=1) values (10,100,'aaa'),(11,101,'bbb'),(12,102,'ccc') ") -.await(); -tableEnv.executeSql( -"insert into fact partition (p=2) values (20,200,'aaa'),(21,201,'bbb'),(22,202,'ccc') ") -.await(); -tableEnv.executeSql( -"insert into fact partition (p=3) values (30,300,'aaa'),(31,301,'bbb'),(32,302,'ccc') ") -.await(); - -System.out.println( -tableEnv.explainSql( -"select a, b, c, p, x, y from fact, dim where x = p and z = 1 order by a")); - -tableEnv.getConfig().set(TaskManagerOptions.NUM_TASK_SLOTS, 4); - tableEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); - tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); - -String sql = "select a, b, c, p, x, y from fact, dim where x = p and z = 1 order by a"; -String sqlSwapFactDim = -"select a, b, c, p, x, y from dim, fact where x = p and z = 1 order by a"; - -String expected = -"[+I[10, 100, aaa, 1, 1, a], +I[11, 101, bbb, 1, 1, a], +I[12, 102, ccc, 1, 1, a], " -+ "+I[20, 200, aaa, 2, 2, b
[flink] branch release-1.16 updated: [hotfix] Show paramter for templated tests with junit5
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new d2ab2896c3e [hotfix] Show paramter for templated tests with junit5 d2ab2896c3e is described below commit d2ab2896c3e0a00d60805371d5c95dbe5dff40db Author: Yun Gao AuthorDate: Mon Sep 26 23:36:19 2022 +0800 [hotfix] Show paramter for templated tests with junit5 --- .../java/org/apache/flink/util/TestLoggerExtension.java | 13 - 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/TestLoggerExtension.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/TestLoggerExtension.java index ba5c7763c9e..82595b5a9fb 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/TestLoggerExtension.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/TestLoggerExtension.java @@ -35,30 +35,33 @@ public class TestLoggerExtension implements TestWatcher, BeforeEachCallback { public void beforeEach(ExtensionContext context) { LOG.info( "\n" -+ "\nTest {}.{} is running." ++ "\nTest {}.{}[{}] is running." + "\n", context.getRequiredTestClass().getCanonicalName(), -context.getRequiredTestMethod().getName()); +context.getRequiredTestMethod().getName(), +context.getDisplayName()); } @Override public void testSuccessful(ExtensionContext context) { LOG.info( "\n" -+ "\nTest {}.{} successfully run." ++ "\nTest {}.{}[{}] successfully run." + "\n", context.getRequiredTestClass().getCanonicalName(), -context.getRequiredTestMethod().getName()); +context.getRequiredTestMethod().getName(), +context.getDisplayName()); } @Override public void testFailed(ExtensionContext context, Throwable cause) { LOG.error( "\n" -+ "\nTest {}.{} failed with:\n{}" ++ "\nTest {}.{}[{}] failed with:\n{}" + "\n", context.getRequiredTestClass().getCanonicalName(), context.getRequiredTestMethod().getName(), +context.getDisplayName(), exceptionToString(cause)); }
[flink] branch master updated (54f81c4a9d1 -> f360a8740aa)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 54f81c4a9d1 [FLINK-14896][connectors/kinesis] Shade and relocate Jackson dependencies add f360a8740aa [hotfix] Show paramter for templated tests with junit5 No new revisions were added by this update. Summary of changes: .../java/org/apache/flink/util/TestLoggerExtension.java | 13 - 1 file changed, 8 insertions(+), 5 deletions(-)
[flink] branch release-1.16 updated: [hotfix] Make ParquetProtoWriters.ParquetProtoWriterBuilder public to support customized scenarios
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new c963ba6589b [hotfix] Make ParquetProtoWriters.ParquetProtoWriterBuilder public to support customized scenarios c963ba6589b is described below commit c963ba6589bf8fb547dd97c5f47d3d881d5cf46b Author: Jin AuthorDate: Tue Aug 30 18:34:30 2022 -0700 [hotfix] Make ParquetProtoWriters.ParquetProtoWriterBuilder public to support customized scenarios This closes #20786. --- .../apache/flink/formats/parquet/protobuf/ParquetProtoWriters.java| 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/protobuf/ParquetProtoWriters.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/protobuf/ParquetProtoWriters.java index 022b2a6e63e..698b32b14ea 100644 --- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/protobuf/ParquetProtoWriters.java +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/protobuf/ParquetProtoWriters.java @@ -45,12 +45,12 @@ public class ParquetProtoWriters { // /** The builder for Protobuf {@link ParquetWriter}. */ -private static class ParquetProtoWriterBuilder +public static class ParquetProtoWriterBuilder extends ParquetWriter.Builder> { private final Class clazz; -protected ParquetProtoWriterBuilder(OutputFile outputFile, Class clazz) { +public ParquetProtoWriterBuilder(OutputFile outputFile, Class clazz) { super(outputFile); this.clazz = clazz; }
[flink] branch master updated (05600f844a9 -> aab13977bea)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 05600f844a9 [FLINK-29325][docs] Fix documentation bug on how to enable batch mode for streaming examples add aab13977bea [hotfix] Make ParquetProtoWriters.ParquetProtoWriterBuilder public to support customized scenarios No new revisions were added by this update. Summary of changes: .../apache/flink/formats/parquet/protobuf/ParquetProtoWriters.java| 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-)
[flink] branch release-1.16 updated: [FLINK-29132][rest] Cleanup subtask attempt metrics according to the JobDetails to avoid memory leak.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.16 by this push: new 4dad2e29a09 [FLINK-29132][rest] Cleanup subtask attempt metrics according to the JobDetails to avoid memory leak. 4dad2e29a09 is described below commit 4dad2e29a090c731a0474a80e320264040c348ce Author: Gen Luo AuthorDate: Thu Sep 1 17:08:53 2022 +0800 [FLINK-29132][rest] Cleanup subtask attempt metrics according to the JobDetails to avoid memory leak. This closes #20733. --- .../runtime/messages/webmonitor/JobDetails.java| 96 - .../handler/job/JobVertexBackPressureHandler.java | 34 +++--- .../rest/handler/legacy/metrics/MetricStore.java | 49 +++-- .../messages/webmonitor/JobDetailsTest.java| 31 -- .../job/JobVertexBackPressureHandlerTest.java | 12 +-- .../handler/legacy/metrics/MetricStoreTest.java| 114 - 6 files changed, 198 insertions(+), 138 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java index 74b2964228c..04fce9483c0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; @@ -40,8 +41,11 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.S import java.io.IOException; import java.io.Serializable; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -60,8 +64,6 @@ public class JobDetails implements Serializable { private static final String FIELD_NAME_STATUS = "state"; private static final String FIELD_NAME_LAST_MODIFICATION = "last-modification"; private static final String FIELD_NAME_TOTAL_NUMBER_TASKS = "total"; -private static final String FIELD_NAME_CURRENT_EXECUTION_ATTEMPTS = -"current-execution-attempts"; private final JobID jobId; @@ -84,10 +86,12 @@ public class JobDetails implements Serializable { /** * The map holds the attempt number of the current execution attempt in the Execution, which is * considered as the representing execution for the subtask of the vertex. The keys and values - * are JobVertexID -> SubtaskIndex -> CurrentExecutionAttemptNumber. It is used to accumulate - * the metrics of a subtask in MetricFetcher. + * are JobVertexID -> SubtaskIndex -> CurrenAttempts info. + * + * The field is excluded from the json. Any usage from the web UI and the history server is + * not allowed. */ -private final Map> currentExecutionAttempts; +private final Map> currentExecutionAttempts; @VisibleForTesting public JobDetails( @@ -123,7 +127,7 @@ public class JobDetails implements Serializable { long lastUpdateTime, int[] tasksPerState, int numTasks, -Map> currentExecutionAttempts) { +Map> currentExecutionAttempts) { this.jobId = checkNotNull(jobId); this.jobName = checkNotNull(jobName); this.startTime = startTime; @@ -150,22 +154,25 @@ public class JobDetails implements Serializable { int[] countsPerStatus = new int[ExecutionState.values().length]; long lastChanged = 0; int numTotalTasks = 0; -Map> currentExecutionAttempts = new HashMap<>(); +Map> currentExecutionAttempts = new HashMap<>(); for (AccessExecutionJobVertex ejv : job.getVerticesTopologically()) { AccessExecutionVertex[] taskVertices = ejv.getTaskVertices(); numTotalTasks += taskVertices.length; -Map vertexAttempts = new HashMap<>(); +Map vertexAttempts = new HashMap<>(); for (AccessExecutionVertex taskVertex : taskVertices) { -if (taskVertex.getCurrentExecutions().size() > 1) { -
[flink] branch master updated (bd1d97391be -> 26eeabfdd1f)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from bd1d97391be [FLINK-29184][sql-gateway] Close resource manager when closing Session add 26eeabfdd1f [FLINK-29132][rest] Cleanup subtask attempt metrics according to the JobDetails to avoid memory leak. No new revisions were added by this update. Summary of changes: .../runtime/messages/webmonitor/JobDetails.java| 96 - .../handler/job/JobVertexBackPressureHandler.java | 34 +++--- .../rest/handler/legacy/metrics/MetricStore.java | 49 +++-- .../messages/webmonitor/JobDetailsTest.java| 31 -- .../job/JobVertexBackPressureHandlerTest.java | 12 +-- .../handler/legacy/metrics/MetricStoreTest.java| 114 - 6 files changed, 198 insertions(+), 138 deletions(-)
[flink] 03/03: [FLINK-28860][runtime] JobMaster wait for partition promote before close
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git commit f599a8c444ab44660824a7b3e0a08a635c22d3f4 Author: sxnan AuthorDate: Tue Aug 9 21:07:17 2022 +0800 [FLINK-28860][runtime] JobMaster wait for partition promote before close --- .../partition/JobMasterPartitionTracker.java | 27 ++- .../partition/JobMasterPartitionTrackerImpl.java | 89 +++- .../TaskExecutorPartitionTrackerImpl.java | 10 + .../apache/flink/runtime/jobmaster/JobMaster.java | 46 ++-- .../runtime/jobmaster/RpcTaskManagerGateway.java | 3 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 33 ++- .../runtime/taskexecutor/TaskExecutorGateway.java | 20 +- .../TaskExecutorGatewayDecoratorBase.java | 13 +- .../JobMasterPartitionTrackerImplTest.java | 239 + .../partition/NoOpJobMasterPartitionTracker.java | 7 +- .../TestingJobMasterPartitionTracker.java | 31 +-- .../jobmaster/JobMasterPartitionReleaseTest.java | 19 +- .../TaskExecutorPartitionLifecycleTest.java| 28 +-- .../taskexecutor/TestingTaskExecutorGateway.java | 27 +-- .../TestingTaskExecutorGatewayBuilder.java | 30 ++- 15 files changed, 378 insertions(+), 244 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java index 2c116e4951e..fa6886f5076 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java @@ -25,6 +25,8 @@ import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import java.util.Collection; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; /** * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. @@ -57,15 +59,32 @@ public interface JobMasterPartitionTracker Collection resultPartitionIds, boolean releaseOnShuffleMaster); /** - * Releases the job partitions and promotes the cluster partitions, and stops the tracking of - * partitions that were released/promoted. + * Promotes the given partitions, and stops the tracking of partitions that were promoted. + * + * @param resultPartitionIds ID of the partition containing both job partitions and cluster + * partitions. + * @return Future that will be completed if the partitions are promoted. */ -void stopTrackingAndReleaseOrPromotePartitions( +CompletableFuture stopTrackingAndPromotePartitions( Collection resultPartitionIds); -/** Get all the partitions under tracking. */ +/** Gets all the partitions under tracking. */ Collection getAllTrackedPartitions(); +/** Gets all the non-cluster partitions under tracking. */ +default Collection getAllTrackedNonClusterPartitions() { +return getAllTrackedPartitions().stream() +.filter(descriptor -> !descriptor.getPartitionType().isPersistent()) +.collect(Collectors.toList()); +} + +/** Gets all the cluster partitions under tracking. */ +default Collection getAllTrackedClusterPartitions() { +return getAllTrackedPartitions().stream() +.filter(descriptor -> descriptor.getPartitionType().isPersistent()) +.collect(Collectors.toList()); +} + void connectToResourceManager(ResourceManagerGateway resourceManagerGateway); /** Get the shuffle descriptors of the cluster partitions ordered by partition number. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java index 8c8069f9760..a908581b403 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java @@ -21,17 +21,22 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleMaster; +
[flink] 02/03: [hotfix][datastream] Fix cache invalidate with remote session cluster
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git commit 7d13918512362c127588200e3d1d21076f6a23a7 Author: sxnan AuthorDate: Wed Aug 17 13:39:45 2022 +0800 [hotfix][datastream] Fix cache invalidate with remote session cluster --- .../client/deployment/executors/AbstractSessionClusterExecutor.java | 2 +- .../runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java index 52c58e5d035..9397e506ccf 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java @@ -135,7 +135,7 @@ public class AbstractSessionClusterExecutor< final ClusterClient clusterClient = clusterClientProvider.getClusterClient(); return clusterClient .invalidateClusterDataset(new IntermediateDataSetID(clusterDatasetId)) -.thenCompose(acknowledge -> null); +.thenApply(acknowledge -> null); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java index e273d1f5af9..80d9f2c9bb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java @@ -85,7 +85,7 @@ public class ClusterDataSetDeleteHandlers resourceManagerGatewayRetriever); return resourceManagerGateway .releaseClusterPartitions(clusterPartitionId) -.thenApply(ignored -> null); +.thenApply(ignored -> new SerializableVoid()); } @Override
[flink] 01/03: [FLINK-28860][datastream] Cache consumption in stream mode recompute result in case of cache miss
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git commit 4ada9199f19220ebc4b72eb2227ffa0a8ac8842a Author: sxnan AuthorDate: Tue Aug 9 18:08:19 2022 +0800 [FLINK-28860][datastream] Cache consumption in stream mode recompute result in case of cache miss --- .../translators/CacheTransformationTranslator.java | 6 +- .../api/graph/StreamGraphGeneratorTest.java| 17 --- .../flink/test/streaming/runtime/CacheITCase.java | 141 - 3 files changed, 54 insertions(+), 110 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java index c55998ab7fd..cfd958d5ec9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java @@ -72,8 +72,10 @@ public class CacheTransformationTranslator> inputs = transformation.getInputs(); +Preconditions.checkState( +inputs.size() == 1, "There could be only one transformation input to cache"); +return context.getStreamNodeIds(inputs.get(0)); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java index 3e52c036f7a..13495a0d933 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java @@ -907,23 +907,6 @@ public class StreamGraphGeneratorTest extends TestLogger { }); } -@Test -public void testCacheInStreamModeThrowsException() { -final TestingStreamExecutionEnvironment env = new TestingStreamExecutionEnvironment(); -env.setRuntimeMode(RuntimeExecutionMode.STREAMING); - -DataStream source = env.fromElements(1, 2, 3); -final int upstreamParallelism = 3; -CachedDataStream cachedStream = -source.keyBy(i -> i) -.reduce(Integer::sum) -.setParallelism(upstreamParallelism) -.cache(); -cachedStream.print(); - - Assertions.assertThatThrownBy(env::getStreamGraph).isInstanceOf(RuntimeException.class); -} - @Test public void testCacheTransformation() { final TestingStreamExecutionEnvironment env = new TestingStreamExecutionEnvironment(); 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 935c715aabd..9daffc20b35 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 @@ -31,6 +31,7 @@ import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.scheduler.ClusterDatasetCorruptedException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.CachedDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -40,15 +41,12 @@ import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.util.AbstractID; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Collector; 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; @@ -66,9 +64,7 @@ import java.util.UUID; import static org.assertj.core.api.Assertions.assertThat; /** Test datastream cache. */ -@Disabled public class CacheITCase extends AbstractTestBase { - private StreamExecutionEnvironment env; private MiniClusterWithClientResource miniClusterWithClientResource; @@ -110,17 +106,8 @@ public class Cac
[flink] branch release-1.16 updated (38088230bb5 -> f599a8c444a)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch release-1.16 in repository https://gitbox.apache.org/repos/asf/flink.git from 38088230bb5 [FLINK-29096][table] Keep backward compatibility of JdbcCatalog constructor new 4ada9199f19 [FLINK-28860][datastream] Cache consumption in stream mode recompute result in case of cache miss new 7d139185123 [hotfix][datastream] Fix cache invalidate with remote session cluster new f599a8c444a [FLINK-28860][runtime] JobMaster wait for partition promote before close 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: .../executors/AbstractSessionClusterExecutor.java | 2 +- .../partition/JobMasterPartitionTracker.java | 27 ++- .../partition/JobMasterPartitionTrackerImpl.java | 89 +++- .../TaskExecutorPartitionTrackerImpl.java | 10 + .../apache/flink/runtime/jobmaster/JobMaster.java | 46 ++-- .../runtime/jobmaster/RpcTaskManagerGateway.java | 3 +- .../dataset/ClusterDataSetDeleteHandlers.java | 2 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 33 ++- .../runtime/taskexecutor/TaskExecutorGateway.java | 20 +- .../TaskExecutorGatewayDecoratorBase.java | 13 +- .../JobMasterPartitionTrackerImplTest.java | 239 + .../partition/NoOpJobMasterPartitionTracker.java | 7 +- .../TestingJobMasterPartitionTracker.java | 31 +-- .../jobmaster/JobMasterPartitionReleaseTest.java | 19 +- .../TaskExecutorPartitionLifecycleTest.java| 28 +-- .../taskexecutor/TestingTaskExecutorGateway.java | 27 +-- .../TestingTaskExecutorGatewayBuilder.java | 30 ++- .../translators/CacheTransformationTranslator.java | 6 +- .../api/graph/StreamGraphGeneratorTest.java| 17 -- .../flink/test/streaming/runtime/CacheITCase.java | 141 +--- 20 files changed, 434 insertions(+), 356 deletions(-)
[flink] 03/03: [FLINK-28860][runtime] JobMaster wait for partition promote before close
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit b7dd42617a46fcecfffbea3409391e204a40b9b1 Author: sxnan AuthorDate: Tue Aug 9 21:07:17 2022 +0800 [FLINK-28860][runtime] JobMaster wait for partition promote before close --- .../partition/JobMasterPartitionTracker.java | 27 ++- .../partition/JobMasterPartitionTrackerImpl.java | 89 +++- .../TaskExecutorPartitionTrackerImpl.java | 10 + .../apache/flink/runtime/jobmaster/JobMaster.java | 46 ++-- .../runtime/jobmaster/RpcTaskManagerGateway.java | 3 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 33 ++- .../runtime/taskexecutor/TaskExecutorGateway.java | 20 +- .../TaskExecutorGatewayDecoratorBase.java | 13 +- .../JobMasterPartitionTrackerImplTest.java | 239 + .../partition/NoOpJobMasterPartitionTracker.java | 7 +- .../TestingJobMasterPartitionTracker.java | 31 +-- .../jobmaster/JobMasterPartitionReleaseTest.java | 19 +- .../TaskExecutorPartitionLifecycleTest.java| 28 +-- .../taskexecutor/TestingTaskExecutorGateway.java | 27 +-- .../TestingTaskExecutorGatewayBuilder.java | 30 ++- 15 files changed, 378 insertions(+), 244 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java index 2c116e4951e..fa6886f5076 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java @@ -25,6 +25,8 @@ import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import java.util.Collection; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; /** * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. @@ -57,15 +59,32 @@ public interface JobMasterPartitionTracker Collection resultPartitionIds, boolean releaseOnShuffleMaster); /** - * Releases the job partitions and promotes the cluster partitions, and stops the tracking of - * partitions that were released/promoted. + * Promotes the given partitions, and stops the tracking of partitions that were promoted. + * + * @param resultPartitionIds ID of the partition containing both job partitions and cluster + * partitions. + * @return Future that will be completed if the partitions are promoted. */ -void stopTrackingAndReleaseOrPromotePartitions( +CompletableFuture stopTrackingAndPromotePartitions( Collection resultPartitionIds); -/** Get all the partitions under tracking. */ +/** Gets all the partitions under tracking. */ Collection getAllTrackedPartitions(); +/** Gets all the non-cluster partitions under tracking. */ +default Collection getAllTrackedNonClusterPartitions() { +return getAllTrackedPartitions().stream() +.filter(descriptor -> !descriptor.getPartitionType().isPersistent()) +.collect(Collectors.toList()); +} + +/** Gets all the cluster partitions under tracking. */ +default Collection getAllTrackedClusterPartitions() { +return getAllTrackedPartitions().stream() +.filter(descriptor -> descriptor.getPartitionType().isPersistent()) +.collect(Collectors.toList()); +} + void connectToResourceManager(ResourceManagerGateway resourceManagerGateway); /** Get the shuffle descriptors of the cluster partitions ordered by partition number. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java index 8c8069f9760..a908581b403 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java @@ -21,17 +21,22 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleMaster; +
[flink] branch master updated (442ab0ce6cb -> b7dd42617a4)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 442ab0ce6cb [FLINK-29210][Docs][SQL Client] Copy English Docker documentation to Chinese documentation to bring them back in sync new a901911266b [FLINK-28860][datastream] Cache consumption in stream mode recompute result in case of cache miss new 3d05f275225 [hotfix][datastream] Fix cache invalidate with remote session cluster new b7dd42617a4 [FLINK-28860][runtime] JobMaster wait for partition promote before close 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: .../executors/AbstractSessionClusterExecutor.java | 2 +- .../partition/JobMasterPartitionTracker.java | 27 ++- .../partition/JobMasterPartitionTrackerImpl.java | 89 +++- .../TaskExecutorPartitionTrackerImpl.java | 10 + .../apache/flink/runtime/jobmaster/JobMaster.java | 46 ++-- .../runtime/jobmaster/RpcTaskManagerGateway.java | 3 +- .../dataset/ClusterDataSetDeleteHandlers.java | 2 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 33 ++- .../runtime/taskexecutor/TaskExecutorGateway.java | 20 +- .../TaskExecutorGatewayDecoratorBase.java | 13 +- .../JobMasterPartitionTrackerImplTest.java | 239 + .../partition/NoOpJobMasterPartitionTracker.java | 7 +- .../TestingJobMasterPartitionTracker.java | 31 +-- .../jobmaster/JobMasterPartitionReleaseTest.java | 19 +- .../TaskExecutorPartitionLifecycleTest.java| 28 +-- .../taskexecutor/TestingTaskExecutorGateway.java | 27 +-- .../TestingTaskExecutorGatewayBuilder.java | 30 ++- .../translators/CacheTransformationTranslator.java | 6 +- .../api/graph/StreamGraphGeneratorTest.java| 17 -- .../flink/test/streaming/runtime/CacheITCase.java | 141 +--- 20 files changed, 434 insertions(+), 356 deletions(-)
[flink] 02/03: [hotfix][datastream] Fix cache invalidate with remote session cluster
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 3d05f275225022984695cf2a14e3800580c3fea2 Author: sxnan AuthorDate: Wed Aug 17 13:39:45 2022 +0800 [hotfix][datastream] Fix cache invalidate with remote session cluster --- .../client/deployment/executors/AbstractSessionClusterExecutor.java | 2 +- .../runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java index 52c58e5d035..9397e506ccf 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java @@ -135,7 +135,7 @@ public class AbstractSessionClusterExecutor< final ClusterClient clusterClient = clusterClientProvider.getClusterClient(); return clusterClient .invalidateClusterDataset(new IntermediateDataSetID(clusterDatasetId)) -.thenCompose(acknowledge -> null); +.thenApply(acknowledge -> null); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java index e273d1f5af9..80d9f2c9bb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/dataset/ClusterDataSetDeleteHandlers.java @@ -85,7 +85,7 @@ public class ClusterDataSetDeleteHandlers resourceManagerGatewayRetriever); return resourceManagerGateway .releaseClusterPartitions(clusterPartitionId) -.thenApply(ignored -> null); +.thenApply(ignored -> new SerializableVoid()); } @Override
[flink] 01/03: [FLINK-28860][datastream] Cache consumption in stream mode recompute result in case of cache miss
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit a901911266bc1e4fde3c1b4e81a52e60c4469a50 Author: sxnan AuthorDate: Tue Aug 9 18:08:19 2022 +0800 [FLINK-28860][datastream] Cache consumption in stream mode recompute result in case of cache miss --- .../translators/CacheTransformationTranslator.java | 6 +- .../api/graph/StreamGraphGeneratorTest.java| 17 --- .../flink/test/streaming/runtime/CacheITCase.java | 141 - 3 files changed, 54 insertions(+), 110 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java index c55998ab7fd..cfd958d5ec9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/CacheTransformationTranslator.java @@ -72,8 +72,10 @@ public class CacheTransformationTranslator> inputs = transformation.getInputs(); +Preconditions.checkState( +inputs.size() == 1, "There could be only one transformation input to cache"); +return context.getStreamNodeIds(inputs.get(0)); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java index 3e52c036f7a..13495a0d933 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java @@ -907,23 +907,6 @@ public class StreamGraphGeneratorTest extends TestLogger { }); } -@Test -public void testCacheInStreamModeThrowsException() { -final TestingStreamExecutionEnvironment env = new TestingStreamExecutionEnvironment(); -env.setRuntimeMode(RuntimeExecutionMode.STREAMING); - -DataStream source = env.fromElements(1, 2, 3); -final int upstreamParallelism = 3; -CachedDataStream cachedStream = -source.keyBy(i -> i) -.reduce(Integer::sum) -.setParallelism(upstreamParallelism) -.cache(); -cachedStream.print(); - - Assertions.assertThatThrownBy(env::getStreamGraph).isInstanceOf(RuntimeException.class); -} - @Test public void testCacheTransformation() { final TestingStreamExecutionEnvironment env = new TestingStreamExecutionEnvironment(); 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 935c715aabd..9daffc20b35 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 @@ -31,6 +31,7 @@ import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.scheduler.ClusterDatasetCorruptedException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.CachedDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -40,15 +41,12 @@ import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.util.AbstractID; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Collector; 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; @@ -66,9 +64,7 @@ import java.util.UUID; import static org.assertj.core.api.Assertions.assertThat; /** Test datastream cache. */ -@Disabled public class CacheITCase extends AbstractTestBase { - private StreamExecutionEnvironment env; private MiniClusterWithClientResource miniClusterWithClientResource; @@ -110,17 +106,8 @@ public class CacheITCase extends
[flink] branch master updated (481ed78bec4 -> a8ca381c577)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 481ed78bec4 [FLINK-29096][table] Keep backward compatibility of JdbcCatalog constructor add a8ca381c577 [FLINK-14101][jdbc-connector] Support SQLServer dialect in the jdbc connector. No new revisions were added by this update. Summary of changes: docs/content.zh/docs/connectors/table/jdbc.md | 35 docs/content/docs/connectors/table/jdbc.md | 53 +++-- flink-connectors/flink-connector-jdbc/pom.xml | 16 ++ .../SqlServerDialect.java} | 148 +++-- .../SqlServerDialectFactory.java} | 10 +- ...owConverter.java => SqlServerRowConverter.java} | 19 +- ...flink.connector.jdbc.dialect.JdbcDialectFactory | 1 + .../sqlserver/SqlServerPreparedStatementTest.java | 94 + .../SqlServerTableSinkITCase.java} | 94 ++--- .../sqlserver/SqlServerTableSourceITCase.java | 229 + 10 files changed, 577 insertions(+), 122 deletions(-) copy flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/{oracle/OracleDialect.java => sqlserver/SqlServerDialect.java} (50%) copy flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/{mysql/MySqlDialectFactory.java => sqlserver/SqlServerDialectFactory.java} (80%) copy flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/{DerbyRowConverter.java => SqlServerRowConverter.java} (69%) create mode 100644 flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerPreparedStatementTest.java copy flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/{oracle/OracleTableSinkITCase.java => sqlserver/SqlServerTableSinkITCase.java} (84%) create mode 100644 flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSourceITCase.java
[flink] branch master updated (5c41e7bc0a9 -> 9955c85a53d)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 5c41e7bc0a9 [FLINK-27718][hive] Fix fail to count mutiple fields excpetion in Hive dialect (#19406) add 9955c85a53d [FLINK-28659][flink-java][JUnit5 Migration] Migrate flink-java to use junit5. No new revisions were added by this update. Summary of changes: .../flink/hadoopcompatibility/HadoopUtilsTest.java |6 +- .../api/common/io/SequentialFormatTestBase.java| 98 +- .../flink/api/common/io/SerializedFormatTest.java | 28 +- .../CollectionExecutionAccumulatorsTest.java | 15 +- .../CollectionExecutionIterationTest.java | 58 +- ...llectionExecutionWithBroadcastVariableTest.java | 29 +- .../base/CoGroupOperatorCollectionTest.java| 50 +- .../operators/base/GroupReduceOperatorTest.java| 59 +- .../operators/base/InnerJoinOperatorBaseTest.java | 66 +- .../common/operators/base/ReduceOperatorTest.java | 32 +- .../java/ExecutorDiscoveryAndJobClientTest.java| 13 +- .../flink/api/java/MultipleInvokationsTest.java| 27 +- .../apache/flink/api/java/TypeExtractionTest.java | 12 +- .../api/java/functions/SelectByFunctionsTest.java | 207 ++- .../api/java/functions/SemanticPropUtilTest.java | 1483 +++- .../SemanticPropertiesPrecedenceTest.java | 38 +- .../SemanticPropertiesProjectionTest.java | 207 +-- .../SemanticPropertiesTranslationTest.java | 455 +++--- .../apache/flink/api/java/io/CSVReaderTest.java| 242 ++-- .../api/java/io/CollectionInputFormatTest.java | 58 +- .../flink/api/java/io/CsvInputFormatTest.java | 602 .../flink/api/java/io/CsvOutputFormatTest.java | 34 +- .../apache/flink/api/java/io/FromElementsTest.java | 21 +- .../api/java/io/PrimitiveInputFormatTest.java | 71 +- .../api/java/io/PrintingOutputFormatTest.java | 48 +- .../flink/api/java/io/RowCsvInputFormatTest.java | 467 +++--- .../flink/api/java/io/TextInputFormatTest.java | 102 +- .../api/java/io/TypeSerializerFormatTest.java | 33 +- .../api/java/operator/AggregateOperatorTest.java | 55 +- .../api/java/operator/CoGroupOperatorTest.java | 360 +++-- .../flink/api/java/operator/CrossOperatorTest.java | 153 +- .../flink/api/java/operator/DataSinkTest.java | 147 +- .../api/java/operator/DistinctOperatorTest.java| 83 +- .../api/java/operator/FirstNOperatorTest.java | 111 +- .../java/operator/FullOuterJoinOperatorTest.java | 77 +- .../java/operator/GroupCombineOperatorTest.java| 287 ++-- .../api/java/operator/GroupReduceOperatorTest.java | 287 ++-- .../flink/api/java/operator/GroupingTest.java | 344 ++--- .../flink/api/java/operator/JoinOperatorTest.java | 656 - .../java/operator/LeftOuterJoinOperatorTest.java | 72 +- .../flink/api/java/operator/MaxByOperatorTest.java | 102 +- .../flink/api/java/operator/MinByOperatorTest.java | 97 +- .../flink/api/java/operator/OperatorTest.java | 20 +- .../api/java/operator/PartitionOperatorTest.java | 194 ++- .../api/java/operator/ProjectionOperatorTest.java | 99 +- .../api/java/operator/ReduceOperatorTest.java | 170 +-- .../java/operator/RightOuterJoinOperatorTest.java | 70 +- .../flink/api/java/operator/SortPartitionTest.java | 161 ++- .../apache/flink/api/java/operators/NamesTest.java | 34 +- .../translation/AggregateTranslationTest.java | 26 +- .../translation/BulkIterationTranslationTest.java | 18 +- .../translation/CoGroupSortTranslationTest.java| 67 +- .../translation/DeltaIterationTranslationTest.java | 69 +- .../translation/DistinctTranslationTest.java | 115 +- .../translation/ReduceTranslationTests.java| 76 +- .../translation/UnionTranslationTest.java | 42 +- .../flink/api/java/sampling/RandomSamplerTest.java | 183 +-- .../aggregation/AggregateCombineHarness.java |2 +- .../aggregation/BooleanSummaryAggregatorTest.java | 63 +- .../BooleanValueSummaryAggregatorTest.java | 12 +- .../summarize/aggregation/CompensatedSumTest.java | 32 +- .../aggregation/DoubleSummaryAggregatorTest.java | 204 +-- .../DoubleValueSummaryAggregatorTest.java | 17 +- .../aggregation/FloatSummaryAggregatorTest.java| 201 ++- .../FloatValueSummaryAggregatorTest.java | 17 +- .../aggregation/IntegerSummaryAggregatorTest.java | 148 +- .../IntegerValueSummaryAggregatorTest.java | 43 +- .../aggregation/LongSummaryAggregatorTest.java | 158 +-- .../LongValueSummaryAggregatorTest.java| 43 +- .../aggregation/ShortSummaryAggregatorTest.java| 150 +- .../ShortValueSummaryAggregatorTest.java | 43 +- .../aggregation/StringSummaryAggregatorTest.java | 74
[flink] branch master updated: [FLINK-28857][docs] Add Document for DataStream Cache API
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 21712c932cd [FLINK-28857][docs] Add Document for DataStream Cache API 21712c932cd is described below commit 21712c932cdbcc6be26f46a201412646c56f9650 Author: sxnan AuthorDate: Mon Aug 8 15:21:14 2022 +0800 [FLINK-28857][docs] Add Document for DataStream Cache API This closes #20491. --- .../docs/dev/datastream/operators/overview.md | 36 .../docs/dev/datastream/operators/overview.md | 38 ++ 2 files changed, 74 insertions(+) diff --git a/docs/content.zh/docs/dev/datastream/operators/overview.md b/docs/content.zh/docs/dev/datastream/operators/overview.md index e0da3f5d911..d16f89e2923 100644 --- a/docs/content.zh/docs/dev/datastream/operators/overview.md +++ b/docs/content.zh/docs/dev/datastream/operators/overview.md @@ -572,6 +572,42 @@ Python 中尚不支持此特性。 {{< /tab >}} {{< /tabs>}} +### Cache + DataStream CachedDataStream + +把算子的结果缓存起来。目前只支持批执行模式下运行的作业。算子的结果在算子第一次执行的时候会被缓存起来,之后的 +作业中会复用该算子缓存的结果。如果算子的结果丢失了,它会被原来的算子重新计算并缓存。 + +{{< tabs cache >}} +{{< tab "Java" >}} +```java +DataStream dataStream = //... +CachedDataStream cachedDataStream = dataStream.cache(); +cachedDataStream.print(); // Do anything with the cachedDataStream +... +env.execute(); // Execute and create cache. + +cachedDataStream.print(); // Consume cached result. +env.execute(); +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +val dataStream : DataStream[Int] = //... +val cachedDataStream = dataStream.cache() +cachedDataStream.print() // Do anything with the cachedDataStream +... +env.execute() // Execute and create cache. + +cachedDataStream.print() // Consume cached result. +env.execute() +``` +{{< /tab >}} +{{< tab "Python" >}} +Python 中尚不支持此特性。 +{{< /tab >}} +{{< /tabs>}} + ## 物理分区 Flink 也提供以下方法让用户根据需要在数据转换完成后对数据分区进行更细粒度的配置。 diff --git a/docs/content/docs/dev/datastream/operators/overview.md b/docs/content/docs/dev/datastream/operators/overview.md index 81e183aa1b7..a396a5af923 100644 --- a/docs/content/docs/dev/datastream/operators/overview.md +++ b/docs/content/docs/dev/datastream/operators/overview.md @@ -575,6 +575,44 @@ This feature is not yet supported in Python {{< /tab >}} {{< /tabs>}} +### Cache + DataStream CachedDataStream + +Cache the intermediate result of the transformation. Currently, only jobs that run with batch +execution mode are supported. The cache intermediate result is generated lazily at the first time +the intermediate result is computed so that the result can be reused by later jobs. If the cache is +lost, it will be recomputed using the original transformations. + +{{< tabs cache >}} +{{< tab "Java" >}} +```java +DataStream dataStream = //... +CachedDataStream cachedDataStream = dataStream.cache(); +cachedDataStream.print(); // Do anything with the cachedDataStream +... +env.execute(); // Execute and create cache. + +cachedDataStream.print(); // Consume cached result. +env.execute(); +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +val dataStream : DataStream[Int] = //... +val cachedDataStream = dataStream.cache() +cachedDataStream.print() // Do anything with the cachedDataStream +... +env.execute() // Execute and create cache. + +cachedDataStream.print() // Consume cached result. +env.execute() +``` +{{< /tab >}} +{{< tab "Python" >}} +This feature is not yet supported in Python +{{< /tab >}} +{{< /tabs>}} + ## Physical Partitioning Flink also gives low-level control (if desired) on the exact stream partitioning after a transformation, via the following functions.
[flink] branch master updated (64b430b0738 -> 1719ff85b8e)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 64b430b0738 [hotfix][python] Refactor thread mode Python runtime to make it more readable add 1719ff85b8e [FLINK-28871][table-planner] Force the output edges of dynamic filtering data collector to be BLOCKING No new revisions were added by this update. Summary of changes: .../nodes/exec/batch/BatchExecTableSourceScan.java | 43 +- .../DynamicFilteringDependencyProcessor.java | 120 +++ .../plan/batch/sql/DynamicFilteringTest.java | 28 + .../runtime/batch/sql/DynamicFilteringITCase.java | 57 +- .../plan/batch/sql/DynamicFilteringTest.xml| 920 ++--- 5 files changed, 1026 insertions(+), 142 deletions(-)
[flink] branch master updated: [FLINK-28860][tests] Disable CacheITCase temporarily
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 208f08b406a [FLINK-28860][tests] Disable CacheITCase temporarily 208f08b406a is described below commit 208f08b406a7fd48890cda16d317a30ee892a2e7 Author: sxnan AuthorDate: Tue Aug 9 16:41:50 2022 +0800 [FLINK-28860][tests] Disable CacheITCase temporarily This closes #20512. --- .../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 b3cd7298a56..935c715aabd 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; @@ -65,6 +66,7 @@ import java.util.UUID; import static org.assertj.core.api.Assertions.assertThat; /** Test datastream cache. */ +@Disabled public class CacheITCase extends AbstractTestBase { private StreamExecutionEnvironment env;
[flink] 05/06: [FLINK-28588][rest] MetricStore supports to store and query metrics of multiple execution attempts of a subtask.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 1a48fd53bd317ac2102adb00c1209350b57a687e Author: Gen Luo AuthorDate: Tue Jul 26 16:54:10 2022 +0800 [FLINK-28588][rest] MetricStore supports to store and query metrics of multiple execution attempts of a subtask. --- .../history/HistoryServerArchiveFetcher.java | 3 +- .../runtime/messages/webmonitor/JobDetails.java| 98 ++- .../metrics/dump/MetricDumpSerialization.java | 10 +- .../flink/runtime/metrics/dump/QueryScopeInfo.java | 28 +++- .../groups/InternalOperatorMetricGroup.java| 1 + .../runtime/metrics/groups/TaskMetricGroup.java| 5 +- .../handler/legacy/metrics/MetricFetcherImpl.java | 1 + .../rest/handler/legacy/metrics/MetricStore.java | 186 + .../rest/handler/util/MutableIOMetrics.java| 7 +- .../messages/webmonitor/JobDetailsTest.java| 31 .../metrics/dump/MetricDumpSerializerTest.java | 5 +- .../runtime/metrics/dump/QueryScopeInfoTest.java | 10 +- .../job/JobVertexBackPressureHandlerTest.java | 7 +- .../AggregatingSubtasksMetricsHandlerTest.java | 6 +- .../job/metrics/JobVertexMetricsHandlerTest.java | 4 +- .../job/metrics/SubtaskMetricsHandlerTest.java | 4 +- .../handler/legacy/metrics/MetricFetcherTest.java | 5 +- .../handler/legacy/metrics/MetricStoreTest.java| 61 ++- 18 files changed, 408 insertions(+), 64 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java index eb5a34b2c2d..8e5aee9c633 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java @@ -425,7 +425,8 @@ class HistoryServerArchiveFetcher { state, lastMod, tasksPerState, -numTasks); +numTasks, +new HashMap<>()); MultipleJobsDetails multipleJobsDetails = new MultipleJobsDetails(Collections.singleton(jobDetails)); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java index f9b609ac1cf..74b2964228c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/JobDetails.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.messages.webmonitor; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.execution.ExecutionState; @@ -39,6 +40,8 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.S import java.io.IOException; import java.io.Serializable; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -57,6 +60,8 @@ public class JobDetails implements Serializable { private static final String FIELD_NAME_STATUS = "state"; private static final String FIELD_NAME_LAST_MODIFICATION = "last-modification"; private static final String FIELD_NAME_TOTAL_NUMBER_TASKS = "total"; +private static final String FIELD_NAME_CURRENT_EXECUTION_ATTEMPTS = +"current-execution-attempts"; private final JobID jobId; @@ -76,6 +81,15 @@ public class JobDetails implements Serializable { private final int numTasks; +/** + * The map holds the attempt number of the current execution attempt in the Execution, which is + * considered as the representing execution for the subtask of the vertex. The keys and values + * are JobVertexID -> SubtaskIndex -> CurrentExecutionAttemptNumber. It is used to accumulate + * the metrics of a subtask in MetricFetcher. + */ +private final Map> currentExecutionAttempts; + +@VisibleForTesting public JobDetails( JobID jobId, String jobName, @@ -86,7 +100,30 @@ public class JobDetails implements Serializable { long lastUpdateTime, int[] tasksPerState, int numTasks) { +this( +jobId, +jobName, +startTime, +endTime, +duration, +status, +
[flink] 03/06: [FLINK-28588][rest] Add blocked task manager count and blocked slot count in ResourceOverview and ClusterOverview
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 0263b55288be7b569f56dd42a94c5e48bcc1607b Author: Gen Luo AuthorDate: Tue Jul 26 16:16:45 2022 +0800 [FLINK-28588][rest] Add blocked task manager count and blocked slot count in ResourceOverview and ClusterOverview --- .../src/test/resources/rest_api_v1.snapshot| 6 ++ .../messages/webmonitor/ClusterOverview.java | 52 +--- .../runtime/resourcemanager/ResourceManager.java | 26 +- .../runtime/resourcemanager/ResourceOverview.java | 18 +++- .../messages/ClusterOverviewWithVersion.java | 26 +++--- .../resourcemanager/ResourceManagerTest.java | 96 ++ .../utils/TestingResourceManagerGateway.java | 2 +- .../messages/ClusterOverviewWithVersionTest.java | 2 +- .../runtime/webmonitor/TestingRestfulGateway.java | 2 +- 9 files changed, 196 insertions(+), 34 deletions(-) diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index 427f59c2723..873e5062d7d 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -3225,6 +3225,12 @@ "slots-available" : { "type" : "integer" }, +"taskmanagers-blocked" : { + "type" : "integer" +}, +"slots-free-and-blocked" : { + "type" : "integer" +}, "jobs-running" : { "type" : "integer" }, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ClusterOverview.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ClusterOverview.java index 34bf05d0711..7ea327c8029 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ClusterOverview.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ClusterOverview.java @@ -21,8 +21,12 @@ package org.apache.flink.runtime.messages.webmonitor; import org.apache.flink.runtime.resourcemanager.ResourceOverview; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude.Include; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; +import javax.annotation.Nullable; + /** * Response to the {@link RequestStatusOverview} message, carrying a description of the Flink * cluster status. @@ -34,6 +38,8 @@ public class ClusterOverview extends JobsOverview { public static final String FIELD_NAME_TASKMANAGERS = "taskmanagers"; public static final String FIELD_NAME_SLOTS_TOTAL = "slots-total"; public static final String FIELD_NAME_SLOTS_AVAILABLE = "slots-available"; +public static final String FIELD_NAME_TASKMANAGERS_BLOCKED = "taskmanagers-blocked"; +public static final String FIELD_NAME_SLOTS_FREE_AND_BLOCKED = "slots-free-and-blocked"; @JsonProperty(FIELD_NAME_TASKMANAGERS) private final int numTaskManagersConnected; @@ -44,11 +50,24 @@ public class ClusterOverview extends JobsOverview { @JsonProperty(FIELD_NAME_SLOTS_AVAILABLE) private final int numSlotsAvailable; +@JsonProperty(FIELD_NAME_TASKMANAGERS_BLOCKED) +@JsonInclude(Include.NON_DEFAULT) +private final int numTaskManagersBlocked; + +@JsonProperty(FIELD_NAME_SLOTS_FREE_AND_BLOCKED) +@JsonInclude(Include.NON_DEFAULT) +private final int numSlotsFreeAndBlocked; + @JsonCreator +// numTaskManagersBlocked and numSlotsFreeAndBlocked is Nullable since Jackson will assign null +// if the field is absent while parsing public ClusterOverview( @JsonProperty(FIELD_NAME_TASKMANAGERS) int numTaskManagersConnected, @JsonProperty(FIELD_NAME_SLOTS_TOTAL) int numSlotsTotal, @JsonProperty(FIELD_NAME_SLOTS_AVAILABLE) int numSlotsAvailable, +@JsonProperty(FIELD_NAME_TASKMANAGERS_BLOCKED) @Nullable Integer numTaskManagersBlocked, +@JsonProperty(FIELD_NAME_SLOTS_FREE_AND_BLOCKED) @Nullable +Integer numSlotsFreeAndBlocked, @JsonProperty(FIELD_NAME_JOBS_RUNNING) int numJobsRunningOrPending, @JsonProperty(FIELD_NAME_JOBS_FINISHED) int numJobsFinished, @JsonProperty(FIELD_NAME_JOBS_CANCELLED) int numJobsCancelled, @@ -59,18 +78,8 @@ public class ClusterOverview extends JobsOverview { this.numTaskManagersConnected = numTaskManagersConnected;
[flink] 02/06: [FLINK-28588][rest] Add blocked flag in TaskManagerInfo and TaskManagerDetailsInfo
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 53938680abe846e1947243809b4b43d5f67610d3 Author: Gen Luo AuthorDate: Tue Jul 26 16:06:10 2022 +0800 [FLINK-28588][rest] Add blocked flag in TaskManagerInfo and TaskManagerDetailsInfo --- .../src/test/resources/rest_api_v1.snapshot| 6 .../runtime/resourcemanager/ResourceManager.java | 7 +++-- .../taskmanager/TaskManagerDetailsInfo.java| 7 - .../rest/messages/taskmanager/TaskManagerInfo.java | 32 ++ .../taskmanager/TaskManagerDetailsHandlerTest.java | 3 +- .../messages/taskmanager/TaskManagerInfoTest.java | 3 +- 6 files changed, 48 insertions(+), 10 deletions(-) diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index f0e069087af..427f59c2723 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -3439,6 +3439,9 @@ "type" : "integer" } } + }, + "blocked" : { +"type" : "boolean" } } } @@ -3597,6 +3600,9 @@ } } }, +"blocked" : { + "type" : "boolean" +}, "allocatedSlots" : { "type" : "array", "items" : { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 9a9a17b4cbc..fb85cf88231 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -646,7 +646,8 @@ public abstract class ResourceManager slotManager.getRegisteredResourceOf(taskExecutor.getInstanceID()), slotManager.getFreeResourceOf(taskExecutor.getInstanceID()), taskExecutor.getHardwareDescription(), -taskExecutor.getMemoryConfiguration())); +taskExecutor.getMemoryConfiguration(), + blocklistHandler.isBlockedTaskManager(taskExecutor.getResourceID(; } return CompletableFuture.completedFuture(taskManagerInfos); @@ -675,7 +676,9 @@ public abstract class ResourceManager slotManager.getRegisteredResourceOf(instanceId), slotManager.getFreeResourceOf(instanceId), taskExecutor.getHardwareDescription(), -taskExecutor.getMemoryConfiguration()), +taskExecutor.getMemoryConfiguration(), +blocklistHandler.isBlockedTaskManager( +taskExecutor.getResourceID())), slotManager.getAllocatedSlotsOf(instanceId)); return CompletableFuture.completedFuture(taskManagerInfoWithSlots); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java index 657dfe7af8c..c9df8325e10 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsInfo.java @@ -33,6 +33,8 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgn import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.Objects; @@ -64,6 +66,7 @@ public class TaskManagerDetailsInfo extends TaskManagerInfo { @JsonProperty(FIELD_NAME_AVAILABLE_RESOURCE) ResourceProfileInfo freeResource, @JsonProperty(FIELD_NAME_HARDWARE) HardwareDescription hardwareDescription, @JsonProperty(FIELD_NAME_MEMORY) TaskExecutorMemoryConfiguration memoryConfiguration, +@JsonProperty(FIELD_NAME_BLOCKED) @Nullable Boolean blocked, @JsonProperty(FIELD_NAME_ALLOCATED_SLOTS) Collection allocatedSlots, @JsonProperty(FIELD_NAME_METRICS) TaskManagerMetricsI
[flink] 06/06: [FLINK-28588][rest] Acquire information of all current executions in REST handlers if applicable
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit f436b20429b55ada2a9e8936a5e80fc672a397de Author: Gen Luo AuthorDate: Sun Jul 31 22:11:46 2022 +0800 [FLINK-28588][rest] Acquire information of all current executions in REST handlers if applicable This closes #20296. --- .../src/test/resources/rest_api_v1.snapshot| 31 +++ .../handler/job/AbstractSubtaskAttemptHandler.java | 15 +- .../rest/handler/job/JobDetailsHandler.java| 2 + .../rest/handler/job/JobExceptionsHandler.java | 33 ++-- .../handler/job/JobVertexBackPressureHandler.java | 87 +++-- .../rest/handler/job/JobVertexDetailsHandler.java | 17 +- .../handler/job/JobVertexTaskManagersHandler.java | 73 +--- .../job/SubtaskCurrentAttemptDetailsHandler.java | 19 +- ...SubtaskExecutionAttemptAccumulatorsHandler.java | 38 ++-- .../job/SubtaskExecutionAttemptDetailsHandler.java | 52 +++--- .../job/SubtasksAllAccumulatorsHandler.java| 32 ++-- .../rest/handler/job/SubtasksTimesHandler.java | 3 +- .../rest/messages/JobVertexBackPressureInfo.java | 45 - .../job/SubtaskExecutionAttemptDetailsInfo.java| 105 +++ .../threadinfo/JobVertexThreadInfoTracker.java | 24 +-- .../job/JobVertexBackPressureHandlerTest.java | 207 + .../SubtaskCurrentAttemptDetailsHandlerTest.java | 3 +- .../SubtaskExecutionAttemptDetailsHandlerTest.java | 3 +- .../messages/AggregatedTaskDetailsInfoTest.java| 3 +- .../messages/JobVertexBackPressureInfoTest.java| 24 ++- .../rest/messages/JobVertexDetailsInfoTest.java| 24 ++- .../SubtaskExecutionAttemptDetailsInfoTest.java| 3 +- 22 files changed, 651 insertions(+), 192 deletions(-) diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index 873e5062d7d..85337fa8795 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -2403,6 +2403,13 @@ "type" : "integer" } }, + "other-concurrent-attempts" : { +"type" : "array", +"items" : { + "type" : "object", + "$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:SubtaskExecutionAttemptDetailsInfo" +} + }, "start_time" : { "type" : "integer" } @@ -2522,6 +2529,9 @@ "subtask" : { "type" : "integer" }, + "attempt-number" : { +"type" : "integer" + }, "backpressure-level" : { "type" : "string", "enum" : [ "ok", "low", "high" ] @@ -2534,6 +2544,13 @@ }, "busyRatio" : { "type" : "number" + }, + "other-concurrent-attempts" : { +"type" : "array", +"items" : { + "type" : "object", + "$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobVertexBackPressureInfo:SubtaskBackPressureInfo" +} } } } @@ -2803,6 +2820,13 @@ "type" : "integer" } }, +"other-concurrent-attempts" : { + "type" : "array", + "items" : { +"type" : "object", +"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:SubtaskExecutionAttemptDetailsInfo" + } +}, "start_time" : { "type" : "integer" } @@ -2904,6 +2928,13 @@ "type" : "integer" } }, +"other-concurrent-attempts" : { + "type" : "array", + "items" : { +"type" : "object", +"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:SubtaskExecutionAttemptDetailsInfo" + } +}, "start_time" : { "type" : "integer" } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler
[flink] 01/06: [hotfix][runtime][tests] Migrates some tests to Junit5
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 247d4263c1a484ba12ce5c7826ff938b83285d11 Author: Gen Luo AuthorDate: Tue Jul 26 15:56:13 2022 +0800 [hotfix][runtime][tests] Migrates some tests to Junit5 --- .../executiongraph/ArchivedExecutionGraphTest.java | 356 + .../ArchivedExecutionGraphTestUtils.java | 170 ++ .../messages/webmonitor/JobDetailsTest.java| 15 +- .../metrics/dump/MetricDumpSerializerTest.java | 135 .../runtime/metrics/dump/QueryScopeInfoTest.java | 168 +- .../job/JobVertexBackPressureHandlerTest.java | 67 ++-- .../handler/legacy/metrics/MetricStoreTest.java| 50 +-- .../taskmanager/TaskManagerDetailsHandlerTest.java | 18 +- 8 files changed, 475 insertions(+), 504 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java index b6e9cefc2b4..4427c5b3042 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.core.testutils.CommonTestUtils; -import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStatsSummarySnapshot; @@ -42,44 +41,34 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.testutils.TestingUtils; -import org.apache.flink.testutils.executor.TestExecutorResource; -import org.apache.flink.util.OptionalFailure; -import org.apache.flink.util.SerializedValue; -import org.apache.flink.util.TestLogger; +import org.apache.flink.testutils.executor.TestExecutorExtension; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import java.io.IOException; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; import static java.util.Arrays.asList; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link ArchivedExecutionGraph}. */ -public class ArchivedExecutionGraphTest extends TestLogger { +public class ArchivedExecutionGraphTest { -@ClassRule -public static final TestExecutorResource EXECUTOR_RESOURCE = -TestingUtils.defaultExecutorResource(); +@RegisterExtension +static final TestExecutorExtension EXECUTOR_RESOURCE = +TestingUtils.defaultExecutorExtension(); private static ExecutionGraph runtimeGraph; -@BeforeClass -public static void setupExecutionGraph() throws Exception { +@BeforeAll +static void setupExecutionGraph() throws Exception { // - // Setup // - @@ -146,21 +135,21 @@ public class ArchivedExecutionGraphTest extends TestLogger { } @Test -public void testArchive() throws IOException, ClassNotFoundException { +void testArchive() throws IOException, ClassNotFoundException { ArchivedExecutionGraph archivedGraph = ArchivedExecutionGraph.createFrom(runtimeGraph); compareExecutionGraph(runtimeGraph, archivedGraph); } @Test -public void testSerialization() throws IOException, ClassNotFoundException { +void testSerialization() throws IOException, ClassNotFoundException { ArchivedExecutionGraph archivedGraph = ArchivedExecutionGraph.createFrom(runtimeGraph
[flink] 04/06: [FLINK-28588][rest] Archive all current executions in ArchivedExecutionVertex.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 44c00dbb4a083ff197442e2ce6440558be252787 Author: Gen Luo AuthorDate: Tue Jul 26 16:39:40 2022 +0800 [FLINK-28588][rest] Archive all current executions in ArchivedExecutionVertex. --- .../executiongraph/AccessExecutionVertex.java | 9 + .../executiongraph/ArchivedExecutionVertex.java| 26 ++- .../ArchivedSpeculativeExecutionVertex.java| 52 - .../executiongraph/SpeculativeExecutionVertex.java | 4 +- .../ArchivedExecutionGraphTestUtils.java | 21 ++ ...xecutionVertexWithSpeculativeExecutionTest.java | 223 + 6 files changed, 280 insertions(+), 55 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionVertex.java index 6775424ba4b..f8d4581c7e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionVertex.java @@ -21,6 +21,7 @@ package org.apache.flink.runtime.executiongraph; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import java.util.Collection; import java.util.Optional; /** Common interface for the runtime {@link ExecutionVertex} and {@link ArchivedExecutionVertex}. */ @@ -46,6 +47,14 @@ public interface AccessExecutionVertex { */ AccessExecution getCurrentExecutionAttempt(); +/** + * Returns the current executions for this execution vertex. The returned collection must + * contain the current execution attempt. + * + * @return current executions + */ + Collection getCurrentExecutions(); + /** * Returns the current {@link ExecutionState} for this execution vertex. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionVertex.java index e5feba5445a..d9f8448a702 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionVertex.java @@ -18,10 +18,14 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.Optional; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -40,15 +44,29 @@ public class ArchivedExecutionVertex implements AccessExecutionVertex, Serializa private final ArchivedExecution currentExecution; // this field must never be null +private final Collection currentExecutions; + // public ArchivedExecutionVertex(ExecutionVertex vertex) { this.subTaskIndex = vertex.getParallelSubtaskIndex(); this.executionHistory = getCopyOfExecutionHistory(vertex); this.taskNameWithSubtask = vertex.getTaskNameWithSubtaskIndex(); -this.currentExecution = vertex.getCurrentExecutionAttempt().archive(); + +Execution vertexCurrentExecution = vertex.getCurrentExecutionAttempt(); +ArrayList currentExecutionList = +new ArrayList<>(vertex.getCurrentExecutions().size()); +currentExecution = vertexCurrentExecution.archive(); +currentExecutionList.add(currentExecution); +for (Execution execution : vertex.getCurrentExecutions()) { +if (execution != vertexCurrentExecution) { +currentExecutionList.add(execution.archive()); +} +} +currentExecutions = Collections.unmodifiableList(currentExecutionList); } +@VisibleForTesting public ArchivedExecutionVertex( int subTaskIndex, String taskNameWithSubtask, @@ -58,6 +76,7 @@ public class ArchivedExecutionVertex implements AccessExecutionVertex, Serializa this.taskNameWithSubtask = checkNotNull(taskNameWithSubtask); this.currentExecution = checkNotNull(currentExecution); this.executionHistory = checkNotNull(executionHistory); +this.currentExecutions = Collections.singletonList(currentExecution); } // @@ -79,6 +98,11 @@ public class ArchivedExecutionVertex impl
[flink] branch master updated (aae0462cb39 -> f436b20429b)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from aae0462cb39 [FLINK-24787][docs] Add more details of state latency tracking documentation new 247d4263c1a [hotfix][runtime][tests] Migrates some tests to Junit5 new 53938680abe [FLINK-28588][rest] Add blocked flag in TaskManagerInfo and TaskManagerDetailsInfo new 0263b55288b [FLINK-28588][rest] Add blocked task manager count and blocked slot count in ResourceOverview and ClusterOverview new 44c00dbb4a0 [FLINK-28588][rest] Archive all current executions in ArchivedExecutionVertex. new 1a48fd53bd3 [FLINK-28588][rest] MetricStore supports to store and query metrics of multiple execution attempts of a subtask. new f436b20429b [FLINK-28588][rest] Acquire information of all current executions in REST handlers if applicable The 6 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: .../history/HistoryServerArchiveFetcher.java | 3 +- .../src/test/resources/rest_api_v1.snapshot| 43 +++ .../executiongraph/AccessExecutionVertex.java | 9 + .../executiongraph/ArchivedExecutionVertex.java| 26 +- .../ArchivedSpeculativeExecutionVertex.java| 52 --- .../executiongraph/SpeculativeExecutionVertex.java | 4 +- .../messages/webmonitor/ClusterOverview.java | 52 ++- .../runtime/messages/webmonitor/JobDetails.java| 98 +- .../metrics/dump/MetricDumpSerialization.java | 10 +- .../flink/runtime/metrics/dump/QueryScopeInfo.java | 28 +- .../groups/InternalOperatorMetricGroup.java| 1 + .../runtime/metrics/groups/TaskMetricGroup.java| 5 +- .../runtime/resourcemanager/ResourceManager.java | 33 +- .../runtime/resourcemanager/ResourceOverview.java | 18 +- .../handler/job/AbstractSubtaskAttemptHandler.java | 15 +- .../rest/handler/job/JobDetailsHandler.java| 2 + .../rest/handler/job/JobExceptionsHandler.java | 33 +- .../handler/job/JobVertexBackPressureHandler.java | 87 - .../rest/handler/job/JobVertexDetailsHandler.java | 17 +- .../handler/job/JobVertexTaskManagersHandler.java | 73 +++-- .../job/SubtaskCurrentAttemptDetailsHandler.java | 19 +- ...SubtaskExecutionAttemptAccumulatorsHandler.java | 38 +-- .../job/SubtaskExecutionAttemptDetailsHandler.java | 52 +-- .../job/SubtasksAllAccumulatorsHandler.java| 32 +- .../rest/handler/job/SubtasksTimesHandler.java | 3 +- .../messages/ClusterOverviewWithVersion.java | 26 +- .../handler/legacy/metrics/MetricFetcherImpl.java | 1 + .../rest/handler/legacy/metrics/MetricStore.java | 186 +-- .../rest/handler/util/MutableIOMetrics.java| 7 +- .../rest/messages/JobVertexBackPressureInfo.java | 45 ++- .../job/SubtaskExecutionAttemptDetailsInfo.java| 105 +++--- .../taskmanager/TaskManagerDetailsInfo.java| 7 +- .../rest/messages/taskmanager/TaskManagerInfo.java | 32 +- .../threadinfo/JobVertexThreadInfoTracker.java | 24 +- .../executiongraph/ArchivedExecutionGraphTest.java | 356 + .../ArchivedExecutionGraphTestUtils.java | 191 +++ ...ecutionVertexWithSpeculativeExecutionTest.java} | 185 +++ .../messages/webmonitor/JobDetailsTest.java| 46 ++- .../metrics/dump/MetricDumpSerializerTest.java | 140 .../runtime/metrics/dump/QueryScopeInfoTest.java | 182 +-- .../resourcemanager/ResourceManagerTest.java | 96 ++ .../utils/TestingResourceManagerGateway.java | 2 +- .../job/JobVertexBackPressureHandlerTest.java | 279 +--- .../SubtaskCurrentAttemptDetailsHandlerTest.java | 3 +- .../SubtaskExecutionAttemptDetailsHandlerTest.java | 3 +- .../AggregatingSubtasksMetricsHandlerTest.java | 6 +- .../job/metrics/JobVertexMetricsHandlerTest.java | 4 +- .../job/metrics/SubtaskMetricsHandlerTest.java | 4 +- .../messages/ClusterOverviewWithVersionTest.java | 2 +- .../handler/legacy/metrics/MetricFetcherTest.java | 5 +- .../handler/legacy/metrics/MetricStoreTest.java| 111 +-- .../taskmanager/TaskManagerDetailsHandlerTest.java | 21 +- .../messages/AggregatedTaskDetailsInfoTest.java| 3 +- .../messages/JobVertexBackPressureInfoTest.java| 24 +- .../rest/messages/JobVertexDetailsInfoTest.java| 24 +- .../SubtaskExecutionAttemptDetailsInfoTest.java| 3 +- .../messages/taskmanager/TaskManagerInfoTest.java | 3 +- .../runtime/webmonitor/TestingRestfulGateway.java | 2 +- 58 files changed, 1879 insertions(+), 1002 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/ru
[flink] branch master updated: [FLINK-27878][datastream] Add Retry Support For Async I/O In DataStream API
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 e85cf8c4cdf [FLINK-27878][datastream] Add Retry Support For Async I/O In DataStream API e85cf8c4cdf is described below commit e85cf8c4cdf417b47f8d53bf3bb202f79e92b205 Author: lincoln lee AuthorDate: Fri Apr 29 14:56:23 2022 +0800 [FLINK-27878][datastream] Add Retry Support For Async I/O In DataStream API This closes #19983. --- .../docs/dev/datastream/operators/asyncio.md | 59 ++- .../docs/dev/datastream/operators/asyncio.md | 72 +++- .../streaming/api/datastream/AsyncDataStream.java | 176 - .../api/functions/async/AsyncRetryPredicate.java | 47 +++ .../api/functions/async/AsyncRetryStrategy.java| 37 ++ .../api/operators/async/AsyncWaitOperator.java | 227 +++- .../operators/async/AsyncWaitOperatorFactory.java | 15 + .../util/retryable/AsyncRetryStrategies.java | 240 + .../streaming/util/retryable/RetryPredicates.java | 85 + .../api/operators/async/AsyncWaitOperatorTest.java | 240 - .../streaming/api/scala/AsyncDataStream.scala | 397 - .../api/scala/async/AsyncRetryPredicate.scala | 47 +++ .../api/scala/async/AsyncRetryStrategy.scala | 34 ++ .../api/scala/AsyncDataStreamITCase.scala | 122 ++- 14 files changed, 1755 insertions(+), 43 deletions(-) diff --git a/docs/content.zh/docs/dev/datastream/operators/asyncio.md b/docs/content.zh/docs/dev/datastream/operators/asyncio.md index d1054879bef..6dbddab5824 100644 --- a/docs/content.zh/docs/dev/datastream/operators/asyncio.md +++ b/docs/content.zh/docs/dev/datastream/operators/asyncio.md @@ -30,6 +30,8 @@ under the License. 对于不熟悉异步或者事件驱动编程的用户,建议先储备一些关于 Future 和事件驱动编程的知识。 提示:这篇文档 [FLIP-12: 异步 I/O 的设计和实现](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=65870673)介绍了关于设计和实现异步 I/O 功能的细节。 +对于新增的重试支持的实现细节可以参考[FLIP-232: 为 DataStream API 异步 I/O 操作增加重试支持](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211883963)。 + ## 对于异步 I/O 操作的需求 @@ -60,7 +62,7 @@ Flink 的异步 I/O API 允许用户在流处理中使用异步请求客户端 - 实现分发请求的 `AsyncFunction` - 获取数据库交互的结果并发送给 `ResultFuture` 的 *回调* 函数 -- 将异步 I/O 操作应用于 `DataStream` 作为 `DataStream` 的一次转换操作。 +- 将异步 I/O 操作应用于 `DataStream` 作为 `DataStream` 的一次转换操作, 启用或者不启用重试。 下面是基本的代码模板: @@ -115,10 +117,21 @@ class AsyncDatabaseRequest extends RichAsyncFunction stream = ...; -// 应用异步 I/O 转换操作 +// 应用异步 I/O 转换操作,不启用重试 DataStream> resultStream = AsyncDataStream.unorderedWait(stream, new AsyncDatabaseRequest(), 1000, TimeUnit.MILLISECONDS, 100); +// 或 应用异步 I/O 转换操作并启用重试 +// 通过工具类创建一个异步重试策略, 或用户实现自定义的策略 +AsyncRetryStrategy asyncRetryStrategy = + new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms + .retryIfResult(RetryPredicates.EMPTY_RESULT_PREDICATE) + .retryIfException(RetryPredicates.HAS_EXCEPTION_PREDICATE) + .build(); + +// 应用异步 I/O 转换操作并启用重试 +DataStream> resultStream = + AsyncDataStream.unorderedWaitWithRetry(stream, new AsyncDatabaseRequest(), 1000, TimeUnit.MILLISECONDS, 100, asyncRetryStrategy); ``` {{< /tab >}} {{< tab "Scala" >}} @@ -151,10 +164,17 @@ class AsyncDatabaseRequest extends AsyncFunction[String, (String, String)] { // 创建初始 DataStream val stream: DataStream[String] = ... -// 应用异步 I/O 转换操作 +// 应用异步 I/O 转换操作,不启用重试 val resultStream: DataStream[(String, String)] = AsyncDataStream.unorderedWait(stream, new AsyncDatabaseRequest(), 1000, TimeUnit.MILLISECONDS, 100) +// 或 应用异步 I/O 转换操作并启用重试 +// 创建一个异步重试策略 +val asyncRetryStrategy: AsyncRetryStrategy[OUT] = ... + +// 应用异步 I/O 转换操作并启用重试 +val resultStream: DataStream[(String, String)] = + AsyncDataStream.unorderedWaitWithRetry(stream, new AsyncDatabaseRequest(), 1000, TimeUnit.MILLISECONDS, 100, asyncRetryStrategy) ``` {{< /tab >}} {{< /tabs >}} @@ -164,11 +184,12 @@ val resultStream: DataStream[(String, String)] = 下面两个参数控制异步操作: - - **Timeout**: 超时参数定义了异步请求发出多久后未得到响应即被认定为失败。 它可以防止一直等待得不到响应的请求。 + - **Timeout**: 超时参数定义了异步操作执行多久未完成、最终认定为失败的时长,如果启用重试,则可能包括多个重试请求。 它可以防止一直等待得不到响应的请求。 - **Capacity**: 容量参数定义了可以同时进行的异步请求数。 即使异步 I/O 通常带来更高的吞吐量,执行异步 I/O 操作的算子仍然可能成为流处理的瓶颈。 限制并发请求的数量可以确保算子不会持续累积待处理的请求进而造成积压,而是在容量耗尽时触发反压。 + - **AsyncRetryStrategy**: 重试策略参数定义了什么条件会触发延迟重试以及延迟的策略,例如,固定延迟、指数后退延迟、自定义实现等。 ### 超时处理 @@ -211,6 +232,16 @@ Flink 提供两种模式控制结果记录以何种顺序发出。 异步 I/O 算子提供了完全的精确一次容错保证。它将在途的异步请求的记录保存在 checkpoint 中,在故障恢复时重新触发请求。 +### 重试支持 + +重试支持为异步 I/O 操作引入了一个内置重试机制,它对用户的异步函数实现逻辑是透明的。 + + - **AsyncRetryStrategy**: 异步重试策略包含了触发重试条件 `AsyncRetryPredicate` 定义,以及根据当前已尝试次数判断是否继续重试、下次重试间隔时长的接口方法。 + 需要注意,在满足触发重试条件后,有可能因为当前重试次数超过预设的上限放弃重试,或是在
[flink] branch master updated (21242800842 -> ed236ee19d7)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 21242800842 [FLINK-28486][docs-zh] Flink FileSystem SQL Connector Doc is not right add ed236ee19d7 [FLINK-28457][runtime] Introduce JobStatusHook to allow users to listen for the job status changes No new revisions were added by this update. Summary of changes: .../executiongraph/DefaultExecutionGraph.java | 35 +- .../DefaultExecutionGraphBuilder.java | 3 +- .../{JobStatusListener.java => JobStatusHook.java} | 41 --- .../apache/flink/runtime/jobgraph/JobGraph.java| 13 .../executiongraph/TestingJobStatusHook.java | 69 +++ .../runtime/scheduler/DefaultSchedulerTest.java| 80 ++ .../flink/streaming/api/graph/StreamGraph.java | 15 .../api/graph/StreamingJobGraphGenerator.java | 4 ++ 8 files changed, 247 insertions(+), 13 deletions(-) copy flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/{JobStatusListener.java => JobStatusHook.java} (51%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingJobStatusHook.java
[flink-web] 02/02: Rebuild website
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git commit 366692567c841794c581bd6772918960166f5425 Author: Yun Gao AuthorDate: Mon Jul 11 21:41:51 2022 +0800 Rebuild website --- content/2022/07/11/final-checkpoint-part2.html | 2 +- content/blog/feed.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/content/2022/07/11/final-checkpoint-part2.html b/content/2022/07/11/final-checkpoint-part2.html index 847bebd98..81c871ff2 100644 --- a/content/2022/07/11/final-checkpoint-part2.html +++ b/content/2022/07/11/final-checkpoint-part2.html @@ -237,7 +237,7 @@ 11 Jul 2022 Yun Gao , Dawid Wysakowicz , Daisy Tsang -In the first part of this blog, +In the first part of this blog, we have briefly introduced the work to support checkpoints after tasks get finished and revised the process of finishing. In this part we will present more details on the implementation, including how we support checkpoints with finished tasks and the revised protocol of the finish process. diff --git a/content/blog/feed.xml b/content/blog/feed.xml index 1b5e30ed6..079fbbdbc 100644 --- a/content/blog/feed.xml +++ b/content/blog/feed.xml @@ -8,7 +8,7 @@ FLIP-147: Support Checkpoints After Tasks Finished - Part Two -pIn the a href=/2022/06/01/final-checkpoint-part1.htmlfirst part/a of this blog, +pIn the a href=/2022/07/11/final-checkpoint-part1.htmlfirst part/a of this blog, we have briefly introduced the work to support checkpoints after tasks get finished and revised the process of finishing. In this part we will present more details on the implementation, including how we support checkpoints with finished tasks and the revised protocol of the finish process./p
[flink-web] 01/02: [hotfix] Fix the link in FLIP-147 blog
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git commit a044b43680eed0b94801140def048c3801064072 Author: Yun Gao AuthorDate: Mon Jul 11 21:41:34 2022 +0800 [hotfix] Fix the link in FLIP-147 blog --- _posts/2022-07-11-final-checkpoint-part2.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/_posts/2022-07-11-final-checkpoint-part2.md b/_posts/2022-07-11-final-checkpoint-part2.md index c6eda82e5..8271f8e3b 100644 --- a/_posts/2022-07-11-final-checkpoint-part2.md +++ b/_posts/2022-07-11-final-checkpoint-part2.md @@ -13,7 +13,7 @@ excerpt: This post presents more details on the changes on the checkpoint proced --- -In the [first part]({{site.baseurl}}/2022/06/01/final-checkpoint-part1.html) of this blog, +In the [first part]({{site.baseurl}}/2022/07/11/final-checkpoint-part1.html) of this blog, we have briefly introduced the work to support checkpoints after tasks get finished and revised the process of finishing. In this part we will present more details on the implementation, including how we support checkpoints with finished tasks and the revised protocol of the finish process.
[flink-web] branch asf-site updated (ec54aa1f2 -> 366692567)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git from ec54aa1f2 Rebuild website new a044b4368 [hotfix] Fix the link in FLIP-147 blog new 366692567 Rebuild website 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: _posts/2022-07-11-final-checkpoint-part2.md| 2 +- content/2022/07/11/final-checkpoint-part2.html | 2 +- content/blog/feed.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-)
[flink-web] branch asf-site updated (0223e1748 -> ec54aa1f2)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git from 0223e1748 Release Flink ML 2.1.0 (#556) new 313d98723 Add blogs for FLIP-147 support checkpoints after tasks finished new ec54aa1f2 Rebuild website 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: _posts/2022-07-11-final-checkpoint-part1.md| 176 ++ _posts/2022-07-11-final-checkpoint-part2.md| 244 .../07/11/final-checkpoint-part1.html} | 302 - content/2022/07/11/final-checkpoint-part2.html | 510 +++ content/blog/feed.xml | 695 - content/blog/index.html| 75 ++- content/blog/page10/index.html | 76 ++- content/blog/page11/index.html | 74 ++- content/blog/page12/index.html | 76 ++- content/blog/page13/index.html | 80 ++- content/blog/page14/index.html | 76 ++- content/blog/page15/index.html | 73 ++- content/blog/page16/index.html | 76 ++- content/blog/page17/index.html | 76 ++- content/blog/page18/index.html | 81 ++- content/blog/page19/index.html | 84 ++- content/blog/page2/index.html | 75 ++- content/blog/page20/index.html | 50 ++ content/blog/page3/index.html | 74 ++- content/blog/page4/index.html | 76 ++- content/blog/page5/index.html | 78 ++- content/blog/page6/index.html | 76 ++- content/blog/page7/index.html | 72 ++- content/blog/page8/index.html | 74 ++- content/blog/page9/index.html | 76 ++- .../checkpoint_format.png | Bin 0 -> 96886 bytes .../checkpoint_trigger.png | Bin 0 -> 146793 bytes .../2022-07-11-final-checkpoint/example_job.png| Bin 0 -> 26922 bytes .../example_job_finish.png | Bin 0 -> 42969 bytes .../2022-07-11-final-checkpoint/finish_cmp.png | Bin 0 -> 26074 bytes .../stream_batch_cmp.png | Bin 0 -> 44100 bytes content/index.html | 15 +- content/zh/index.html | 15 +- .../checkpoint_format.png | Bin 0 -> 96886 bytes .../checkpoint_trigger.png | Bin 0 -> 146793 bytes .../2022-07-11-final-checkpoint/example_job.png| Bin 0 -> 26922 bytes .../example_job_finish.png | Bin 0 -> 42969 bytes .../2022-07-11-final-checkpoint/finish_cmp.png | Bin 0 -> 26074 bytes .../stream_batch_cmp.png | Bin 0 -> 44100 bytes 39 files changed, 2446 insertions(+), 1009 deletions(-) create mode 100644 _posts/2022-07-11-final-checkpoint-part1.md create mode 100644 _posts/2022-07-11-final-checkpoint-part2.md copy content/{2020/08/19/statefun.html => 2022/07/11/final-checkpoint-part1.html} (50%) create mode 100644 content/2022/07/11/final-checkpoint-part2.html create mode 100644 content/img/blog/2022-07-11-final-checkpoint/checkpoint_format.png create mode 100644 content/img/blog/2022-07-11-final-checkpoint/checkpoint_trigger.png create mode 100644 content/img/blog/2022-07-11-final-checkpoint/example_job.png create mode 100644 content/img/blog/2022-07-11-final-checkpoint/example_job_finish.png create mode 100644 content/img/blog/2022-07-11-final-checkpoint/finish_cmp.png create mode 100644 content/img/blog/2022-07-11-final-checkpoint/stream_batch_cmp.png create mode 100644 img/blog/2022-07-11-final-checkpoint/checkpoint_format.png create mode 100644 img/blog/2022-07-11-final-checkpoint/checkpoint_trigger.png create mode 100644 img/blog/2022-07-11-final-checkpoint/example_job.png create mode 100644 img/blog/2022-07-11-final-checkpoint/example_job_finish.png create mode 100644 img/blog/2022-07-11-final-checkpoint/finish_cmp.png create mode 100644 img/blog/2022-07-11-final-checkpoint/stream_batch_cmp.png
[flink-web] 01/02: Add blogs for FLIP-147 support checkpoints after tasks finished
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git commit 313d98723676c865c3057145319854af3864a538 Author: Yun Gao AuthorDate: Mon Jun 6 16:01:48 2022 +0800 Add blogs for FLIP-147 support checkpoints after tasks finished --- _posts/2022-07-11-final-checkpoint-part1.md| 176 +++ _posts/2022-07-11-final-checkpoint-part2.md| 244 + .../checkpoint_format.png | Bin 0 -> 96886 bytes .../checkpoint_trigger.png | Bin 0 -> 146793 bytes .../2022-07-11-final-checkpoint/example_job.png| Bin 0 -> 26922 bytes .../example_job_finish.png | Bin 0 -> 42969 bytes .../2022-07-11-final-checkpoint/finish_cmp.png | Bin 0 -> 26074 bytes .../stream_batch_cmp.png | Bin 0 -> 44100 bytes 8 files changed, 420 insertions(+) diff --git a/_posts/2022-07-11-final-checkpoint-part1.md b/_posts/2022-07-11-final-checkpoint-part1.md new file mode 100644 index 0..5eff0740e --- /dev/null +++ b/_posts/2022-07-11-final-checkpoint-part1.md @@ -0,0 +1,176 @@ +--- +layout: post +title: "FLIP-147: Support Checkpoints After Tasks Finished - Part One" +date: 2022-07-11T00:00:00.000Z +authors: +- Yun Gao: + name: "Yun Gao" +- Dawid Wysakowicz: + name: "Dawid Wysakowicz" +- Daisy Tsang: + name: "Daisy Tsang" +excerpt: This post briefly describes the motivation and changes made by the final checkpoint mechanism, including the changes to the checkpoint procedure and how tasks finish. + +--- + +# Motivation + +Flink is a distributed processing engine for both unbounded and bounded streams of data. In recent versions, +Flink has unified the DataStream API and the Table / SQL API to support both streaming and batch cases. +Since most users require both types of data processing pipelines, the unification helps reduce the complexity of developing, +operating, and maintaining consistency between streaming and batch backfilling jobs, like +[the case for Alibaba](https://www.ververica.com/blog/apache-flinks-stream-batch-unification-powers-alibabas-11.11-in-2020). + +Flink provides two execution modes under the unified programming API: the streaming mode and the batch mode. +The streaming mode processes records incrementally based on the states, thus it supports both bounded and unbounded sources. +The batch mode works with bounded sources and usually has a better performance for bounded jobs because it executes all the +tasks in topological order and avoids random state access by pre-sorting the input records. Although batch mode is often the +preferred mode to process bounded jobs, streaming mode is also required for various reasons. For example, users may want to deal +with records containing retraction or exploit the property that data is roughly sorted by event times in streaming mode +(like the case in [Kappa+ Architecture](https://www.youtube.com/watch?t=666=4qSlsYogALo=youtu.be)). Moreover, +users often have mixed jobs involving both unbounded streams and bounded side-inputs, which also require streaming execution mode. + + + + + Figure 1. A comparison of the Streaming mode and Batch mode for the example Count operator. For streaming mode, the arrived + elements are not sorted, the operator would read / write the state corresponding to the element for computation. + For batch mode, the arrived elements are first sorted as a whole and then processed. + + + +In streaming mode, [checkpointing](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/dev/datastream/fault-tolerance/checkpointing/) +is the vital mechanism in supporting exactly-once guarantees. By periodically snapshotting the +aligned states of operators, Flink can recover from the latest checkpoint and continue execution when failover happens. However, +previously Flink could not take checkpoints if any task gets finished. This would cause problems for jobs with both bounded and unbounded +sources: if there are no checkpoints after the bounded part finished, the unbounded part might need to reprocess a large amount of +records in case of a failure. + +Furthermore, being unable to take checkpoints with finished tasks is a problem for jobs using two-phase-commit sinks to achieve +[end-to-end exactly-once processing](https://flink.apache.org/features/2018/03/01/end-to-end-exactly-once-apache-flink.html). +The two-phase-commit sinks first write data to temporary files or external transactions, +and commit the data only after a checkpoint completes to ensure the data would not be replayed on failure. However, if a job +contains bounded sources, committing the results would not be possible after the bounded sources finish. Also because of that, +for bounded jobs we have no way to commit th
svn commit: r55675 - /dev/flink/flink-ml-2.1.0-rc2/ /release/flink/flink-ml-2.1.0/
Author: gaoyunhaii Date: Mon Jul 11 06:05:53 2022 New Revision: 55675 Log: Release Flink ML 2.1.0 Added: release/flink/flink-ml-2.1.0/ - copied from r55674, dev/flink/flink-ml-2.1.0-rc2/ Removed: dev/flink/flink-ml-2.1.0-rc2/
[flink] branch release-1.15 updated: [FLINK-28240][network] Fix the bug that NetworkBufferPool#getRequestedSegmentsUsage may throw ArithmeticException.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.15 by this push: new 95f4b6fc7fd [FLINK-28240][network] Fix the bug that NetworkBufferPool#getRequestedSegmentsUsage may throw ArithmeticException. 95f4b6fc7fd is described below commit 95f4b6fc7fd4a713d8d6348d32a121498ffce1e2 Author: Gen Luo AuthorDate: Thu Jun 30 10:42:44 2022 +0800 [FLINK-28240][network] Fix the bug that NetworkBufferPool#getRequestedSegmentsUsage may throw ArithmeticException. --- .../flink/runtime/io/network/buffer/NetworkBufferPool.java | 7 +-- .../flink/runtime/io/network/buffer/NetworkBufferPoolTest.java | 9 + 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index ffdb9c727a0..1d83796e49f 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -381,8 +381,11 @@ public class NetworkBufferPool } public int getRequestedSegmentsUsage() { -return Math.toIntExact( -100L * getNumberOfRequestedMemorySegments() / getTotalNumberOfMemorySegments()); +int totalNumberOfMemorySegments = getTotalNumberOfMemorySegments(); +return totalNumberOfMemorySegments == 0 +? 0 +: Math.toIntExact( +100L * getNumberOfRequestedMemorySegments() / totalNumberOfMemorySegments); } @VisibleForTesting diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index 25a5d26c580..5a5bd186b3f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -325,6 +325,15 @@ public class NetworkBufferPoolTest extends TestLogger { } } +@Test +public void testEmptyPoolSegmentsUsage() throws IOException { +try (CloseableRegistry closeableRegistry = new CloseableRegistry()) { +NetworkBufferPool globalPool = new NetworkBufferPool(0, 128); +closeableRegistry.registerCloseable(globalPool::destroy); +assertEquals(0, globalPool.getRequestedSegmentsUsage()); +} +} + @Test public void testSegmentsUsage() throws IOException { try (CloseableRegistry closeableRegistry = new CloseableRegistry()) {
[flink] branch master updated (45ad242beb7 -> c6b3a8aa256)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 45ad242beb7 [FLINK-27854][tests] Add support for FlinkContainers to work with both existing Docker images and flink-dist builds add c6b3a8aa256 [FLINK-28240][network] Fix the bug that NetworkBufferPool#getRequestedSegmentsUsage may throw ArithmeticException. No new revisions were added by this update. Summary of changes: .../flink/runtime/io/network/buffer/NetworkBufferPool.java | 7 +-- .../flink/runtime/io/network/buffer/NetworkBufferPoolTest.java | 9 + 2 files changed, 14 insertions(+), 2 deletions(-)
[flink] branch master updated (459711bfd0e -> fcee824c964)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 459711bfd0e [FLINK-28195][python] Annotate Python3.6 as deprecated in PyFlink 1.16 add fcee824c964 [hotfix][runtime] Cleanup for the temporary fix of FLINK-13063 which is completely solved by FLINK-16219 No new revisions were added by this update. Summary of changes: .../api/operators/async/AsyncWaitOperatorTest.java | 48 +++--- 1 file changed, 5 insertions(+), 43 deletions(-)
svn commit: r55306 - /release/flink/KEYS
Author: gaoyunhaii Date: Mon Jun 27 08:10:22 2022 New Revision: 55306 Log: Add key 0789F389E67ADDFA034E603FABF0C46E59C8941C for Zhipeng Zhang Modified: release/flink/KEYS Modified: release/flink/KEYS == --- release/flink/KEYS (original) +++ release/flink/KEYS Mon Jun 27 08:10:22 2022 @@ -2852,3 +2852,62 @@ ymGoHoHg6pS9Q1BufY2TpSe4MBUc3eu2ToobYvvl 5+CggUerb/qIOgyMF/kXAfKF/6a6bA0= =eBIw -END PGP PUBLIC KEY BLOCK- +pub rsa4096 2022-06-23 [SC] + 0789F389E67ADDFA034E603FABF0C46E59C8941C +uid [ ç»å¯¹ ] Zhipeng Zhang (CODE SIGNIGNG KEY) +sig 3ABF0C46E59C8941C 2022-06-23 Zhipeng Zhang (CODE SIGNIGNG KEY) +sub rsa4096 2022-06-23 [E] +sig ABF0C46E59C8941C 2022-06-23 Zhipeng Zhang (CODE SIGNIGNG KEY) + +-BEGIN PGP PUBLIC KEY BLOCK- + +mQINBGK0EFoBEADD3Pj0XAxGdcJ2mDvG7yGFUZnRKX3bl3/CSIBUezj6rlqEtxxv +aTcX51enlDW7dT3ed3bs5ijFpfNglCMKWP4vvuLg7ndTiIWfLC4b6b+D8e0NO3+T +aMx773WGisWQ9/kzuo7kld8tImpURHQnOmyoVt3LdRqr2sUNmk2i4Jjt3rBVtbCF +QPodUwW9YwyGkU1UXIQ9FgC/5ivO5rkGTeLmh/gqBc1nKKbD/g9PGDDl8A7luLIn +ib1MEnCy3xpLWMcko8sNhWwr4BgheGygB7ZAuod2dzmoU+PWkBIbaDyiLLyrBI8f +tUOy5qtmwN4nT/6zkr/OOBqXPk9F8JK1OlaFYeisSlMngI/Bvc+eTYrY7+a8ZISj +V1bAoGtQSx/J7Uj9/FRHMIzje+U/Z2A8VPSSCYdpkRvxnQyDL6jbsobqN0CEkiVq +20laN4D33hne5j+RX7K76iNp8fpGVrhT6kSOjDR797M34xSKz9+edbu4w13ygPac +pxurXoJAmZUCqQnrgAYnnY44GpwvecWPiW840bVzhuj4aMd16PrBraIt7hkrYxGp +gmbRJI6Ni3jutwzxRBPvoN/7A0VIKWJopXjGbEoMEqCV7C+70jvTJ9JOpLMfa8VE +CPlgTRzYW91wd+ZqKQDKBf+nMTpO5/5lQyNUgaoS8yTB5QA0dt5uFQKq+wARAQAB +tDZaaGlwZW5nIFpoYW5nIChDT0RFIFNJR05JR05HIEtFWSkgPHpoYW5nenBAYXBh +Y2hlLm9yZz6JAlIEEwEIADwWIQQHifOJ5nrd+gNOYD+r8MRuWciUHAUCYrQQWgIb +AwULCQgHAgMiAgEGFQoJCAsCBBYCAwECHgcCF4AACgkQq/DEblnIlBzprRAApbv9 +tDgkuXtWHKIvhWhbBeBWNFf/bPH2aqAXDqnt5JbB+kNGq0Tz8LL7VvuLZpxaW2uP +JxmrITXhcwqMBo5U1f2w5Tjrkj6vdQ+JNuyaBJT64ofQtl0wqhN/pITjWiyxDruF +OCXdhV27Cv4hA7vAYUqFrDTuD/cw8rwqOMy2TVggCpz6IvkNB8bNf466ayvRLVd+ +n8am8DHstC/JEBIc94h9c/6odMUX6bNITUCQ0GSw86YNeKApva5RWQvpjSKcr0Gz +txYPNkywBRGOVtLP5L26jNVeNcdlXr7IQ7DUyZCVRgIhoxXkY0XgEgGSBqUUlIMy +fmIvXnfKJwtbpWE6ML47LiqIvIjPhJWNWXpWxZ2+ApwUKYj/20jjybgKyojAOTpG +MK+D5jLwJ8mkJJjsd41S968f8ZbHDlFDUFVlGUSsXOlAoqkYIs9TliVgeKyBO77C +Qxmwg+ZvlbyDygoH6wxxt3G0gjYx/M0ymrN+Lr/+7lwP/nEODt6VdRGNqmqQh4qU +sOrDzJQtwmGHj5VydDtRUcBjIupQO9hJqijRfKJqFWHHB0653UdfiELg5uXbcXSC +KtA6rg0F+n9QNCYDdkZkJGke/eWG9Yc9WQlsyXtmRgugn+Jw68ysbN0P8N4RlVs+ +7tjzM7j2/IimmFWpvjFmSq+KDhdF+16a6Mn7sVG5Ag0EYrQQWgEQAMXgc8vqoesO +/Tts9GMuNHttqYPqdDidYTHJX2aW/RoTWayqVjkU/AkksxsqzqnoFlcPxuKw3BSa +P0zCGDay8wyTnIql48TtjruCqWhllLWwpYiHPhfoMG0imXqUmVHplif7qeL8qPpX +qkaI0DxXQwuSLtOVRnXgh5IBP+lR0jrqnAESDbE3G+ROzGk5MwdzltoTxqF1ImrO +n2LwQGhJBbXj5v87ARwdUqmuJ1OXST1epIsVDsgpuV+8CdCGOmbzpNnSCPDM6L5K +TOr8UmsL/Kz4n+W615j8ZgTH2FKbLq+ZWrcBTpIokIpYAO2ROzd5uaCkNf7F36IJ +y2MY9d4nF/cQW9urfnnmP8e6ogLSc0JPGKBhE0Knx2w42xBxJIByWDpObTXIqDpB +7VwlWMJDYSAQut9xm0oHX3+fbYVDxMPcK6dWRUTjKL1bUnzLErwX4rICmn6v8gIR +HPzn3Tm2ERY9fnyJVJvR+w4KNvUnbQ8fnVrCzDUV1qX7TSckwoVZajjSHupsxziz +j1qeUCaK+VwdV+usa5RbKdjNymQqqcc3sAVUGm0g/s0kHRNlBuGZHMSuV7zg00Yx +OkAoaIQh/DapXd5vbPCjyFn+Zqe2Ejy8EoJAQC3pC6oB+r6phC0gA4PY4acJQjFN +MAgaJMvfdmKL84aDLhE7NIPOn4O+eJjVABEBAAGJAjYEGAEIACAWIQQHifOJ5nrd ++gNOYD+r8MRuWciUHAUCYrQQWgIbDAAKCRCr8MRuWciUHGUAEACrDkg7wARYZRgB +x08FsrKKtlMynWZud3MfqD1RvDjzSrfd3w4Fe8/iMt6SYGNI/TnriG0B9qoo6125 +8bzHeslI3I5PlXCTBz54jKoTofiwZOVBKWi+rjOA4VS4rLK6lfyBTDKgaGgahi4N +TxburT0qXfzfZilUtdC/0P4zj3izo7iaBYO890aASjWAnk0atSEc83idjofwxmby +GRVBsLwdbS0+Wgn7sYpG7uesZ6318zFLrV5Mo2O0ZDOG2n3xWQYUfIX47aGO+VVT +TupHTSW+7hbEt67iB//FQxaPWCd6KfOOtvKcYyYsUs+X70nj/YpknWyYnTsKHsw/ +hji2dJUyotNZNazhmH/hBzaswBUQHqtVt2ESAQr9EqJQNvr9Jf99vOFl/qZkM6X2 +Mvvf1jLSnYdMtjmewcfzznniOTIG93BE9ajQSRoTGBo5gAe9Jm8kulo21KRL4FHe +KKeUS5Q0FW/ospeDTgTR33h1cE7UBVxDup0w8Hq903y/ICOuPoyrKEMmi0LvCq86 +vDlSc5Nyo6Ur/kISXA2MqGHPI7QwVjsn0vCLN+1hFsLJN+6YIrqMoP+2APU/Alli +r+FvTBpuDvGhDEmmu6iohktStoLqT26ipIK1xCaEV3ZLWSQxVHrJA4Tb5KwXRifC +iWe0jeRflJhH0A8EjaCFTzvqsBlbHA== +=kHAa +-END PGP PUBLIC KEY BLOCK-
[flink] branch master updated: [FLINK-27523][runtime] Runtime supports producing and consuming cached intermediate results
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 a9f2e20375f [FLINK-27523][runtime] Runtime supports producing and consuming cached intermediate results a9f2e20375f is described below commit a9f2e20375f669a5f42944c3ba12a903a4624e43 Author: sxnan AuthorDate: Wed May 25 10:55:41 2022 +0800 [FLINK-27523][runtime] Runtime supports producing and consuming cached intermediate results This closes #19653. --- .../TaskDeploymentDescriptorFactory.java | 76 +- .../executiongraph/DefaultExecutionGraph.java | 7 + .../executiongraph/EdgeManagerBuildUtil.java | 6 +- .../InternalExecutionGraphAccessor.java| 8 +- .../RegionPartitionGroupReleaseStrategy.java | 3 +- .../network/partition/ClusterPartitionManager.java | 23 ++ .../io/network/partition/DataSetMetaInfo.java | 19 ++ .../partition/JobMasterPartitionTracker.java | 10 + .../partition/JobMasterPartitionTrackerImpl.java | 37 +++ .../partition/ResourceManagerPartitionTracker.java | 10 + .../ResourceManagerPartitionTrackerImpl.java | 21 +- .../partition/TaskExecutorPartitionInfo.java | 15 +- .../TaskExecutorPartitionTrackerImpl.java | 21 +- .../apache/flink/runtime/jobgraph/JobVertex.java | 29 ++- .../apache/flink/runtime/jobmaster/JobMaster.java | 1 + .../runtime/resourcemanager/ResourceManager.java | 19 ++ ...achedIntermediateDataSetCorruptedException.java | 44 .../flink/runtime/scheduler/DefaultScheduler.java | 34 ++- .../scheduler/strategy/ConsumedPartitionGroup.java | 24 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 7 +- .../partition/ClusterPartitionReport.java | 22 +- .../TaskDeploymentDescriptorFactoryTest.java | 3 +- .../JobMasterPartitionTrackerImplTest.java | 58 + .../partition/NoOpJobMasterPartitionTracker.java | 13 + .../NoOpResourceManagerPartitionTracker.java | 8 + .../ResourceManagerPartitionTrackerImplTest.java | 67 - .../TaskExecutorPartitionTrackerImplTest.java | 80 +- .../TestingJobMasterPartitionTracker.java | 13 + .../jobmaster/JobIntermediateDatasetReuseTest.java | 270 + .../ResourceManagerPartitionLifecycleTest.java | 36 ++- .../utils/TestingResourceManagerGateway.java | 15 ++ .../adapter/DefaultExecutionVertexTest.java| 4 +- .../runtime/scheduler/adaptive/ExecutingTest.java | 9 + .../strategy/TestingSchedulingExecutionVertex.java | 8 +- .../strategy/TestingSchedulingTopology.java| 3 +- .../runtime/taskexecutor/TaskExecutorTest.java | 19 +- 36 files changed, 971 insertions(+), 71 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java index dbe20738531..6da8f4fabca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactory.java @@ -39,11 +39,13 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobType; +import org.apache.flink.runtime.scheduler.CachedIntermediateDataSetCorruptedException; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; import org.apache.flink.types.Either; import org.apache.flink.util.CompressedSerializedValue; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; import javax.annotation.Nullable; @@ -51,7 +53,9 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.function.Function; @@ -71,6 +75,8 @@ public class TaskDeploymentDescriptorFactory { private final Function resultPartitionRetriever; private final BlobWriter blobWriter; +private final Map +consumedClusterPartitionShuffleDescriptors; private TaskDeploymentDescriptorFactory( ExecutionAttemptID executionId, @@ -81,7 +87,9 @@ public class TaskDeploymentDescriptorFactory { List consumedPartitionGroups, Function resultPartitionRetriever
[flink] branch release-1.15 updated: [FLINK-27910][connector/filesystem] Register the timer to enforce rolling policy when file sink starts from scratch.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.15 by this push: new a9905e2a14b [FLINK-27910][connector/filesystem] Register the timer to enforce rolling policy when file sink starts from scratch. a9905e2a14b is described below commit a9905e2a14b54a10c429af8b75f414c8ac3b7638 Author: Gen Luo AuthorDate: Tue Jun 7 11:58:44 2022 +0800 [FLINK-27910][connector/filesystem] Register the timer to enforce rolling policy when file sink starts from scratch. This closes #19889. --- flink-connectors/flink-connector-files/pom.xml | 8 .../apache/flink/connector/file/sink/FileSink.java | 4 +- .../flink/connector/file/sink/FileSinkTest.java| 46 ++ 3 files changed, 56 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-files/pom.xml b/flink-connectors/flink-connector-files/pom.xml index 2a0ed9d4ffc..2486803c9b8 100644 --- a/flink-connectors/flink-connector-files/pom.xml +++ b/flink-connectors/flink-connector-files/pom.xml @@ -90,6 +90,14 @@ under the License. test + + org.apache.flink + flink-connector-base + ${project.version} + test + test-jar + + org.apache.flink flink-connector-test-utils diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java index 8fdf987b11f..236582767e3 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java @@ -140,11 +140,11 @@ public class FileSink @Override public FileWriter createWriter(InitContext context) throws IOException { -return bucketsBuilder.createWriter(context); +return restoreWriter(context, Collections.emptyList()); } @Override -public StatefulSinkWriter restoreWriter( +public FileWriter restoreWriter( InitContext context, Collection recoveredState) throws IOException { FileWriter writer = bucketsBuilder.createWriter(context); diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkTest.java new file mode 100644 index 000..516a071dfb7 --- /dev/null +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkTest.java @@ -0,0 +1,46 @@ +/* + * 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.connector.base.sink.writer.TestSinkInitContext; +import org.apache.flink.connector.file.sink.utils.IntegerFileSinkTestDataUtils; +import org.apache.flink.connector.file.sink.utils.PartSizeAndCheckpointRollingPolicy; +import org.apache.flink.core.fs.Path; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +/** Tests for {@link FileSink}. */ +public class FileSinkTest { + +@Test +public void testCreateFileWriterWithTimerRegistered() throws IOException { +TestSinkInitContext ctx = new TestSinkInitContext(); +FileSink sink = +FileSink.forRowFormat( +new Path("mock"), new IntegerFileSinkTestDataUtils.IntEncoder()) +.withRollingPolicy(new PartSizeAndCheckpointRollingPolicy<>(1024, true)) +.build(); +sink.createWriter(ctx); +assertEquals(ctx.getTestProcessingTimeService().getNumActiveTimers(), 1); +} +}
[flink] branch master updated: [FLINK-27910][connector/filesystem] Register the timer to enforce rolling policy when file sink starts from scratch.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 d036c23c0e5 [FLINK-27910][connector/filesystem] Register the timer to enforce rolling policy when file sink starts from scratch. d036c23c0e5 is described below commit d036c23c0e5c079eaafef250a5a14b7f3eead8f1 Author: Gen Luo AuthorDate: Tue Jun 7 11:58:44 2022 +0800 [FLINK-27910][connector/filesystem] Register the timer to enforce rolling policy when file sink starts from scratch. This closes #19889. --- flink-connectors/flink-connector-files/pom.xml | 8 .../apache/flink/connector/file/sink/FileSink.java | 4 +- .../flink/connector/file/sink/FileSinkTest.java| 46 ++ 3 files changed, 56 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-files/pom.xml b/flink-connectors/flink-connector-files/pom.xml index 6ea044c6b80..96c507096a9 100644 --- a/flink-connectors/flink-connector-files/pom.xml +++ b/flink-connectors/flink-connector-files/pom.xml @@ -89,6 +89,14 @@ under the License. test + + org.apache.flink + flink-connector-base + ${project.version} + test + test-jar + + org.apache.flink flink-connector-test-utils diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java index 8fdf987b11f..236582767e3 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/sink/FileSink.java @@ -140,11 +140,11 @@ public class FileSink @Override public FileWriter createWriter(InitContext context) throws IOException { -return bucketsBuilder.createWriter(context); +return restoreWriter(context, Collections.emptyList()); } @Override -public StatefulSinkWriter restoreWriter( +public FileWriter restoreWriter( InitContext context, Collection recoveredState) throws IOException { FileWriter writer = bucketsBuilder.createWriter(context); diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkTest.java new file mode 100644 index 000..516a071dfb7 --- /dev/null +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/sink/FileSinkTest.java @@ -0,0 +1,46 @@ +/* + * 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.connector.base.sink.writer.TestSinkInitContext; +import org.apache.flink.connector.file.sink.utils.IntegerFileSinkTestDataUtils; +import org.apache.flink.connector.file.sink.utils.PartSizeAndCheckpointRollingPolicy; +import org.apache.flink.core.fs.Path; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +/** Tests for {@link FileSink}. */ +public class FileSinkTest { + +@Test +public void testCreateFileWriterWithTimerRegistered() throws IOException { +TestSinkInitContext ctx = new TestSinkInitContext(); +FileSink sink = +FileSink.forRowFormat( +new Path("mock"), new IntegerFileSinkTestDataUtils.IntEncoder()) +.withRollingPolicy(new PartSizeAndCheckpointRollingPolicy<>(1024, true)) +.build(); +sink.createWriter(ctx); +assertEquals(ctx.getTestProcessingTimeService().getNumActiveTimers(), 1); +} +}
[flink] branch master updated: [FLINK-25705][docs]Translate "Metric Reporters" page of "Deployment" in to Chinese.
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 5176669bc0e [FLINK-25705][docs]Translate "Metric Reporters" page of "Deployment" in to Chinese. 5176669bc0e is described below commit 5176669bc0ee678f21934f239d2b34a59102b1b8 Author: Chengkai Yang AuthorDate: Fri Apr 15 19:47:45 2022 +0800 [FLINK-25705][docs]Translate "Metric Reporters" page of "Deployment" in to Chinese. This fixes #19496. Co-authored-by: Roc Marshal --- .../content.zh/docs/deployment/metric_reporters.md | 207 +++-- .../influxdb_reporter_configuration_zh.html| 72 +++ .../generated/metric_reporters_section_zh.html | 60 ++ ...eus_push_gateway_reporter_configuration_zh.html | 48 + 4 files changed, 292 insertions(+), 95 deletions(-) diff --git a/docs/content.zh/docs/deployment/metric_reporters.md b/docs/content.zh/docs/deployment/metric_reporters.md index 87d6605f8f5..36eba808a7f 100644 --- a/docs/content.zh/docs/deployment/metric_reporters.md +++ b/docs/content.zh/docs/deployment/metric_reporters.md @@ -24,22 +24,24 @@ specific language governing permissions and limitations under the License. --> + + # Metric Reporters -Flink allows reporting metrics to external systems. -For more information about Flink's metric system go to the [metric system documentation]({{< ref "docs/ops/metrics" >}}). +Flink 支持用户将 Flink 的各项运行时指标发送给外部系统。 +了解更多指标方面信息可查看 [metric 系统相关文档]({{< ref "docs/ops/metrics" >}})。 -Metrics can be exposed to an external system by configuring one or several reporters in `conf/flink-conf.yaml`. These -reporters will be instantiated on each job and task manager when they are started. +你可以通过 `conf/flink-conf.yaml` 文件来配置一种或多种发送器,将运行时指标暴露给外部系统。 +发送器会在 TaskManager、Flink 作业启动时进行实例化。 -Below is a list of parameters that are generally applicable to all reporters. All properties are configured by setting `metrics.reporter..` in the configuration. Reporters may additionally offer implementation-specific parameters, which are documented in the respective reporter's section. +下面列出了所有发送器都适用的参数,可以通过配置文件中的 `metrics.reporter..` 项进行配置。有些发送器有自己特有的配置,详见该发送器章节下的具体说明。 -{{< include_reporter_config "layouts/shortcodes/generated/metric_reporters_section.html" >}} +{{< include_reporter_config "layouts/shortcodes/generated/metric_reporters_section_zh.html" >}} -All reporter configurations must contain the `factory.class` property. -Some reporters (referred to as `Scheduled`) allow specifying a reporting `interval`. +每种发送器的配置需要有 `factory.class` 属性。 +有些基于定时调度的发送器还可以通过 `interval` 来配置发送间隔。 -Example reporter configuration that specifies multiple reporters: +想要同时配置多个发送器,可参考以下示例。 ```yaml metrics.reporters: my_jmx_reporter,my_other_reporter @@ -54,80 +56,85 @@ metrics.reporter.my_other_reporter.host: 192.168.1.1 metrics.reporter.my_other_reporter.port: 1 ``` -**Important:** The jar containing the reporter must be accessible when Flink is started. - Reporters are loaded as [plugins]({{< ref "docs/deployment/filesystems/plugins" >}}). - All reporters documented on this page are available by default. +**注意**:Flink 在启动时必须能访问到发送器所属的 jar 包,发送器会被加载为 [plugins]({{< ref "docs/deployment/filesystems/plugins" >}}),Flink 自带的发送器(文档中已经列出的发送器)无需做其他配置,开箱即用。 + +你可以实现 `org.apache.flink.metrics.reporter.MetricReporter` 接口来自定义发送器,并实现 `Scheduled` 接口让发送器周期性地将运行时指标发送出去。 +另外也可以实现 `MetricReporterFactory` 接口,让发送器作为插件被 Flink 导入。 + + -You can write your own `Reporter` by implementing the `org.apache.flink.metrics.reporter.MetricReporter` interface. -If the Reporter should send out reports regularly you have to implement the `Scheduled` interface as well. -By additionally implementing a `MetricReporterFactory` your reporter can also be loaded as a plugin. +## 基于标志符格式 vs. 基于 tags 格式 -## Identifiers vs. tags +发送器发送运行指标时有以下两种格式: -There are generally 2 formats for how reporters export metrics. +第一种是基于标志符的格式,这种格式将指标所属的详细范围信息与该指标名称连在一起,组成一个长字符串标志符。 +比如 `job.MyJobName.numRestarts` 就是这样的格式。 -Identifier-based reporters assemble a flat string containing all scope information and the metric name. -An example could be `job.MyJobName.numRestarts`. +第二种是基于 tags 的格式,这种格式由指标的逻辑范围和该指标名称组成,代表某一类通用的指标,比如 `job.numRestarts`。 +这类格式描述的特定指标实例会以“键值对”的方式体现出来,即所谓的标签或变量,比如 "jobName=MyJobName"。 -Tag-based reporters on the other hand define a generic class of metrics consisting of a logical scope and metric name (e.g., `job.numRestarts`), -and report a particular instance of said metric as a set of `key-value` pairs, so called "tags" or "variables" (e.g., "jobName=MyJobName"). + ## Push vs. Pull -Metrics are
[flink] branch release-1.14 updated: [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.14 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.14 by this push: new 3be78d01a2c [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception 3be78d01a2c is described below commit 3be78d01a2c36cb89a91d43f08155a17ffb6cd7e Author: Ran Tao AuthorDate: Fri Mar 25 12:02:34 2022 +0800 [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception This closes #19241. --- .../flink/api/common/serialization/AbstractDeserializationSchema.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java index d0517300519..0b08828bbeb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java @@ -108,7 +108,8 @@ public abstract class AbstractDeserializationSchema implements Deserializatio "The implementation of AbstractDeserializationSchema is using a generic variable. " + "This is not supported, because due to Java's generic type erasure, it will not be possible to " + "determine the full type at runtime. For generic implementations, please pass the TypeInformation " -+ "or type class explicitly to the constructor."); ++ "or type class explicitly to the constructor.", +e); } }
[flink] branch release-1.15 updated: [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.15 by this push: new 3171cb019df [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception 3171cb019df is described below commit 3171cb019df2ea14bae9057e7da0113b55967bef Author: Ran Tao AuthorDate: Fri Mar 25 12:02:34 2022 +0800 [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception This closes #19241. --- .../flink/api/common/serialization/AbstractDeserializationSchema.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java index d0517300519..0b08828bbeb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java @@ -108,7 +108,8 @@ public abstract class AbstractDeserializationSchema implements Deserializatio "The implementation of AbstractDeserializationSchema is using a generic variable. " + "This is not supported, because due to Java's generic type erasure, it will not be possible to " + "determine the full type at runtime. For generic implementations, please pass the TypeInformation " -+ "or type class explicitly to the constructor."); ++ "or type class explicitly to the constructor.", +e); } }
[flink] branch master updated: [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii 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 0eda2cc35a7 [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception 0eda2cc35a7 is described below commit 0eda2cc35a7185592129df8d4b004acc7a60d3bf Author: Ran Tao AuthorDate: Fri Mar 25 12:02:34 2022 +0800 [FLINK-26788][core] Add cause when AbstractDeserializationSchema throw exception This closes #19241. --- .../flink/api/common/serialization/AbstractDeserializationSchema.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java index d0517300519..0b08828bbeb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java @@ -108,7 +108,8 @@ public abstract class AbstractDeserializationSchema implements Deserializatio "The implementation of AbstractDeserializationSchema is using a generic variable. " + "This is not supported, because due to Java's generic type erasure, it will not be possible to " + "determine the full type at runtime. For generic implementations, please pass the TypeInformation " -+ "or type class explicitly to the constructor."); ++ "or type class explicitly to the constructor.", +e); } }
[flink] branch master updated (c47500c5ba0 -> 74f28a59edd)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from c47500c5ba0 FLINK-27260][Runtime/Web] Expose changelog configurations in web ui add 74f28a59edd [hotfix][table] avoid NPE and add more UTs. Remove System.out.println() call. No new revisions were added by this update. Summary of changes: .../flink/table/codesplit/JavaCodeSplitter.java| 12 .../table/codesplit/JavaCodeSplitterTest.java | 33 ++ 2 files changed, 39 insertions(+), 6 deletions(-)
[flink] 11/13: [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 609be4225dc8b1ab2d040bc47dd439fb2aa03e9a Author: Yun Gao AuthorDate: Sun May 8 00:31:14 2022 +0800 [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots --- .../keyed/AbstractKeyedOperatorRestoreTestBase.java | 15 +-- .../AbstractNonKeyedOperatorRestoreTestBase.java| 15 +-- .../operatorstate/complexKeyed-flink1.15/_metadata | Bin 0 -> 13772 bytes .../operatorstate/nonKeyed-flink1.15/_metadata | Bin 0 -> 3693 bytes 4 files changed, 2 insertions(+), 28 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java index d9c8733adaf..511829c41ce 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java @@ -28,7 +28,6 @@ import org.apache.flink.test.state.operator.restore.ExecutionMode; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; /** Base class for all keyed operator restore tests. */ @@ -39,19 +38,7 @@ public abstract class AbstractKeyedOperatorRestoreTestBase extends AbstractOpera @Parameterized.Parameters(name = "Migrate Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_3, -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_3, FlinkVersion.v1_15); } public AbstractKeyedOperatorRestoreTestBase(FlinkVersion flinkVersion) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java index 257b8a2a6c9..ba2da3a1ded 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java @@ -28,7 +28,6 @@ import org.apache.flink.test.state.operator.restore.ExecutionMode; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createFirstStatefulMap; @@ -46,19 +45,7 @@ public abstract class AbstractNonKeyedOperatorRestoreTestBase @Parameterized.Parameters(name = "Migrate Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_3, -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_3, FlinkVersion.v1_15); } protected AbstractNonKeyedOperatorRestoreTestBase(FlinkVersion flinkVersion) { diff --git a/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.15/_metadata b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.15/_metadata new file mode 100644 index 000..216268e0c6d Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.15/_metadata differ diff --git a/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.15/_metadata b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.15/_metadata new file mode 100644 index 000..17e95d39e0d Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.15/_metadata differ
[flink] 06/13: [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 71fb78a45509c814331a4a8cd64f424c4b0f5ccd Author: Yun Gao AuthorDate: Sat May 7 18:20:29 2022 +0800 [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots --- .../org/apache/flink/cep/operator/CEPMigrationTest.java | 12 +--- .../cep-migration-after-branching-flink1.15-snapshot | Bin 0 -> 5806 bytes .../cep-migration-conditions-flink1.15-snapshot | Bin 0 -> 5453 bytes ...igration-single-pattern-afterwards-flink1.15-snapshot | Bin 0 -> 5203 bytes ...cep-migration-starting-new-pattern-flink1.15-snapshot | Bin 0 -> 5610 bytes 5 files changed, 1 insertion(+), 11 deletions(-) diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java index 7764810f510..05e8ba5212a 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java @@ -41,7 +41,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -72,16 +71,7 @@ public class CEPMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_6, FlinkVersion.v1_15); } public CEPMigrationTest(FlinkVersion migrateVersion) { diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.15-snapshot new file mode 100644 index 000..a05be3af1a4 Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.15-snapshot differ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.15-snapshot new file mode 100644 index 000..7d308fb91c2 Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.15-snapshot differ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.15-snapshot new file mode 100644 index 000..89133ae521e Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.15-snapshot differ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.15-snapshot new file mode 100644 index 000..4b956523c56 Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.15-snapshot differ
[flink] 08/13: [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit f918a1e2a8dd5aa643ba7c197bf66fe87a20c141 Author: Yun Gao AuthorDate: Sat May 7 18:39:09 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots --- .../StatefulJobSnapshotMigrationITCase.java | 2 +- .../0f726f2d-22c0-4dda-879b-bb8624330307 | Bin 0 -> 1580 bytes .../0f97e7c0-1cbc-4f83-856b-ea1e116871e9 | Bin 0 -> 1580 bytes .../0fa1caf1-1763-478a-bc82-e7d550c089fc | Bin 0 -> 1508 bytes .../0fccc5e1-3c9e-44bb-9fa4-fa6f9d71d80d | Bin 0 -> 281 bytes .../49dd3271-64d6-4745-b17d-a6194a9b3446 | Bin 0 -> 1161 bytes .../589e9917-2e39-421f-85d3-8629505ff1aa | Bin 0 -> 1161 bytes .../7072833e-bae0-4135-9ac8-fa727fcf | Bin 0 -> 532 bytes .../75a047a9-2b39-4d79-8a78-d949fc0cf641 | Bin 0 -> 532 bytes .../776b43fe-b178-4385-9855-7bf0dc999563 | Bin 0 -> 1508 bytes .../82d091ea-780a-4d37-9ad3-b51602aabcf7 | Bin 0 -> 1508 bytes .../_metadata| Bin 0 -> 2404 bytes .../a9bb379b-5650-4a6d-8212-1b3eed5f6d7c | Bin 0 -> 1508 bytes .../bb2867a1-c55b-4c02-8ac8-f34d9c397fa2 | Bin 0 -> 532 bytes .../d9feff3f-2c9b-4079-8d2f-8247f43f4cf3 | Bin 0 -> 549 bytes .../da5ff3e2-6581-4c0f-8e08-a9ecabead99a | Bin 0 -> 292 bytes .../df42dd03-a2ac-4853-bb18-d4a8fce7a08c | Bin 0 -> 532 bytes .../e1cd5e43-6878-4514-b461-0524577a4988 | Bin 0 -> 549 bytes .../00b2d84f-e311-4a24-934f-e45595809bff | Bin 0 -> 532 bytes .../02ea1d61-d4a7-41a9-9473-5eb59d7100a1 | Bin 0 -> 1161 bytes .../081c1952-d7d4-4fb5-a004-84121ecb2836 | Bin 0 -> 292 bytes .../0d86033f-614d-4747-b21c-56920fc5113e | Bin 0 -> 1580 bytes .../1416576d-4f63-46c9-b37a-470eff1b259b | Bin 0 -> 549 bytes .../279f4a2e-5c51-4dcd-88a1-c07df44bcee5 | Bin 0 -> 532 bytes .../45a3cfc7-5dcc-4f83-a655-27afe96b88f6 | Bin 0 -> 1161 bytes .../491b3fd2-44a5-49ea-8912-03248569e9e3 | Bin 0 -> 281 bytes .../6817568e-4d0d-4a79-87d8-c7414006addc | Bin 0 -> 532 bytes .../7d3a8daa-7397-4727-a876-ab4fe80f32ac | Bin 0 -> 549 bytes .../82e2a261-9c19-4fa9-9910-ddc03c5a7cd1 | Bin 0 -> 1508 bytes .../_metadata| Bin 0 -> 2404 bytes .../b87309a9-90be-4cd0-a841-dc53139d2aed | Bin 0 -> 532 bytes .../b9b0eb64-8aa3-46e4-ad5c-5a3912a00199 | Bin 0 -> 1580 bytes .../c4f2d6db-93a9-4de1-8852-05dc0af7321f | Bin 0 -> 1508 bytes .../d4722991-5b34-4ecb-a167-3c80a0153afe | Bin 0 -> 1508 bytes .../f3a2d60c-634e-4eb4-9465-fb2779e1104b | Bin 0 -> 1508 bytes .../1f8f7136-fe0a-46b0-b160-c426a3ad0232 | Bin 0 -> 535 bytes .../24d8d7aa-fa9b-4fc5-b7be-cf5b79a0062a | Bin 0 -> 1145 bytes .../2ff8cbe1-e401-4cea-9b33-fe2a2104eaa9 | Bin 0 -> 1620 bytes .../36b8f4be-e1b3-4d87-9d88-80bf9ba4f446 | Bin 0 -> 1145 bytes .../50b62350-80d5-40f6-ad0d-70937ba81840 | Bin 0 -> 535 bytes .../5c5ab7b4-f0b5-4121-85e5-63161d313f72 | Bin 0 -> 561 bytes .../62ac366a-f346-44a5-aeba-6712926f1046 | Bin 0 -> 561 bytes .../6ebee5ce-86b8-420c-bc18-d554603230bf | Bin 0 -> 1521 bytes .../94b7bbf5-8f1d-48e4-aa16-5b0b8aaed183 | Bin 0 -> 1521 bytes .../98596625-5eb0-4d29-b884-d398ba982811 | Bin 0 -> 535 bytes .../_metadata| Bin 0 -> 1872 bytes .../b8fdc030-5aa2-488d-8780-6025cc1bacdf | Bin 0 -> 535 bytes .../b94d1838-1ba5-4f6f-bd1a-d8e7471998d2 | Bin 0 -> 1521 bytes .../c7ffda36-7a31-4d9e-8dc7-e61939575d17 | Bin 0 -> 1521 bytes .../c95c6554-acbf-4b64-858e-35d147549d7a | Bin 0 -> 1620 bytes .../d719e1ae-34be-4ca3-97bd-9d16c7f0ca0b | Bin 0 -> 281 bytes .../fbdf5b8e-8c2e-4089-b22b-1748b426ad19 | Bin 0 -> 292 bytes .../2077517d-f5aa-426d-a66a-2b80fd60919d | Bin 0 -> 561 bytes .../29153b77-923e-4b66-94b9-147373fbba0b | Bin 0 -> 1521 bytes .../2a6c5fb1-9332-4fce-8b5f-0b710f26dbe3 | Bin 0 -> 1145 bytes .../4a4a9416-73ec-4ad0-ad3b-b421dc419c1d | Bin 0 -> 162
[flink] 07/13: [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 7d5a8138cf57e407e799db66eb9b66695be026cb Author: Yun Gao AuthorDate: Sat May 7 18:23:16 2022 +0800 [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots --- .../operators/windowing/WindowOperatorMigrationTest.java | 15 +-- ...op-migration-test-apply-event-time-flink1.15-snapshot | Bin 0 -> 1799 bytes ...gration-test-apply-processing-time-flink1.15-snapshot | Bin 0 -> 1691 bytes ...migration-test-kryo-serialized-key-flink1.15-snapshot | Bin 0 -> 4021 bytes ...p-migration-test-reduce-event-time-flink1.15-snapshot | Bin 0 -> 1655 bytes ...ration-test-reduce-processing-time-flink1.15-snapshot | Bin 0 -> 1596 bytes ...test-session-with-stateful-trigger-flink1.15-snapshot | Bin 0 -> 2707 bytes ...session-with-stateful-trigger-mint-flink1.15-snapshot | Bin 0 -> 2201 bytes 8 files changed, 1 insertion(+), 14 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java index 5d34b7eb5f4..33da08f1fc0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java @@ -61,7 +61,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.concurrent.ConcurrentLinkedQueue; @@ -85,19 +84,7 @@ public class WindowOperatorMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_3, -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_3, FlinkVersion.v1_15); } private static final TypeInformation> STRING_INT_TUPLE = diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.15-snapshot new file mode 100644 index 000..0aa53ce4408 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.15-snapshot new file mode 100644 index 000..a8f3ab38bfb Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-kryo-serialized-key-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-kryo-serialized-key-flink1.15-snapshot new file mode 100644 index 000..2afb02375bb Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-kryo-serialized-key-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.15-snapshot new file mode 100644 index 000..eca89981174 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.15-snapshot new file mode 100644 index 000..ade4333cb36 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigge
[flink] 13/13: [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 6f69f4e74054d3c7c40d4a91957f9212e1c215d0 Author: Yun Gao AuthorDate: Sun May 8 00:37:57 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots --- .../089594ed-2f58-4d20-aa9a-da9d018ae002| Bin 0 -> 684 bytes .../13345fae-1542-4ea6-bfae-e29a8d2872ba| Bin 0 -> 1959 bytes .../200e50e2-6909-4571-bc02-4ec64f2c52f4| Bin 0 -> 8205 bytes .../32243c1b-33e2-4c51-86ca-5600282722b5| Bin 0 -> 684 bytes .../326e2e77-5e4a-4763-89e4-f9331eea0f0b| Bin 0 -> 684 bytes .../5562f012-828d-4421-a727-7972235a66c0| Bin 0 -> 457 bytes .../6a97d964-d395-46b9-a3f7-f728673e77a8| Bin 0 -> 1959 bytes .../92aa11ad-122f-4ed2-8215-2a4c81a78139| Bin 0 -> 10165 bytes .../_metadata | Bin 0 -> 1905 bytes .../ab35adb3-8649-43e6-b2c4-e00b260b5705| Bin 0 -> 1959 bytes .../b9dad9f3-dfe1-4c9c-8485-b4a7a50e2c22| Bin 0 -> 10165 bytes .../ca0fc48a-1ec3-4128-8961-b484916e7a5f| Bin 0 -> 12125 bytes .../d9c140d6-bf78-4d51-9fee-8bd0a3911a76| Bin 0 -> 457 bytes .../e9732c32-2338-4a31-8b53-77695d60df60| Bin 0 -> 684 bytes .../f0760787-e6e2-4392-a22f-2e5fead874fc| Bin 0 -> 1959 bytes .../1446d2b3-88a3-42df-8fc0-0cbf314286f9| Bin 0 -> 1959 bytes .../4d4f766e-12bf-4af2-a00e-9720e012cdb8| Bin 0 -> 1959 bytes .../5ee1b4b3-0eb6-405a-9c38-bd58dfdd00c7| Bin 0 -> 1959 bytes .../5f89eafd-abcb-4f3d-86f0-9d57df1da496| Bin 0 -> 10165 bytes .../6cc27fc4-30d3-46ed-baa3-ce603fe4fd1e| Bin 0 -> 684 bytes .../927da194-6e71-4aa2-a483-b2ba478e1e91| Bin 0 -> 684 bytes .../9a91e108-133b-4511-8f10-60063f15f37f| Bin 0 -> 8205 bytes .../_metadata | Bin 0 -> 1905 bytes .../aba9de6f-ae0d-46b7-bd9b-3463c4bd650d| Bin 0 -> 10165 bytes .../b8f765a4-9d97-40b2-a313-f2959c671cc3| Bin 0 -> 457 bytes .../bdfd9f5e-35eb-42bf-b53c-a8c26048ed58| Bin 0 -> 684 bytes .../c13e50cd-6b0c-4e9c-9a7c-49b7c1900f88| Bin 0 -> 684 bytes .../e3dd146d-784e-4faf-94f2-f730911248b2| Bin 0 -> 12125 bytes .../f6c5156e-8a2c-4770-a19a-c2b03ffb5afc| Bin 0 -> 1959 bytes .../f95821e0-3472-4d17-ad28-c8d6885cd3f9| Bin 0 -> 457 bytes .../0fad68a0-1901-4c76-a351-8d1f0e18818d| Bin 0 -> 684 bytes .../2ec2051b-ef79-4264-afb1-966daa2c7978| Bin 0 -> 457 bytes .../35b11c9d-d021-468a-9fab-e3f95309fa5c| Bin 0 -> 10207 bytes .../409fcc80-3c59-4ae2-b57b-2f21dd6ee84b| Bin 0 -> 1943 bytes .../6068c6fc-2625-425b-b0df-45eb9d99a7d1| Bin 0 -> 1943 bytes .../6c52ea22-1f78-4cd3-bd3d-d0009bb95c9d| Bin 0 -> 10207 bytes .../78d03aaa-5112-4385-8547-1e13f4b44485| Bin 0 -> 684 bytes .../7aa365eb-859f-403c-bc55-02fca134002e| Bin 0 -> 8135 bytes .../7b68e813-44c9-42a2-847b-09bbe06abbd4| Bin 0 -> 457 bytes .../7e8d60e1-bee8-4b12-acbe-f46d953927b9| Bin 0 -> 1943 bytes .../88e0e19a-4a9f-4061-9ae3-a9aad1459135| Bin 0 -> 684 bytes .../8a7978d3-38cf-4587-8d62-49582820882a| Bin 0 -> 12257 bytes .../_metadata | Bin 0 -> 1601 bytes .../adab88ff-f874-4ba6-ab7a-f646b98ef645| Bin 0 -> 1943 bytes .../b310fe20-cd1c-4f3e-96af-b3a5dc72aa11| Bin 0 -> 684 bytes .../0ef22927-6cc8-43fb-9c51-ad1b660e4f0a| Bin 0 -> 12257 bytes .../144220e2-a4cd-48b2-9d52-b04f1b4b65b2| Bin 0 -> 10207 bytes .../1cb1461e-70a4-448d-91d9-3a6eae981d71| Bin 0 -> 1943 bytes .../1ebaeb6e-7f31-4d6a-88e5-11a25182a1c8| Bin 0 -> 457 bytes .../267a46e2-3a6d-47fd-8455-fde983d35a6d| Bin 0 -> 1943 bytes .../3161a920-833b-4dd5-9c93-8e79bef3c4f8| Bin 0 -> 8135 bytes .../4f0730bc-8130-4895-b8eb-ff56fdbec35c| Bin 0 -> 684 bytes .../6395fb41-c202-4c8a-a5ca-7adefe1d9484| Bin 0 -> 684 bytes .../6558d958-50c9-40d1-80bd-42a9e63ea771| Bin 0 -> 1943 bytes .../7dc70b15-e411-4fd7-b3ba-7ad12fd825ad| Bin 0 -> 10207 bytes .../_metadata | Bin 0 -> 2221 bytes .../a0af6b4
[flink] 10/13: [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit a295871d5e660a44c8812331021d2e6dec0ca6d5 Author: Yun Gao AuthorDate: Sat May 7 18:59:55 2022 +0800 [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots --- .../migration/TypeSerializerSnapshotMigrationITCase.java | 2 +- .../95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 | Bin 0 -> 609 bytes .../_metadata | Bin 0 -> 381 bytes .../ea58951f-6090-4285-802f-8eacefd354cf | Bin 0 -> 292 bytes .../213443f4-6e70-4333-a4cd-64cc6e210de1 | Bin 0 -> 292 bytes .../213c4746-830c-4da3-80ec-01a41ab78aa3 | Bin 0 -> 609 bytes .../_metadata | Bin 0 -> 381 bytes .../7a28d2df-54c8-4415-b055-1e48e83427ed | Bin 0 -> 639 bytes .../_metadata | Bin 0 -> 343 bytes .../e00ed3e4-e8af-4ead-8e2e-5477fcb74780 | Bin 0 -> 292 bytes .../0fb32bfd-093a-43d6-b1cc-e3ae9f277e4b | Bin 0 -> 639 bytes .../1b2988c1-ec2f-4fa2-af44-51ae2f21443d | Bin 0 -> 292 bytes .../_metadata | Bin 0 -> 381 bytes .../6e1f6918-2e88-4357-82c7-1bdf533ef301 | Bin 0 -> 292 bytes .../8936e936-8b0f-40ae-85c9-82907b0ee605 | Bin 0 -> 639 bytes .../_metadata | Bin 0 -> 343 bytes .../10e5c37a-3ca9-48f9-b06c-95fd44f0b996 | Bin 0 -> 292 bytes .../58b28015-e14e-4514-91f1-3cc4900708ab | Bin 0 -> 639 bytes .../_metadata | Bin 0 -> 343 bytes 19 files changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java index 96d607bcc93..b56e2359631 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java @@ -63,7 +63,7 @@ public class TypeSerializerSnapshotMigrationITCase extends SnapshotMigrationTest private static final int NUM_SOURCE_ELEMENTS = 4; // TODO increase this to newer version to create and test snapshot migration for newer versions -private static final FlinkVersion currentVersion = FlinkVersion.v1_14; +private static final FlinkVersion currentVersion = FlinkVersion.v1_15; // TODO change this to CREATE_SNAPSHOT to (re)create binary snapshots // TODO Note: You should generate the snapshot based on the release branch instead of the diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 new file mode 100644 index 000..2ec60abdc1b Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 differ diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/_metadata b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/_metadata new file mode 100644 index 000..2cf98021776 Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/_metadata differ diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/ea58951f-6090-4285-802f-8eacefd354cf b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/ea58951f-6090-4285-802f-8eacefd354cf new file mode 100644 index 000..2f020711c6f Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/ea58951f-6090-4285-802f-8eacefd354cf differ diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-savepoint-native/213443f4-6e70-4333-a4cd-64cc6e210de1 b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-savepoint-native/213443f4-6e70-4333-a4cd-64cc6e210de1 new file mode 100644 index 000..2f020711c6f Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer
[flink] 05/13: [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 5b953bf9b260937b907eeaa9bd677bf087d68714 Author: Yun Gao AuthorDate: Sat May 7 18:15:35 2022 +0800 [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots --- .../hdfstests/ContinuousFileProcessingMigrationTest.java | 3 ++- ...ction-migration-test-165191845-flink1.15-snapshot | Bin 0 -> 324 bytes ...ction-migration-test-1651918451000-flink1.15-snapshot | Bin 0 -> 324 bytes ...ction-migration-test-1651918452000-flink1.15-snapshot | Bin 0 -> 324 bytes .../resources/reader-migration-test-flink1.15-snapshot | Bin 0 -> 2622 bytes 5 files changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java index 08895c621f8..49254c0ce91 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java @@ -84,7 +84,8 @@ public class ContinuousFileProcessingMigrationTest { Tuple2.of(FlinkVersion.v1_11, 1594561663000L), Tuple2.of(FlinkVersion.v1_12, 1613720148000L), Tuple2.of(FlinkVersion.v1_13, 1627550216000L), -Tuple2.of(FlinkVersion.v1_14, 1633938795000L)); +Tuple2.of(FlinkVersion.v1_14, 1633938795000L), +Tuple2.of(FlinkVersion.v1_15, 165191845L)); } /** diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-165191845-flink1.15-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-165191845-flink1.15-snapshot new file mode 100644 index 000..008d20a2aae Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-165191845-flink1.15-snapshot differ diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918451000-flink1.15-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918451000-flink1.15-snapshot new file mode 100644 index 000..619f020e01c Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918451000-flink1.15-snapshot differ diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918452000-flink1.15-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918452000-flink1.15-snapshot new file mode 100644 index 000..fa8e3e38fba Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918452000-flink1.15-snapshot differ diff --git a/flink-fs-tests/src/test/resources/reader-migration-test-flink1.15-snapshot b/flink-fs-tests/src/test/resources/reader-migration-test-flink1.15-snapshot new file mode 100644 index 000..ba9d842c199 Binary files /dev/null and b/flink-fs-tests/src/test/resources/reader-migration-test-flink1.15-snapshot differ
[flink] 03/13: [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 5e7b0d41ca200d932c6d5f4fcf53e7bf194c339a Author: Yun Gao AuthorDate: Sat May 7 14:07:27 2022 +0800 [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots --- .../kafka/FlinkKafkaProducerMigrationTest.java | 10 +- .../kafka-migration-kafka-producer-flink-1.15-snapshot | Bin 0 -> 1242 bytes 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java index 67a74879c64..754d28dff48 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import java.util.Properties; @@ -43,14 +42,7 @@ import java.util.Properties; public class FlinkKafkaProducerMigrationTest extends KafkaMigrationTestBase { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_15); } public FlinkKafkaProducerMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot new file mode 100644 index 000..35af28dd7b1 Binary files /dev/null and b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot differ
[flink] 02/13: [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 9738c07434866e87d429d0bf38a5cced99c967cf Author: Yun Gao AuthorDate: Sat May 7 12:11:13 2022 +0800 [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots --- .../kafka/FlinkKafkaConsumerBaseMigrationTest.java | 14 +- ...onsumer-migration-test-flink1.15-empty-state-snapshot | Bin 0 -> 1465 bytes .../kafka-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 1519 bytes 3 files changed, 1 insertion(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index 0ba99452922..645278c9597 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -46,7 +46,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -94,18 +93,7 @@ public class FlinkKafkaConsumerBaseMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_4, FlinkVersion.v1_15); } public FlinkKafkaConsumerBaseMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot new file mode 100644 index 000..f6d3a17968c Binary files /dev/null and b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot differ diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot new file mode 100644 index 000..7039d10413c Binary files /dev/null and b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot differ
[flink] 09/13: [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 6aaae5e199de69ac7cf63465cfff27153f56b43e Author: Yun Gao AuthorDate: Sat May 7 18:58:34 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots --- .../StatefulJobWBroadcastStateMigrationITCase.java | 2 +- .../4b5079fa-e717-4e10-b2c4-8a802e89bcea | Bin 0 -> 704 bytes .../4c7c9c93-35c9-4dca-b8ce-78e05e7deecb | Bin 0 -> 786 bytes .../4d985f85-8f12-403b-b20c-c9757803676e | Bin 0 -> 786 bytes .../569f64fc-5b0a-4155-b583-85bcefddf6c3 | Bin 0 -> 704 bytes .../69cf5401-62e2-49c8-bdf1-ddca7e715e9c | Bin 0 -> 346 bytes .../6a80ecb7-08f5-438f-b48d-47f952fe6815 | Bin 0 -> 281 bytes .../75b2bf59-13f5-4005-a790-87c578d830b6 | Bin 0 -> 346 bytes .../7e8ed8a9-9f30-4260-9a41-bc1b753cc241 | Bin 0 -> 292 bytes .../87a4742c-8b19-49fa-9d44-fb436d2c3d55 | Bin 0 -> 786 bytes .../8e8e15d8-a729-49c8-824e-f35d69a7eb27 | Bin 0 -> 786 bytes .../8ec2e3f6-d6e7-494a-999c-320d8430e840 | Bin 0 -> 346 bytes .../9dd0ece3-2846-4ad7-96c3-e6ed1f240b00 | Bin 0 -> 281 bytes .../_metadata| Bin 0 -> 2716 bytes .../a7a92281-af21-4bff-92b9-796dac4a5d8b | Bin 0 -> 346 bytes .../a8f40852-fef2-4692-aab5-0c5b7b4795b7 | Bin 0 -> 704 bytes .../acf08fa6-8f88-4eda-bb33-5a59b57c15a2 | Bin 0 -> 786 bytes .../df37ac32-6ee6-4515-8949-d22d834bb602 | Bin 0 -> 786 bytes .../e00940d2-26c9-4df1-baa7-91fe14fb22ff | Bin 0 -> 786 bytes .../ecd5797c-e28e-483c-bf9e-9fb65bb65d0c | Bin 0 -> 786 bytes .../ee8efabe-9d32-4938-a929-49111eaa604b | Bin 0 -> 292 bytes .../fd302dc6-6626-4667-be02-7977abd22951 | Bin 0 -> 704 bytes .../03a3a2fc-4a96-40be-9335-385a0c197af1 | Bin 0 -> 704 bytes .../23ab27ad-ad73-4301-9c66-8489ce73f15b | Bin 0 -> 786 bytes .../35ccf95e-2d8f-472c-930e-505fc6192f55 | Bin 0 -> 786 bytes .../37e7749e-caca-4c6f-a37c-a108500b064a | Bin 0 -> 704 bytes .../477bd007-8515-4899-8f59-db439bddfcd1 | Bin 0 -> 292 bytes .../5006528f-67f3-45ee-8ba6-3f0723c6e7f6 | Bin 0 -> 786 bytes .../6989aa3d-0bac-4c4f-beb5-0b37ce22fedc | Bin 0 -> 346 bytes .../75e9f691-54e2-4d6f-b452-3f7f3315c4b8 | Bin 0 -> 704 bytes .../946ea2be-f497-4f5a-b146-aadd37987bed | Bin 0 -> 786 bytes .../9980a6de-8584-4ffc-9212-ab8b7b1fbd6d | Bin 0 -> 292 bytes .../9c688e98-92ad-4cbe-b9eb-2ec0eb721f92 | Bin 0 -> 704 bytes .../9eee63c8-4ef4-4a24-9746-2369bf368128 | Bin 0 -> 786 bytes .../_metadata| Bin 0 -> 2716 bytes .../b09d095c-824c-4031-b70d-9418db96def2 | Bin 0 -> 346 bytes .../bed0e914-2759-45d2-88f3-cc738d3636db | Bin 0 -> 281 bytes .../c53bc18f-fe96-4c76-9a8e-44e831b2ea0b | Bin 0 -> 786 bytes .../cd253f16-8f00-4cca-ab4e-e301eeeb7f4f | Bin 0 -> 786 bytes .../d1dc0055-4a6a-412a-9c41-04992cd57ec4 | Bin 0 -> 281 bytes .../e3d661ce-138e-4e12-9a8f-a0f21dd80819 | Bin 0 -> 786 bytes .../f1473696-0449-4694-94fd-4e25508028bf | Bin 0 -> 346 bytes .../fb3ae0a2-38c5-4f9d-897d-8af8ca73aa6b | Bin 0 -> 346 bytes .../1ae28f9a-3a1b-45c7-87d8-a06a5f62745f | Bin 0 -> 770 bytes .../2053a329-7a99-400c-8c01-da3e7d051472 | Bin 0 -> 770 bytes .../3278afbd-d04c-4b9c-9c3c-fdb67910f7c1 | Bin 0 -> 704 bytes .../4212bda0-8bdb-49af-b74e-136a7f9b1d8c | Bin 0 -> 346 bytes .../475271f3-4869-4e43-9c7f-9bbdf4985ce1 | Bin 0 -> 281 bytes .../4c472f03-e617-4ec6-b228-d3aa86a96118 | Bin 0 -> 770 bytes .../4cf87420-7df2-4ceb-b281-f6282000755c | Bin 0 -> 292 bytes .../4d8967a4-ee89-4de7-a666-b0739061f651 | Bin 0 -> 770 bytes .../516059bc-19a1-41b4-b3d6-09f42e1a860c | Bin 0 -> 770 bytes .../58c51c75-a5c3-48c8-adf1-dc09d0dc8005 | Bin 0 -> 346 bytes .../814813d2-a7e6-4178-8159-8c6697a501a3 | Bin 0 -> 292 bytes .../84789399-ea8e-4927-8eda-5b445c98ab82 | Bin 0 -> 770 bytes .../_metadata| Bin 0 -> 2412 bytes .../c8370c7a-5d
[flink] 12/13: [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit 6e004ce76c816c446ec22b8a6ee685d5ff40ba3e Author: Yun Gao AuthorDate: Sun May 8 00:33:49 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots --- .../3274cbaf-eb41-4cd4-b4e1-0498a8b561f6| Bin 0 -> 12185 bytes .../3ed5904f-af16-443b-a49b-b123aa1aeace| Bin 0 -> 457 bytes .../_metadata | Bin 0 -> 791 bytes .../af59b4b8-c925-4b32-b080-366794e3c3c1| Bin 0 -> 8205 bytes .../b91f3d7c-89e9-4bf7-8354-89bee7b43abd| Bin 0 -> 10195 bytes .../d79a7c86-3b0b-4ac5-ac52-99fe66b69475| Bin 0 -> 10195 bytes .../6cf48fc1-413c-4619-8224-d845f0304dd7| Bin 0 -> 8205 bytes .../728e5173-51c6-4dac-a2ff-f90daf1cef80| Bin 0 -> 10195 bytes .../_metadata | Bin 0 -> 791 bytes .../ae434126-b093-4a13-9f74-654df745a8bb| Bin 0 -> 457 bytes .../bd5f46c6-8e16-4366-a90d-d1a45eb5e20e| Bin 0 -> 12185 bytes .../c5799547-ffab-4984-a4c5-ee121f66a0bf| Bin 0 -> 10195 bytes .../566e1ea6-5366-412a-9fff-23428811805c| Bin 0 -> 457 bytes .../79915441-05d2-478c-83f8-5c8a5f335903| Bin 0 -> 10237 bytes .../_metadata | Bin 0 -> 639 bytes .../cc6b4832-f701-4572-9ecf-402f02adff4d| Bin 0 -> 10237 bytes .../d6e3633f-528b-4730-8bfc-b2262bedfaff| Bin 0 -> 8135 bytes .../e7383688-9835-4a92-b74d-8eabe476fe2f| Bin 0 -> 12317 bytes .../15991b8c-1b22-4e1b-b8c0-2f4de76d7a38| Bin 0 -> 12317 bytes .../228ae815-95e1-4a2a-b301-22d7b493a54e| Bin 0 -> 20 bytes .../85605c1b-c7c5-45c7-aa4e-6f059b46e9ee| Bin 0 -> 10237 bytes .../_metadata | Bin 0 -> 1107 bytes .../b1debad4-c40d-428c-9d5d-c6e50ebb7283| Bin 0 -> 457 bytes .../b3b80a5d-816d-4a21-8f54-053967df42e4| Bin 0 -> 10237 bytes .../f934f45f-538b-41f3-a3f4-d33d2030485b| Bin 0 -> 8135 bytes .../515f85e7-9bb7-4007-93b4-945e12d80c17| Bin 0 -> 10237 bytes .../52334b32-c0b0-413f-b96e-2cb94d2f464b| Bin 0 -> 457 bytes .../61bdd839-4f9e-4c8f-887a-e7b608abc339| Bin 0 -> 12317 bytes .../_metadata | Bin 0 -> 639 bytes .../dda27a2a-0826-43f3-83b4-148d750bc857| Bin 0 -> 8135 bytes .../e41f4b1b-0cf3-4f1f-9880-387d53d51fd0| Bin 0 -> 10237 bytes .../09152b6f-ec40-45bb-b7ac-cd9809eb9571| Bin 0 -> 12317 bytes .../491281eb-28f1-4fa5-84f3-ae5a743968eb| Bin 0 -> 10237 bytes .../72c1150d-8d58-4b03-9d42-7679c91b026f| Bin 0 -> 8135 bytes .../9fa49837-6352-4fd7-98cf-894e32460d4c| Bin 0 -> 457 bytes .../_metadata | Bin 0 -> 639 bytes .../d5e1395a-c0d0-44bf-a3a9-f8b0921a3232| Bin 0 -> 10237 bytes .../migration/StatefulJobSavepointMigrationITCase.scala | 2 +- 38 files changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3274cbaf-eb41-4cd4-b4e1-0498a8b561f6 b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3274cbaf-eb41-4cd4-b4e1-0498a8b561f6 new file mode 100644 index 000..e5eff0f0d3b Binary files /dev/null and b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3274cbaf-eb41-4cd4-b4e1-0498a8b561f6 differ diff --git a/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3ed5904f-af16-443b-a49b-b123aa1aeace b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3ed5904f-af16-443b-a49b-b123aa1aeace new file mode 100644 index 000..bd10233b3ef Binary files /dev/null and b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3ed5904f-af16-443b-a49b-b123aa1aeace differ diff --git a/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/_metadata b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/_metadata new file mode 100644 index 000..0178baab8dd Binary files /dev/null and b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/_metadata differ diff --
[flink] 01/13: [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit da9e6bee0c7771682eda018293f50c3e05a1dc43 Author: Yun Gao AuthorDate: Sat May 7 12:06:39 2022 +0800 [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots --- .../context-state-serializer-1.15/serializer-snapshot| Bin 0 -> 126 bytes .../resources/context-state-serializer-1.15/test-data| Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 134 bytes .../transaction-state-serializer-1.15/test-data | Bin 0 -> 17 bytes .../writeable-serializer-1.15/serializer-snapshot| Bin 0 -> 187 bytes .../test/resources/writeable-serializer-1.15/test-data | Bin 0 -> 7 bytes .../common/typeutils/TypeSerializerUpgradeTestBase.java | 2 +- .../big-dec-serializer-1.15/serializer-snapshot | Bin 0 -> 94 bytes .../src/test/resources/big-dec-serializer-1.15/test-data | Bin 0 -> 24 bytes .../big-int-serializer-1.15/serializer-snapshot | Bin 0 -> 94 bytes .../src/test/resources/big-int-serializer-1.15/test-data | Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 130 bytes .../boolean-primitive-array-serializer-1.15/test-data| Bin 0 -> 6 bytes .../boolean-serializer-1.15/serializer-snapshot | Bin 0 -> 96 bytes .../src/test/resources/boolean-serializer-1.15/test-data | 1 + .../boolean-value-serializer-1.15/serializer-snapshot| Bin 0 -> 106 bytes .../resources/boolean-value-serializer-1.15/test-data| 1 + .../serializer-snapshot | Bin 0 -> 124 bytes .../byte-primitive-array-serializer-1.15/test-data | Bin 0 -> 14 bytes .../resources/byte-serializer-1.15/serializer-snapshot | Bin 0 -> 90 bytes .../src/test/resources/byte-serializer-1.15/test-data| 1 + .../byte-value-serializer-1.15/serializer-snapshot | Bin 0 -> 100 bytes .../test/resources/byte-value-serializer-1.15/test-data | 1 + .../serializer-snapshot | Bin 0 -> 124 bytes .../char-primitive-array-serializer-1.15/test-data | Bin 0 -> 24 bytes .../resources/char-serializer-1.15/serializer-snapshot | Bin 0 -> 90 bytes .../src/test/resources/char-serializer-1.15/test-data| 1 + .../char-value-serializer-1.15/serializer-snapshot | Bin 0 -> 100 bytes .../test/resources/char-value-serializer-1.15/test-data | Bin 0 -> 2 bytes .../copyable-value-serializer-1.15/serializer-snapshot | Bin 0 -> 199 bytes .../resources/copyable-value-serializer-1.15/test-data | Bin 0 -> 8 bytes .../resources/date-serializer-1.15/serializer-snapshot | Bin 0 -> 90 bytes .../src/test/resources/date-serializer-1.15/test-data| Bin 0 -> 8 bytes .../serializer-snapshot | Bin 0 -> 128 bytes .../double-primitive-array-serializer-1.15/test-data | Bin 0 -> 84 bytes .../resources/double-serializer-1.15/serializer-snapshot | Bin 0 -> 94 bytes .../src/test/resources/double-serializer-1.15/test-data | 1 + .../double-value-serializer-1.15/serializer-snapshot | Bin 0 -> 104 bytes .../resources/double-value-serializer-1.15/test-data | 1 + .../resources/either-serializer-1.15/serializer-snapshot | Bin 0 -> 276 bytes .../src/test/resources/either-serializer-1.15/test-data | 1 + .../resources/enum-serializer-1.15/serializer-snapshot | Bin 0 -> 188 bytes .../src/test/resources/enum-serializer-1.15/test-data| Bin 0 -> 4 bytes .../enum-serializerreconfig-1.15/serializer-snapshot | Bin 0 -> 163 bytes .../resources/enum-serializerreconfig-1.15/test-data | Bin 0 -> 4 bytes .../serializer-snapshot | Bin 0 -> 126 bytes .../float-primitive-array-serializer-1.15/test-data | Bin 0 -> 44 bytes .../resources/float-serializer-1.15/serializer-snapshot | Bin 0 -> 92 bytes .../src/test/resources/float-serializer-1.15/test-data | 1 + .../float-value-serializer-1.15/serializer-snapshot | Bin 0 -> 102 bytes .../test/resources/float-value-serializer-1.15/test-data | 1 + .../generic-array-serializer-1.15/serializer-snapshot| Bin 0 -> 211 bytes .../resources/generic-array-serializer-1.15/test-data| Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 122 bytes .../int-primitive-array-serializer-1.15/test-data| Bin 0 -> 44 bytes .../resources/int-serializer-1.15/serializer-snapshot| Bin 0 -> 88 bytes .../src/test/resources/int-serializer-1.15/test-data | Bin 0 -> 4 bytes .../int-value-serializer-1.15/serializer-snapshot| Bin 0 -> 98 bytes .../test/resources/int-value-
[flink] 04/13: [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git commit efe8bd3d59cd93e9282ec3493d6219cbb3022edd Author: Yun Gao AuthorDate: Sat May 7 14:10:56 2022 +0800 [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots --- .../kinesis/FlinkKinesisConsumerMigrationTest.java | 3 ++- ...esis-consumer-migration-test-flink1.15-empty-snapshot | Bin 0 -> 2870 bytes .../kinesis-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 2938 bytes 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java index 8f17da7a9db..da041effe5a 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java @@ -108,7 +108,8 @@ public class FlinkKinesisConsumerMigrationTest { FlinkVersion.v1_11, FlinkVersion.v1_12, FlinkVersion.v1_13, -FlinkVersion.v1_14); +FlinkVersion.v1_14, +FlinkVersion.v1_15); } public FlinkKinesisConsumerMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-empty-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-empty-snapshot new file mode 100644 index 000..c96a82729ca Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-empty-snapshot differ diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-snapshot new file mode 100644 index 000..cfae7b6e560 Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-snapshot differ
[flink] branch master updated (2727d0234cf -> 6f69f4e7405)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 2727d0234cf [FLINK-27461][Kubernetes] Add option to set userAgent for kubeclient new da9e6bee0c7 [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots new 9738c074348 [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots new 5e7b0d41ca2 [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots new efe8bd3d59c [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots new 5b953bf9b26 [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots new 71fb78a4550 [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots new 7d5a8138cf5 [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots new f918a1e2a8d [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots new 6aaae5e199d [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots new a295871d5e6 [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots new 609be4225dc [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots new 6e004ce76c8 [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots new 6f69f4e7405 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots The 13 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: .../kafka/FlinkKafkaConsumerBaseMigrationTest.java | 14 +- .../kafka/FlinkKafkaProducerMigrationTest.java | 10 +- .../serializer-snapshot | Bin .../test-data | Bin ...nsumer-migration-test-flink1.15-empty-state-snapshot | Bin 0 -> 1465 bytes .../kafka-consumer-migration-test-flink1.15-snapshot| Bin 0 -> 1519 bytes .../kafka-migration-kafka-producer-flink-1.15-snapshot | Bin 0 -> 1242 bytes .../serializer-snapshot | Bin .../test-data | Bin .../kinesis/FlinkKinesisConsumerMigrationTest.java | 3 ++- ...sis-consumer-migration-test-flink1.15-empty-snapshot | Bin 0 -> 2870 bytes .../kinesis-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 2938 bytes .../serializer-snapshot | Bin .../test-data | Bin .../common/typeutils/TypeSerializerUpgradeTestBase.java | 2 +- .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data
[flink] branch release-1.15 updated (8ca31005c54 -> c63cc9a52a1)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git from 8ca31005c54 [hotfix][ci] Update libssl download link add 5f4fa7f234d [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots add c21c2b1cd4b [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots add ac5f9521d75 [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots add 6b72514c63a [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots add 6b2dbf9ff2f [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots add 859f2c6 [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots add afd82fe4e5e [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots add a25893c6f11 [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots add 0a5eb156edb [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots add 838f9b7fcd9 [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots add 2b387b59de9 [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots add 346ed673529 [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots add c63cc9a52a1 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots No new revisions were added by this update. Summary of changes: .../kafka/FlinkKafkaConsumerBaseMigrationTest.java | 14 +- .../kafka/FlinkKafkaProducerMigrationTest.java | 10 +- .../serializer-snapshot | Bin .../test-data | Bin ...nsumer-migration-test-flink1.15-empty-state-snapshot | Bin 0 -> 1465 bytes .../kafka-consumer-migration-test-flink1.15-snapshot| Bin 0 -> 1519 bytes .../kafka-migration-kafka-producer-flink-1.15-snapshot | Bin 0 -> 1242 bytes .../serializer-snapshot | Bin .../test-data | Bin .../kinesis/FlinkKinesisConsumerMigrationTest.java | 3 ++- ...sis-consumer-migration-test-flink1.15-empty-snapshot | Bin 0 -> 2870 bytes .../kinesis-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 2938 bytes .../serializer-snapshot | Bin .../test-data | Bin .../common/typeutils/TypeSerializerUpgradeTestBase.java | 2 +- .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | Bin .../serializer-snapshot | Bin .../test-data | 0 .../serializer-snapshot | Bin .../test-data | 0
[flink] branch release-1.15 updated: [hotfix][ci] Update libssl download link
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.15 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.15 by this push: new 8ca31005c54 [hotfix][ci] Update libssl download link 8ca31005c54 is described below commit 8ca31005c54bf650e7eed9bd7bc72bb025ff5af5 Author: Chesnay Schepler AuthorDate: Thu May 5 21:11:28 2022 +0200 [hotfix][ci] Update libssl download link --- tools/azure-pipelines/e2e-template.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tools/azure-pipelines/e2e-template.yml b/tools/azure-pipelines/e2e-template.yml index 637ea476d30..7301ff7c8a6 100644 --- a/tools/azure-pipelines/e2e-template.yml +++ b/tools/azure-pipelines/e2e-template.yml @@ -102,8 +102,8 @@ jobs: echo "Installing required software" sudo apt-get install -y bc libapr1 # install libssl1.0.0 for netty tcnative -wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.8_amd64.deb -sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.8_amd64.deb +wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.9_amd64.deb +sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.9_amd64.deb displayName: Prepare E2E run condition: not(eq(variables['SKIP'], '1')) - script: ${{parameters.environment}} PROFILE="$PROFILE -Dfast -Pskip-webui-build" ./tools/ci/compile.sh
[flink] 13/13: [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 3a30f2714b3e7dce0589aaa9f9aeaf7dc8b6b8bb Author: Yun Gao AuthorDate: Sun May 8 00:37:57 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots --- .../089594ed-2f58-4d20-aa9a-da9d018ae002| Bin 0 -> 684 bytes .../13345fae-1542-4ea6-bfae-e29a8d2872ba| Bin 0 -> 1959 bytes .../200e50e2-6909-4571-bc02-4ec64f2c52f4| Bin 0 -> 8205 bytes .../32243c1b-33e2-4c51-86ca-5600282722b5| Bin 0 -> 684 bytes .../326e2e77-5e4a-4763-89e4-f9331eea0f0b| Bin 0 -> 684 bytes .../5562f012-828d-4421-a727-7972235a66c0| Bin 0 -> 457 bytes .../6a97d964-d395-46b9-a3f7-f728673e77a8| Bin 0 -> 1959 bytes .../92aa11ad-122f-4ed2-8215-2a4c81a78139| Bin 0 -> 10165 bytes .../_metadata | Bin 0 -> 1905 bytes .../ab35adb3-8649-43e6-b2c4-e00b260b5705| Bin 0 -> 1959 bytes .../b9dad9f3-dfe1-4c9c-8485-b4a7a50e2c22| Bin 0 -> 10165 bytes .../ca0fc48a-1ec3-4128-8961-b484916e7a5f| Bin 0 -> 12125 bytes .../d9c140d6-bf78-4d51-9fee-8bd0a3911a76| Bin 0 -> 457 bytes .../e9732c32-2338-4a31-8b53-77695d60df60| Bin 0 -> 684 bytes .../f0760787-e6e2-4392-a22f-2e5fead874fc| Bin 0 -> 1959 bytes .../1446d2b3-88a3-42df-8fc0-0cbf314286f9| Bin 0 -> 1959 bytes .../4d4f766e-12bf-4af2-a00e-9720e012cdb8| Bin 0 -> 1959 bytes .../5ee1b4b3-0eb6-405a-9c38-bd58dfdd00c7| Bin 0 -> 1959 bytes .../5f89eafd-abcb-4f3d-86f0-9d57df1da496| Bin 0 -> 10165 bytes .../6cc27fc4-30d3-46ed-baa3-ce603fe4fd1e| Bin 0 -> 684 bytes .../927da194-6e71-4aa2-a483-b2ba478e1e91| Bin 0 -> 684 bytes .../9a91e108-133b-4511-8f10-60063f15f37f| Bin 0 -> 8205 bytes .../_metadata | Bin 0 -> 1905 bytes .../aba9de6f-ae0d-46b7-bd9b-3463c4bd650d| Bin 0 -> 10165 bytes .../b8f765a4-9d97-40b2-a313-f2959c671cc3| Bin 0 -> 457 bytes .../bdfd9f5e-35eb-42bf-b53c-a8c26048ed58| Bin 0 -> 684 bytes .../c13e50cd-6b0c-4e9c-9a7c-49b7c1900f88| Bin 0 -> 684 bytes .../e3dd146d-784e-4faf-94f2-f730911248b2| Bin 0 -> 12125 bytes .../f6c5156e-8a2c-4770-a19a-c2b03ffb5afc| Bin 0 -> 1959 bytes .../f95821e0-3472-4d17-ad28-c8d6885cd3f9| Bin 0 -> 457 bytes .../0fad68a0-1901-4c76-a351-8d1f0e18818d| Bin 0 -> 684 bytes .../2ec2051b-ef79-4264-afb1-966daa2c7978| Bin 0 -> 457 bytes .../35b11c9d-d021-468a-9fab-e3f95309fa5c| Bin 0 -> 10207 bytes .../409fcc80-3c59-4ae2-b57b-2f21dd6ee84b| Bin 0 -> 1943 bytes .../6068c6fc-2625-425b-b0df-45eb9d99a7d1| Bin 0 -> 1943 bytes .../6c52ea22-1f78-4cd3-bd3d-d0009bb95c9d| Bin 0 -> 10207 bytes .../78d03aaa-5112-4385-8547-1e13f4b44485| Bin 0 -> 684 bytes .../7aa365eb-859f-403c-bc55-02fca134002e| Bin 0 -> 8135 bytes .../7b68e813-44c9-42a2-847b-09bbe06abbd4| Bin 0 -> 457 bytes .../7e8d60e1-bee8-4b12-acbe-f46d953927b9| Bin 0 -> 1943 bytes .../88e0e19a-4a9f-4061-9ae3-a9aad1459135| Bin 0 -> 684 bytes .../8a7978d3-38cf-4587-8d62-49582820882a| Bin 0 -> 12257 bytes .../_metadata | Bin 0 -> 1601 bytes .../adab88ff-f874-4ba6-ab7a-f646b98ef645| Bin 0 -> 1943 bytes .../b310fe20-cd1c-4f3e-96af-b3a5dc72aa11| Bin 0 -> 684 bytes .../0ef22927-6cc8-43fb-9c51-ad1b660e4f0a| Bin 0 -> 12257 bytes .../144220e2-a4cd-48b2-9d52-b04f1b4b65b2| Bin 0 -> 10207 bytes .../1cb1461e-70a4-448d-91d9-3a6eae981d71| Bin 0 -> 1943 bytes .../1ebaeb6e-7f31-4d6a-88e5-11a25182a1c8| Bin 0 -> 457 bytes .../267a46e2-3a6d-47fd-8455-fde983d35a6d| Bin 0 -> 1943 bytes .../3161a920-833b-4dd5-9c93-8e79bef3c4f8| Bin 0 -> 8135 bytes .../4f0730bc-8130-4895-b8eb-ff56fdbec35c| Bin 0 -> 684 bytes .../6395fb41-c202-4c8a-a5ca-7adefe1d9484| Bin 0 -> 684 bytes .../6558d958-50c9-40d1-80bd-42a9e63ea771| Bin 0 -> 1943 bytes .../7dc70b15-e411-4fd7-b3ba-7ad12fd825ad| Bin 0 -> 10207 bytes .../_metadata | Bin 0 -> 2221 bytes .../a0
[flink] 11/13: [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 68cc19f78cb234437cd8638f8738452dede3edc7 Author: Yun Gao AuthorDate: Sun May 8 00:31:14 2022 +0800 [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots --- .../keyed/AbstractKeyedOperatorRestoreTestBase.java | 15 +-- .../AbstractNonKeyedOperatorRestoreTestBase.java| 15 +-- .../operatorstate/complexKeyed-flink1.15/_metadata | Bin 0 -> 13772 bytes .../operatorstate/nonKeyed-flink1.15/_metadata | Bin 0 -> 3693 bytes 4 files changed, 2 insertions(+), 28 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java index d9c8733adaf..511829c41ce 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java @@ -28,7 +28,6 @@ import org.apache.flink.test.state.operator.restore.ExecutionMode; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; /** Base class for all keyed operator restore tests. */ @@ -39,19 +38,7 @@ public abstract class AbstractKeyedOperatorRestoreTestBase extends AbstractOpera @Parameterized.Parameters(name = "Migrate Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_3, -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_3, FlinkVersion.v1_15); } public AbstractKeyedOperatorRestoreTestBase(FlinkVersion flinkVersion) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java index 257b8a2a6c9..ba2da3a1ded 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java @@ -28,7 +28,6 @@ import org.apache.flink.test.state.operator.restore.ExecutionMode; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createFirstStatefulMap; @@ -46,19 +45,7 @@ public abstract class AbstractNonKeyedOperatorRestoreTestBase @Parameterized.Parameters(name = "Migrate Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_3, -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_3, FlinkVersion.v1_15); } protected AbstractNonKeyedOperatorRestoreTestBase(FlinkVersion flinkVersion) { diff --git a/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.15/_metadata b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.15/_metadata new file mode 100644 index 000..216268e0c6d Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.15/_metadata differ diff --git a/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.15/_metadata b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.15/_metadata new file mode 100644 index 000..17e95d39e0d Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.15/_metadata differ
[flink] 04/13: [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit b3388a510e0f93340b4760c64e299dfe005ce523 Author: Yun Gao AuthorDate: Sat May 7 14:10:56 2022 +0800 [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots --- .../kinesis/FlinkKinesisConsumerMigrationTest.java | 3 ++- ...esis-consumer-migration-test-flink1.15-empty-snapshot | Bin 0 -> 2870 bytes .../kinesis-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 2938 bytes 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java index 8f17da7a9db..da041effe5a 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java @@ -108,7 +108,8 @@ public class FlinkKinesisConsumerMigrationTest { FlinkVersion.v1_11, FlinkVersion.v1_12, FlinkVersion.v1_13, -FlinkVersion.v1_14); +FlinkVersion.v1_14, +FlinkVersion.v1_15); } public FlinkKinesisConsumerMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-empty-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-empty-snapshot new file mode 100644 index 000..c96a82729ca Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-empty-snapshot differ diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-snapshot new file mode 100644 index 000..cfae7b6e560 Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.15-snapshot differ
[flink] 08/13: [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 8223a11c5bbe9ed6d007465664e2e05ac14504a6 Author: Yun Gao AuthorDate: Sat May 7 18:39:09 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots --- .../StatefulJobSnapshotMigrationITCase.java | 2 +- .../0f726f2d-22c0-4dda-879b-bb8624330307 | Bin 0 -> 1580 bytes .../0f97e7c0-1cbc-4f83-856b-ea1e116871e9 | Bin 0 -> 1580 bytes .../0fa1caf1-1763-478a-bc82-e7d550c089fc | Bin 0 -> 1508 bytes .../0fccc5e1-3c9e-44bb-9fa4-fa6f9d71d80d | Bin 0 -> 281 bytes .../49dd3271-64d6-4745-b17d-a6194a9b3446 | Bin 0 -> 1161 bytes .../589e9917-2e39-421f-85d3-8629505ff1aa | Bin 0 -> 1161 bytes .../7072833e-bae0-4135-9ac8-fa727fcf | Bin 0 -> 532 bytes .../75a047a9-2b39-4d79-8a78-d949fc0cf641 | Bin 0 -> 532 bytes .../776b43fe-b178-4385-9855-7bf0dc999563 | Bin 0 -> 1508 bytes .../82d091ea-780a-4d37-9ad3-b51602aabcf7 | Bin 0 -> 1508 bytes .../_metadata| Bin 0 -> 2404 bytes .../a9bb379b-5650-4a6d-8212-1b3eed5f6d7c | Bin 0 -> 1508 bytes .../bb2867a1-c55b-4c02-8ac8-f34d9c397fa2 | Bin 0 -> 532 bytes .../d9feff3f-2c9b-4079-8d2f-8247f43f4cf3 | Bin 0 -> 549 bytes .../da5ff3e2-6581-4c0f-8e08-a9ecabead99a | Bin 0 -> 292 bytes .../df42dd03-a2ac-4853-bb18-d4a8fce7a08c | Bin 0 -> 532 bytes .../e1cd5e43-6878-4514-b461-0524577a4988 | Bin 0 -> 549 bytes .../00b2d84f-e311-4a24-934f-e45595809bff | Bin 0 -> 532 bytes .../02ea1d61-d4a7-41a9-9473-5eb59d7100a1 | Bin 0 -> 1161 bytes .../081c1952-d7d4-4fb5-a004-84121ecb2836 | Bin 0 -> 292 bytes .../0d86033f-614d-4747-b21c-56920fc5113e | Bin 0 -> 1580 bytes .../1416576d-4f63-46c9-b37a-470eff1b259b | Bin 0 -> 549 bytes .../279f4a2e-5c51-4dcd-88a1-c07df44bcee5 | Bin 0 -> 532 bytes .../45a3cfc7-5dcc-4f83-a655-27afe96b88f6 | Bin 0 -> 1161 bytes .../491b3fd2-44a5-49ea-8912-03248569e9e3 | Bin 0 -> 281 bytes .../6817568e-4d0d-4a79-87d8-c7414006addc | Bin 0 -> 532 bytes .../7d3a8daa-7397-4727-a876-ab4fe80f32ac | Bin 0 -> 549 bytes .../82e2a261-9c19-4fa9-9910-ddc03c5a7cd1 | Bin 0 -> 1508 bytes .../_metadata| Bin 0 -> 2404 bytes .../b87309a9-90be-4cd0-a841-dc53139d2aed | Bin 0 -> 532 bytes .../b9b0eb64-8aa3-46e4-ad5c-5a3912a00199 | Bin 0 -> 1580 bytes .../c4f2d6db-93a9-4de1-8852-05dc0af7321f | Bin 0 -> 1508 bytes .../d4722991-5b34-4ecb-a167-3c80a0153afe | Bin 0 -> 1508 bytes .../f3a2d60c-634e-4eb4-9465-fb2779e1104b | Bin 0 -> 1508 bytes .../1f8f7136-fe0a-46b0-b160-c426a3ad0232 | Bin 0 -> 535 bytes .../24d8d7aa-fa9b-4fc5-b7be-cf5b79a0062a | Bin 0 -> 1145 bytes .../2ff8cbe1-e401-4cea-9b33-fe2a2104eaa9 | Bin 0 -> 1620 bytes .../36b8f4be-e1b3-4d87-9d88-80bf9ba4f446 | Bin 0 -> 1145 bytes .../50b62350-80d5-40f6-ad0d-70937ba81840 | Bin 0 -> 535 bytes .../5c5ab7b4-f0b5-4121-85e5-63161d313f72 | Bin 0 -> 561 bytes .../62ac366a-f346-44a5-aeba-6712926f1046 | Bin 0 -> 561 bytes .../6ebee5ce-86b8-420c-bc18-d554603230bf | Bin 0 -> 1521 bytes .../94b7bbf5-8f1d-48e4-aa16-5b0b8aaed183 | Bin 0 -> 1521 bytes .../98596625-5eb0-4d29-b884-d398ba982811 | Bin 0 -> 535 bytes .../_metadata| Bin 0 -> 1872 bytes .../b8fdc030-5aa2-488d-8780-6025cc1bacdf | Bin 0 -> 535 bytes .../b94d1838-1ba5-4f6f-bd1a-d8e7471998d2 | Bin 0 -> 1521 bytes .../c7ffda36-7a31-4d9e-8dc7-e61939575d17 | Bin 0 -> 1521 bytes .../c95c6554-acbf-4b64-858e-35d147549d7a | Bin 0 -> 1620 bytes .../d719e1ae-34be-4ca3-97bd-9d16c7f0ca0b | Bin 0 -> 281 bytes .../fbdf5b8e-8c2e-4089-b22b-1748b426ad19 | Bin 0 -> 292 bytes .../2077517d-f5aa-426d-a66a-2b80fd60919d | Bin 0 -> 561 bytes .../29153b77-923e-4b66-94b9-147373fbba0b | Bin 0 -> 1521 bytes .../2a6c5fb1-9332-4fce-8b5f-0b710f26dbe3 | Bin 0 -> 1145 bytes .../4a4a9416-73ec-4ad0-ad3b-b421dc419c1d | Bin 0 -> 162
[flink] 10/13: [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit d1b47bbdc79dd0e641c54e04f32acb5684dff546 Author: Yun Gao AuthorDate: Sat May 7 18:59:55 2022 +0800 [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots --- .../migration/TypeSerializerSnapshotMigrationITCase.java | 2 +- .../95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 | Bin 0 -> 609 bytes .../_metadata | Bin 0 -> 381 bytes .../ea58951f-6090-4285-802f-8eacefd354cf | Bin 0 -> 292 bytes .../213443f4-6e70-4333-a4cd-64cc6e210de1 | Bin 0 -> 292 bytes .../213c4746-830c-4da3-80ec-01a41ab78aa3 | Bin 0 -> 609 bytes .../_metadata | Bin 0 -> 381 bytes .../7a28d2df-54c8-4415-b055-1e48e83427ed | Bin 0 -> 639 bytes .../_metadata | Bin 0 -> 343 bytes .../e00ed3e4-e8af-4ead-8e2e-5477fcb74780 | Bin 0 -> 292 bytes .../0fb32bfd-093a-43d6-b1cc-e3ae9f277e4b | Bin 0 -> 639 bytes .../1b2988c1-ec2f-4fa2-af44-51ae2f21443d | Bin 0 -> 292 bytes .../_metadata | Bin 0 -> 381 bytes .../6e1f6918-2e88-4357-82c7-1bdf533ef301 | Bin 0 -> 292 bytes .../8936e936-8b0f-40ae-85c9-82907b0ee605 | Bin 0 -> 639 bytes .../_metadata | Bin 0 -> 343 bytes .../10e5c37a-3ca9-48f9-b06c-95fd44f0b996 | Bin 0 -> 292 bytes .../58b28015-e14e-4514-91f1-3cc4900708ab | Bin 0 -> 639 bytes .../_metadata | Bin 0 -> 343 bytes 19 files changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java index 96d607bcc93..b56e2359631 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/migration/TypeSerializerSnapshotMigrationITCase.java @@ -63,7 +63,7 @@ public class TypeSerializerSnapshotMigrationITCase extends SnapshotMigrationTest private static final int NUM_SOURCE_ELEMENTS = 4; // TODO increase this to newer version to create and test snapshot migration for newer versions -private static final FlinkVersion currentVersion = FlinkVersion.v1_14; +private static final FlinkVersion currentVersion = FlinkVersion.v1_15; // TODO change this to CREATE_SNAPSHOT to (re)create binary snapshots // TODO Note: You should generate the snapshot based on the release branch instead of the diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 new file mode 100644 index 000..2ec60abdc1b Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/95ac2fd3-cd2e-458d-b7a3-eeebb77c0ea7 differ diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/_metadata b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/_metadata new file mode 100644 index 000..2cf98021776 Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/_metadata differ diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/ea58951f-6090-4285-802f-8eacefd354cf b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/ea58951f-6090-4285-802f-8eacefd354cf new file mode 100644 index 000..2f020711c6f Binary files /dev/null and b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-checkpoint/ea58951f-6090-4285-802f-8eacefd354cf differ diff --git a/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-savepoint-native/213443f4-6e70-4333-a4cd-64cc6e210de1 b/flink-tests/src/test/resources/type-serializer-snapshot-migration-itcase-flink1.15-hashmap-savepoint-native/213443f4-6e70-4333-a4cd-64cc6e210de1 new file mode 100644 index 000..2f020711c6f Binary files /dev/null and b/flink-tests/src/test/resources/type-
[flink] 09/13: [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit b8618c21c33ea3257b90e373708a06be9d720309 Author: Yun Gao AuthorDate: Sat May 7 18:58:34 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots --- .../StatefulJobWBroadcastStateMigrationITCase.java | 2 +- .../4b5079fa-e717-4e10-b2c4-8a802e89bcea | Bin 0 -> 704 bytes .../4c7c9c93-35c9-4dca-b8ce-78e05e7deecb | Bin 0 -> 786 bytes .../4d985f85-8f12-403b-b20c-c9757803676e | Bin 0 -> 786 bytes .../569f64fc-5b0a-4155-b583-85bcefddf6c3 | Bin 0 -> 704 bytes .../69cf5401-62e2-49c8-bdf1-ddca7e715e9c | Bin 0 -> 346 bytes .../6a80ecb7-08f5-438f-b48d-47f952fe6815 | Bin 0 -> 281 bytes .../75b2bf59-13f5-4005-a790-87c578d830b6 | Bin 0 -> 346 bytes .../7e8ed8a9-9f30-4260-9a41-bc1b753cc241 | Bin 0 -> 292 bytes .../87a4742c-8b19-49fa-9d44-fb436d2c3d55 | Bin 0 -> 786 bytes .../8e8e15d8-a729-49c8-824e-f35d69a7eb27 | Bin 0 -> 786 bytes .../8ec2e3f6-d6e7-494a-999c-320d8430e840 | Bin 0 -> 346 bytes .../9dd0ece3-2846-4ad7-96c3-e6ed1f240b00 | Bin 0 -> 281 bytes .../_metadata| Bin 0 -> 2716 bytes .../a7a92281-af21-4bff-92b9-796dac4a5d8b | Bin 0 -> 346 bytes .../a8f40852-fef2-4692-aab5-0c5b7b4795b7 | Bin 0 -> 704 bytes .../acf08fa6-8f88-4eda-bb33-5a59b57c15a2 | Bin 0 -> 786 bytes .../df37ac32-6ee6-4515-8949-d22d834bb602 | Bin 0 -> 786 bytes .../e00940d2-26c9-4df1-baa7-91fe14fb22ff | Bin 0 -> 786 bytes .../ecd5797c-e28e-483c-bf9e-9fb65bb65d0c | Bin 0 -> 786 bytes .../ee8efabe-9d32-4938-a929-49111eaa604b | Bin 0 -> 292 bytes .../fd302dc6-6626-4667-be02-7977abd22951 | Bin 0 -> 704 bytes .../03a3a2fc-4a96-40be-9335-385a0c197af1 | Bin 0 -> 704 bytes .../23ab27ad-ad73-4301-9c66-8489ce73f15b | Bin 0 -> 786 bytes .../35ccf95e-2d8f-472c-930e-505fc6192f55 | Bin 0 -> 786 bytes .../37e7749e-caca-4c6f-a37c-a108500b064a | Bin 0 -> 704 bytes .../477bd007-8515-4899-8f59-db439bddfcd1 | Bin 0 -> 292 bytes .../5006528f-67f3-45ee-8ba6-3f0723c6e7f6 | Bin 0 -> 786 bytes .../6989aa3d-0bac-4c4f-beb5-0b37ce22fedc | Bin 0 -> 346 bytes .../75e9f691-54e2-4d6f-b452-3f7f3315c4b8 | Bin 0 -> 704 bytes .../946ea2be-f497-4f5a-b146-aadd37987bed | Bin 0 -> 786 bytes .../9980a6de-8584-4ffc-9212-ab8b7b1fbd6d | Bin 0 -> 292 bytes .../9c688e98-92ad-4cbe-b9eb-2ec0eb721f92 | Bin 0 -> 704 bytes .../9eee63c8-4ef4-4a24-9746-2369bf368128 | Bin 0 -> 786 bytes .../_metadata| Bin 0 -> 2716 bytes .../b09d095c-824c-4031-b70d-9418db96def2 | Bin 0 -> 346 bytes .../bed0e914-2759-45d2-88f3-cc738d3636db | Bin 0 -> 281 bytes .../c53bc18f-fe96-4c76-9a8e-44e831b2ea0b | Bin 0 -> 786 bytes .../cd253f16-8f00-4cca-ab4e-e301eeeb7f4f | Bin 0 -> 786 bytes .../d1dc0055-4a6a-412a-9c41-04992cd57ec4 | Bin 0 -> 281 bytes .../e3d661ce-138e-4e12-9a8f-a0f21dd80819 | Bin 0 -> 786 bytes .../f1473696-0449-4694-94fd-4e25508028bf | Bin 0 -> 346 bytes .../fb3ae0a2-38c5-4f9d-897d-8af8ca73aa6b | Bin 0 -> 346 bytes .../1ae28f9a-3a1b-45c7-87d8-a06a5f62745f | Bin 0 -> 770 bytes .../2053a329-7a99-400c-8c01-da3e7d051472 | Bin 0 -> 770 bytes .../3278afbd-d04c-4b9c-9c3c-fdb67910f7c1 | Bin 0 -> 704 bytes .../4212bda0-8bdb-49af-b74e-136a7f9b1d8c | Bin 0 -> 346 bytes .../475271f3-4869-4e43-9c7f-9bbdf4985ce1 | Bin 0 -> 281 bytes .../4c472f03-e617-4ec6-b228-d3aa86a96118 | Bin 0 -> 770 bytes .../4cf87420-7df2-4ceb-b281-f6282000755c | Bin 0 -> 292 bytes .../4d8967a4-ee89-4de7-a666-b0739061f651 | Bin 0 -> 770 bytes .../516059bc-19a1-41b4-b3d6-09f42e1a860c | Bin 0 -> 770 bytes .../58c51c75-a5c3-48c8-adf1-dc09d0dc8005 | Bin 0 -> 346 bytes .../814813d2-a7e6-4178-8159-8c6697a501a3 | Bin 0 -> 292 bytes .../84789399-ea8e-4927-8eda-5b445c98ab82 | Bin 0 -> 770 bytes .../_metadata| Bin 0 -> 2412 bytes .../c8370c
[flink] 07/13: [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit ab28f7430f4747ce43be79b653de99e43a3e6efe Author: Yun Gao AuthorDate: Sat May 7 18:23:16 2022 +0800 [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots --- .../operators/windowing/WindowOperatorMigrationTest.java | 15 +-- ...op-migration-test-apply-event-time-flink1.15-snapshot | Bin 0 -> 1799 bytes ...gration-test-apply-processing-time-flink1.15-snapshot | Bin 0 -> 1691 bytes ...migration-test-kryo-serialized-key-flink1.15-snapshot | Bin 0 -> 4021 bytes ...p-migration-test-reduce-event-time-flink1.15-snapshot | Bin 0 -> 1655 bytes ...ration-test-reduce-processing-time-flink1.15-snapshot | Bin 0 -> 1596 bytes ...test-session-with-stateful-trigger-flink1.15-snapshot | Bin 0 -> 2707 bytes ...session-with-stateful-trigger-mint-flink1.15-snapshot | Bin 0 -> 2201 bytes 8 files changed, 1 insertion(+), 14 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java index 5d34b7eb5f4..33da08f1fc0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java @@ -61,7 +61,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.concurrent.ConcurrentLinkedQueue; @@ -85,19 +84,7 @@ public class WindowOperatorMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_3, -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_3, FlinkVersion.v1_15); } private static final TypeInformation> STRING_INT_TUPLE = diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.15-snapshot new file mode 100644 index 000..0aa53ce4408 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.15-snapshot new file mode 100644 index 000..a8f3ab38bfb Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-kryo-serialized-key-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-kryo-serialized-key-flink1.15-snapshot new file mode 100644 index 000..2afb02375bb Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-kryo-serialized-key-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.15-snapshot new file mode 100644 index 000..eca89981174 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.15-snapshot new file mode 100644 index 000..ade4333cb36 Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.15-snapshot differ diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.15-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-t
[flink] 05/13: [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 5cae5c5a17a54c1f07d4a0087336a870202180bc Author: Yun Gao AuthorDate: Sat May 7 18:15:35 2022 +0800 [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots --- .../hdfstests/ContinuousFileProcessingMigrationTest.java | 3 ++- ...ction-migration-test-165191845-flink1.15-snapshot | Bin 0 -> 324 bytes ...ction-migration-test-1651918451000-flink1.15-snapshot | Bin 0 -> 324 bytes ...ction-migration-test-1651918452000-flink1.15-snapshot | Bin 0 -> 324 bytes .../resources/reader-migration-test-flink1.15-snapshot | Bin 0 -> 2622 bytes 5 files changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java index 08895c621f8..49254c0ce91 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java @@ -84,7 +84,8 @@ public class ContinuousFileProcessingMigrationTest { Tuple2.of(FlinkVersion.v1_11, 1594561663000L), Tuple2.of(FlinkVersion.v1_12, 1613720148000L), Tuple2.of(FlinkVersion.v1_13, 1627550216000L), -Tuple2.of(FlinkVersion.v1_14, 1633938795000L)); +Tuple2.of(FlinkVersion.v1_14, 1633938795000L), +Tuple2.of(FlinkVersion.v1_15, 165191845L)); } /** diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-165191845-flink1.15-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-165191845-flink1.15-snapshot new file mode 100644 index 000..008d20a2aae Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-165191845-flink1.15-snapshot differ diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918451000-flink1.15-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918451000-flink1.15-snapshot new file mode 100644 index 000..619f020e01c Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918451000-flink1.15-snapshot differ diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918452000-flink1.15-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918452000-flink1.15-snapshot new file mode 100644 index 000..fa8e3e38fba Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1651918452000-flink1.15-snapshot differ diff --git a/flink-fs-tests/src/test/resources/reader-migration-test-flink1.15-snapshot b/flink-fs-tests/src/test/resources/reader-migration-test-flink1.15-snapshot new file mode 100644 index 000..ba9d842c199 Binary files /dev/null and b/flink-fs-tests/src/test/resources/reader-migration-test-flink1.15-snapshot differ
[flink] 12/13: [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit f2e75c591b4ae169e257c33fa47f4bcf390e398d Author: Yun Gao AuthorDate: Sun May 8 00:33:49 2022 +0800 [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots --- .../3274cbaf-eb41-4cd4-b4e1-0498a8b561f6| Bin 0 -> 12185 bytes .../3ed5904f-af16-443b-a49b-b123aa1aeace| Bin 0 -> 457 bytes .../_metadata | Bin 0 -> 791 bytes .../af59b4b8-c925-4b32-b080-366794e3c3c1| Bin 0 -> 8205 bytes .../b91f3d7c-89e9-4bf7-8354-89bee7b43abd| Bin 0 -> 10195 bytes .../d79a7c86-3b0b-4ac5-ac52-99fe66b69475| Bin 0 -> 10195 bytes .../6cf48fc1-413c-4619-8224-d845f0304dd7| Bin 0 -> 8205 bytes .../728e5173-51c6-4dac-a2ff-f90daf1cef80| Bin 0 -> 10195 bytes .../_metadata | Bin 0 -> 791 bytes .../ae434126-b093-4a13-9f74-654df745a8bb| Bin 0 -> 457 bytes .../bd5f46c6-8e16-4366-a90d-d1a45eb5e20e| Bin 0 -> 12185 bytes .../c5799547-ffab-4984-a4c5-ee121f66a0bf| Bin 0 -> 10195 bytes .../566e1ea6-5366-412a-9fff-23428811805c| Bin 0 -> 457 bytes .../79915441-05d2-478c-83f8-5c8a5f335903| Bin 0 -> 10237 bytes .../_metadata | Bin 0 -> 639 bytes .../cc6b4832-f701-4572-9ecf-402f02adff4d| Bin 0 -> 10237 bytes .../d6e3633f-528b-4730-8bfc-b2262bedfaff| Bin 0 -> 8135 bytes .../e7383688-9835-4a92-b74d-8eabe476fe2f| Bin 0 -> 12317 bytes .../15991b8c-1b22-4e1b-b8c0-2f4de76d7a38| Bin 0 -> 12317 bytes .../228ae815-95e1-4a2a-b301-22d7b493a54e| Bin 0 -> 20 bytes .../85605c1b-c7c5-45c7-aa4e-6f059b46e9ee| Bin 0 -> 10237 bytes .../_metadata | Bin 0 -> 1107 bytes .../b1debad4-c40d-428c-9d5d-c6e50ebb7283| Bin 0 -> 457 bytes .../b3b80a5d-816d-4a21-8f54-053967df42e4| Bin 0 -> 10237 bytes .../f934f45f-538b-41f3-a3f4-d33d2030485b| Bin 0 -> 8135 bytes .../515f85e7-9bb7-4007-93b4-945e12d80c17| Bin 0 -> 10237 bytes .../52334b32-c0b0-413f-b96e-2cb94d2f464b| Bin 0 -> 457 bytes .../61bdd839-4f9e-4c8f-887a-e7b608abc339| Bin 0 -> 12317 bytes .../_metadata | Bin 0 -> 639 bytes .../dda27a2a-0826-43f3-83b4-148d750bc857| Bin 0 -> 8135 bytes .../e41f4b1b-0cf3-4f1f-9880-387d53d51fd0| Bin 0 -> 10237 bytes .../09152b6f-ec40-45bb-b7ac-cd9809eb9571| Bin 0 -> 12317 bytes .../491281eb-28f1-4fa5-84f3-ae5a743968eb| Bin 0 -> 10237 bytes .../72c1150d-8d58-4b03-9d42-7679c91b026f| Bin 0 -> 8135 bytes .../9fa49837-6352-4fd7-98cf-894e32460d4c| Bin 0 -> 457 bytes .../_metadata | Bin 0 -> 639 bytes .../d5e1395a-c0d0-44bf-a3a9-f8b0921a3232| Bin 0 -> 10237 bytes .../migration/StatefulJobSavepointMigrationITCase.scala | 2 +- 38 files changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3274cbaf-eb41-4cd4-b4e1-0498a8b561f6 b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3274cbaf-eb41-4cd4-b4e1-0498a8b561f6 new file mode 100644 index 000..e5eff0f0d3b Binary files /dev/null and b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3274cbaf-eb41-4cd4-b4e1-0498a8b561f6 differ diff --git a/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3ed5904f-af16-443b-a49b-b123aa1aeace b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3ed5904f-af16-443b-a49b-b123aa1aeace new file mode 100644 index 000..bd10233b3ef Binary files /dev/null and b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/3ed5904f-af16-443b-a49b-b123aa1aeace differ diff --git a/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/_metadata b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/_metadata new file mode 100644 index 000..0178baab8dd Binary files /dev/null and b/flink-tests/src/test/resources/stateful-scala-udf-migration-itcase-flink1.15-hashmap-checkpoint/_metadata diffe
[flink] 02/13: [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit c6a0d15891f533787a6423ceab73862a4acfbab5 Author: Yun Gao AuthorDate: Sat May 7 12:11:13 2022 +0800 [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots --- .../kafka/FlinkKafkaConsumerBaseMigrationTest.java | 14 +- ...onsumer-migration-test-flink1.15-empty-state-snapshot | Bin 0 -> 1465 bytes .../kafka-consumer-migration-test-flink1.15-snapshot | Bin 0 -> 1519 bytes 3 files changed, 1 insertion(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index 0ba99452922..645278c9597 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -46,7 +46,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -94,18 +93,7 @@ public class FlinkKafkaConsumerBaseMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_4, -FlinkVersion.v1_5, -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_4, FlinkVersion.v1_15); } public FlinkKafkaConsumerBaseMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot new file mode 100644 index 000..f6d3a17968c Binary files /dev/null and b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-empty-state-snapshot differ diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot new file mode 100644 index 000..7039d10413c Binary files /dev/null and b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.15-snapshot differ
[flink] 06/13: [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 3b83229944adc769b1d7a3ba9fe267f4581feda1 Author: Yun Gao AuthorDate: Sat May 7 18:20:29 2022 +0800 [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots --- .../org/apache/flink/cep/operator/CEPMigrationTest.java | 12 +--- .../cep-migration-after-branching-flink1.15-snapshot | Bin 0 -> 5806 bytes .../cep-migration-conditions-flink1.15-snapshot | Bin 0 -> 5453 bytes ...igration-single-pattern-afterwards-flink1.15-snapshot | Bin 0 -> 5203 bytes ...cep-migration-starting-new-pattern-flink1.15-snapshot | Bin 0 -> 5610 bytes 5 files changed, 1 insertion(+), 11 deletions(-) diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java index 7764810f510..05e8ba5212a 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java @@ -41,7 +41,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -72,16 +71,7 @@ public class CEPMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_6, -FlinkVersion.v1_7, -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_6, FlinkVersion.v1_15); } public CEPMigrationTest(FlinkVersion migrateVersion) { diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.15-snapshot new file mode 100644 index 000..a05be3af1a4 Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.15-snapshot differ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.15-snapshot new file mode 100644 index 000..7d308fb91c2 Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.15-snapshot differ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.15-snapshot new file mode 100644 index 000..89133ae521e Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.15-snapshot differ diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.15-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.15-snapshot new file mode 100644 index 000..4b956523c56 Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.15-snapshot differ
[flink] 01/13: [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 18b57b3689c6ef4084dd3e922af1e022e50db995 Author: Yun Gao AuthorDate: Sat May 7 12:06:39 2022 +0800 [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots --- .../context-state-serializer-1.15/serializer-snapshot| Bin 0 -> 126 bytes .../resources/context-state-serializer-1.15/test-data| Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 134 bytes .../transaction-state-serializer-1.15/test-data | Bin 0 -> 17 bytes .../writeable-serializer-1.15/serializer-snapshot| Bin 0 -> 187 bytes .../test/resources/writeable-serializer-1.15/test-data | Bin 0 -> 7 bytes .../common/typeutils/TypeSerializerUpgradeTestBase.java | 2 +- .../big-dec-serializer-1.15/serializer-snapshot | Bin 0 -> 94 bytes .../src/test/resources/big-dec-serializer-1.15/test-data | Bin 0 -> 24 bytes .../big-int-serializer-1.15/serializer-snapshot | Bin 0 -> 94 bytes .../src/test/resources/big-int-serializer-1.15/test-data | Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 130 bytes .../boolean-primitive-array-serializer-1.15/test-data| Bin 0 -> 6 bytes .../boolean-serializer-1.15/serializer-snapshot | Bin 0 -> 96 bytes .../src/test/resources/boolean-serializer-1.15/test-data | 1 + .../boolean-value-serializer-1.15/serializer-snapshot| Bin 0 -> 106 bytes .../resources/boolean-value-serializer-1.15/test-data| 1 + .../serializer-snapshot | Bin 0 -> 124 bytes .../byte-primitive-array-serializer-1.15/test-data | Bin 0 -> 14 bytes .../resources/byte-serializer-1.15/serializer-snapshot | Bin 0 -> 90 bytes .../src/test/resources/byte-serializer-1.15/test-data| 1 + .../byte-value-serializer-1.15/serializer-snapshot | Bin 0 -> 100 bytes .../test/resources/byte-value-serializer-1.15/test-data | 1 + .../serializer-snapshot | Bin 0 -> 124 bytes .../char-primitive-array-serializer-1.15/test-data | Bin 0 -> 24 bytes .../resources/char-serializer-1.15/serializer-snapshot | Bin 0 -> 90 bytes .../src/test/resources/char-serializer-1.15/test-data| 1 + .../char-value-serializer-1.15/serializer-snapshot | Bin 0 -> 100 bytes .../test/resources/char-value-serializer-1.15/test-data | Bin 0 -> 2 bytes .../copyable-value-serializer-1.15/serializer-snapshot | Bin 0 -> 199 bytes .../resources/copyable-value-serializer-1.15/test-data | Bin 0 -> 8 bytes .../resources/date-serializer-1.15/serializer-snapshot | Bin 0 -> 90 bytes .../src/test/resources/date-serializer-1.15/test-data| Bin 0 -> 8 bytes .../serializer-snapshot | Bin 0 -> 128 bytes .../double-primitive-array-serializer-1.15/test-data | Bin 0 -> 84 bytes .../resources/double-serializer-1.15/serializer-snapshot | Bin 0 -> 94 bytes .../src/test/resources/double-serializer-1.15/test-data | 1 + .../double-value-serializer-1.15/serializer-snapshot | Bin 0 -> 104 bytes .../resources/double-value-serializer-1.15/test-data | 1 + .../resources/either-serializer-1.15/serializer-snapshot | Bin 0 -> 276 bytes .../src/test/resources/either-serializer-1.15/test-data | 1 + .../resources/enum-serializer-1.15/serializer-snapshot | Bin 0 -> 188 bytes .../src/test/resources/enum-serializer-1.15/test-data| Bin 0 -> 4 bytes .../enum-serializerreconfig-1.15/serializer-snapshot | Bin 0 -> 163 bytes .../resources/enum-serializerreconfig-1.15/test-data | Bin 0 -> 4 bytes .../serializer-snapshot | Bin 0 -> 126 bytes .../float-primitive-array-serializer-1.15/test-data | Bin 0 -> 44 bytes .../resources/float-serializer-1.15/serializer-snapshot | Bin 0 -> 92 bytes .../src/test/resources/float-serializer-1.15/test-data | 1 + .../float-value-serializer-1.15/serializer-snapshot | Bin 0 -> 102 bytes .../test/resources/float-value-serializer-1.15/test-data | 1 + .../generic-array-serializer-1.15/serializer-snapshot| Bin 0 -> 211 bytes .../resources/generic-array-serializer-1.15/test-data| Bin 0 -> 19 bytes .../serializer-snapshot | Bin 0 -> 122 bytes .../int-primitive-array-serializer-1.15/test-data| Bin 0 -> 44 bytes .../resources/int-serializer-1.15/serializer-snapshot| Bin 0 -> 88 bytes .../src/test/resources/int-serializer-1.15/test-data | Bin 0 -> 4 bytes .../int-value-serializer-1.15/serializer-snapshot| Bin 0 -> 98 bytes .../test/resources/int-value-
[flink] branch 1.15_create created (now 3a30f2714b3)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git at 3a30f2714b3 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots This branch includes the following new commits: new 18b57b3689c [release] Upgrade current version to 1.15 for TypeSerializerUpgradeTestBase and create snapshots new c6a0d15891f [release] Upgrade current version to 1.15 for FlinkKafkaConsumerBaseMigrationTest and create snapshots new 78677f4f465 [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots new b3388a510e0 [release] Upgrade current version to 1.15 for FlinkKinesisConsumerMigrationTest and create snapshots new 5cae5c5a17a [release] Upgrade current version to 1.15 for ContinuousFileProcessingMigrationTest and create snapshots new 3b83229944a [release] Upgrade current version to 1.15 for CEPMigrationTest and create snapshots new ab28f7430f4 [release] Upgrade current version to 1.15 for WindowOperatorMigrationTest and create snapshots new 8223a11c5bb [release] Upgrade current version to 1.15 for StatefulJobSnapshotMigrationITCase and create snapshots new b8618c21c33 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots new d1b47bbdc79 [release] Upgrade current version to 1.15 for TypeSerializerSnapshotMigrationITCase and create snapshots new 68cc19f78cb [release] Upgrade current version to 1.15 for AbstractOperatorRestoreTestBase and create snapshots new f2e75c591b4 [release] Upgrade current version to 1.15 for StatefulJobSavepointMigrationITCase and create snapshots new 3a30f2714b3 [release] Upgrade current version to 1.15 for StatefulJobWBroadcastStateMigrationITCase and create snapshots The 13 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.
[flink] 03/13: [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch 1.15_create in repository https://gitbox.apache.org/repos/asf/flink.git commit 78677f4f4653923c1f3b8d65e8e5283ce37e92e8 Author: Yun Gao AuthorDate: Sat May 7 14:07:27 2022 +0800 [release] Upgrade current version to 1.15 for FlinkKafkaProducerMigrationTest and create snapshots --- .../kafka/FlinkKafkaProducerMigrationTest.java | 10 +- .../kafka-migration-kafka-producer-flink-1.15-snapshot | Bin 0 -> 1242 bytes 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java index 67a74879c64..754d28dff48 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.Arrays; import java.util.Collection; import java.util.Properties; @@ -43,14 +42,7 @@ import java.util.Properties; public class FlinkKafkaProducerMigrationTest extends KafkaMigrationTestBase { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { -return Arrays.asList( -FlinkVersion.v1_8, -FlinkVersion.v1_9, -FlinkVersion.v1_10, -FlinkVersion.v1_11, -FlinkVersion.v1_12, -FlinkVersion.v1_13, -FlinkVersion.v1_14); +return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_15); } public FlinkKafkaProducerMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot new file mode 100644 index 000..35af28dd7b1 Binary files /dev/null and b/flink-connectors/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.15-snapshot differ
[flink] branch master updated (6ebdde3b124 -> 3e1874bf05d)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/flink.git from 6ebdde3b124 [FLINK-25970][core] Add type of original exception to the detailMessage of SerializedThrowable. add 3e1874bf05d [release] Change the stable doc versions No new revisions were added by this update. Summary of changes: .github/workflows/docs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-)
[flink] branch release-1.14 updated: [release] Show outdated warning for 1.14 docs
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a commit to branch release-1.14 in repository https://gitbox.apache.org/repos/asf/flink.git The following commit(s) were added to refs/heads/release-1.14 by this push: new 6d034b432fe [release] Show outdated warning for 1.14 docs 6d034b432fe is described below commit 6d034b432fe31502749b607d0892250731b7c24d Author: Yun Gao AuthorDate: Fri May 6 11:30:45 2022 +0800 [release] Show outdated warning for 1.14 docs --- docs/config.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/config.toml b/docs/config.toml index 964ae064d83..54c0251fac0 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -27,7 +27,7 @@ pygmentsUseClasses = true IsStable = true # Flag to indicate whether an outdated warning should be shown. - ShowOutDatedWarning = false + ShowOutDatedWarning = true # This are the version referenced in the docs. Please only use these variables # to reference a specific Flink version, because this is the only place where
[flink-web] branch asf-site updated (528cc5f44 -> 16dbed40c)
This is an automated email from the ASF dual-hosted git repository. gaoyunhaii pushed a change to branch asf-site in repository https://gitbox.apache.org/repos/asf/flink-web.git from 528cc5f44 Rebuild website add b568ef8bf Fix the stable version new 16dbed40c Rebuild website The 1 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: _config.yml| 6 +++--- content/2019/05/03/pulsar-flink.html | 4 ++-- content/2019/05/14/temporal-tables.html| 4 ++-- content/2019/05/19/state-ttl.html | 4 ++-- content/2019/06/05/flink-network-stack.html| 4 ++-- content/2019/06/26/broadcast-state.html| 4 ++-- content/2019/07/23/flink-network-stack-2.html | 4 ++-- content/2020/04/09/pyflink-udf-support-flink.html | 4 ++-- content/2020/07/23/catalogs.html | 4 ++-- .../28/flink-sql-demo-building-e2e-streaming-application.html | 4 ++-- content/2020/08/04/pyflink-pandas-udf-support-flink.html | 4 ++-- content/2020/08/19/statefun.html | 4 ++-- .../2020/09/01/flink-1.11-memory-management-improvements.html | 4 ++-- .../10/15/from-aligned-to-unaligned-checkpoints-part-1.html| 4 ++-- content/2020/12/15/pipelined-region-sheduling.html | 4 ++-- content/2021/01/07/pulsar-flink-connector-270.html | 4 ++-- content/2021/01/18/rocksdb.html| 4 ++-- content/2021/02/10/native-k8s-with-ha.html | 4 ++-- content/2021/03/11/batch-execution-mode.html | 4 ++-- content/2021/05/06/reactive-mode.html | 4 ++-- content/2021/07/07/backpressure.html | 4 ++-- content/2021/09/07/connector-table-sql-api-part1.html | 4 ++-- content/2021/09/07/connector-table-sql-api-part2.html | 4 ++-- content/2021/10/26/sort-shuffle-part1.html | 4 ++-- content/2021/10/26/sort-shuffle-part2.html | 4 ++-- content/2021/11/03/flink-backward.html | 4 ++-- content/2021/12/10/log4j-cve.html | 4 ++-- content/2022/01/04/scheduler-performance-part-one.html | 4 ++-- content/2022/01/04/scheduler-performance-part-two.html | 4 ++-- content/2022/01/20/pravega-connector-101.html | 4 ++-- content/2022/02/22/scala-free.html | 4 ++-- content/blog/feed.xml | 6 +++--- content/blog/index.html| 4 ++-- content/blog/page10/index.html | 4 ++-- content/blog/page11/index.html | 4 ++-- content/blog/page12/index.html | 4 ++-- content/blog/page13/index.html | 4 ++-- content/blog/page14/index.html | 4 ++-- content/blog/page15/index.html | 4 ++-- content/blog/page16/index.html | 4 ++-- content/blog/page17/index.html | 4 ++-- content/blog/page18/index.html | 4 ++-- content/blog/page2/index.html | 4 ++-- content/blog/page3/index.html | 4 ++-- content/blog/page4/index.html | 4 ++-- content/blog/page5/index.html | 4 ++-- content/blog/page6/index.html | 4 ++-- content/blog/page7/index.html | 4 ++-- content/blog/page8/index.html | 4 ++-- content/blog/page9/index.html | 4 ++-- content/blog/release_1.0.0-changelog_known_issues.html | 4 ++-- content/blog/release_1.1.0-changelog.html | 4 ++-- content/blog/release_1.2.0-changelog.html | 4 ++-- content/blog/release_1.3.0-changelog.html | 4 ++-- content/community.html | 6 +++--- content/contributing/code-style-and-quality-common.html| 4 ++-- content/contributing/code-style-and-quality-components.html| 4 ++-- content/contributing/code-style-and-quality-formatting.html| 4 ++-- content/contributing/code-style-and-quality-java.html | 4 ++-- content/contributing/code-style-and-quality-pre