(flink) branch release-1.18 updated: [FLINK-35217] Add missing fsync to #closeForCommit in some subclasses of RecoverableFsDataOutputStream. (#24722) (#24752)

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

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


The following commit(s) were added to refs/heads/release-1.18 by this push:
 new e6726d3b962 [FLINK-35217] Add missing fsync to #closeForCommit in some 
subclasses of RecoverableFsDataOutputStream. (#24722) (#24752)
e6726d3b962 is described below

commit e6726d3b962383d9a2576fe117d7566b205f514a
Author: Stefan Richter 
AuthorDate: Tue Apr 30 22:25:38 2024 +0200

[FLINK-35217] Add missing fsync to #closeForCommit in some subclasses of 
RecoverableFsDataOutputStream. (#24722) (#24752)

(cherry picked from commit 80af4d502318348ba15a8f75a2a622ce9dbdc968)
---
 ...erFromPersistRecoverableFsDataOutputStream.java |  59 +++
 .../local/LocalRecoverableFsDataOutputStream.java  |  23 ++-
 .../AbstractRecoverableFsDataOutputStreamTest.java |  98 +++
 .../LocalRecoverableFsDataOutputStreamTest.java| 188 +
 .../AzureBlobFsRecoverableDataOutputStream.java|  17 +-
 ...AzureBlobFsRecoverableDataOutputStreamTest.java | 100 +++
 .../BaseHadoopFsRecoverableFsDataOutputStream.java |  12 +-
 .../hdfs/HadoopRecoverableFsDataOutputStream.java  |  17 +-
 .../HadoopRecoverableFsDataOutputStreamTest.java   |  89 ++
 9 files changed, 579 insertions(+), 24 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
new file mode 100644
index 000..12e3fbc7f4d
--- /dev/null
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
@@ -0,0 +1,59 @@
+/*
+ * 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.core.fs;
+
+import java.io.IOException;
+
+/**
+ * {@link RecoverableFsDataOutputStream} with fixed implementation of {@link 
#closeForCommit()} that
+ * is based on using {@link #persist()} to ensure durability and creates the 
{@link
+ * org.apache.flink.core.fs.RecoverableFsDataOutputStream.Committer} from the 
corresponding {@link
+ * org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable}.
+ *
+ * @param  return type of #persist()
+ */
+public abstract class CommitterFromPersistRecoverableFsDataOutputStream<
+RESUME_RECOVERABLE extends RecoverableWriter.ResumeRecoverable>
+extends RecoverableFsDataOutputStream {
+
+/** @see RecoverableFsDataOutputStream#persist() */
+@Override
+public abstract RESUME_RECOVERABLE persist() throws IOException;
+
+/**
+ * @see RecoverableFsDataOutputStream#closeForCommit()
+ * @param recoverable a resume recoverable to create the committer from. 
Typically the parameter
+ * is the return value of {@link #persist()}.
+ * @return the committer created from recoverable.
+ */
+protected abstract Committer createCommitterFromResumeRecoverable(
+RESUME_RECOVERABLE recoverable);
+
+/**
+ * @see RecoverableFsDataOutputStream#closeForCommit()
+ * @implNote Calls persist to ensure durability of the written data and 
creates a committer
+ * object from the return value of {@link #persist()}.
+ */
+@Override
+public final Committer closeForCommit() throws IOException {
+Committer committer = createCommitterFromResumeRecoverable(persist());
+close();
+return committer;
+}
+}
diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
index cc9c88fc4f4..c273c31960e 100644
--- 
a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
@@ -19,9 +19,10 @@
 package org.apache.flink.core.fs.local;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.

(flink) branch release-1.19 updated: [FLINK-35217] Add missing fsync to #closeForCommit in some subclasses of RecoverableFsDataOutputStream. (#24722) (#24751)

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

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


The following commit(s) were added to refs/heads/release-1.19 by this push:
 new ac4aa35c6e2 [FLINK-35217] Add missing fsync to #closeForCommit in some 
subclasses of RecoverableFsDataOutputStream. (#24722) (#24751)
ac4aa35c6e2 is described below

commit ac4aa35c6e2e2da87760ffbf45d85888b1976c2f
Author: Stefan Richter 
AuthorDate: Tue Apr 30 22:25:27 2024 +0200

[FLINK-35217] Add missing fsync to #closeForCommit in some subclasses of 
RecoverableFsDataOutputStream. (#24722) (#24751)

(cherry picked from commit 80af4d502318348ba15a8f75a2a622ce9dbdc968)
---
 ...erFromPersistRecoverableFsDataOutputStream.java |  59 +++
 .../local/LocalRecoverableFsDataOutputStream.java  |  23 ++-
 .../AbstractRecoverableFsDataOutputStreamTest.java |  98 +++
 .../LocalRecoverableFsDataOutputStreamTest.java| 188 +
 .../AzureBlobFsRecoverableDataOutputStream.java|  17 +-
 ...AzureBlobFsRecoverableDataOutputStreamTest.java | 100 +++
 .../BaseHadoopFsRecoverableFsDataOutputStream.java |  12 +-
 .../hdfs/HadoopRecoverableFsDataOutputStream.java  |  17 +-
 .../HadoopRecoverableFsDataOutputStreamTest.java   |  89 ++
 9 files changed, 579 insertions(+), 24 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
new file mode 100644
index 000..12e3fbc7f4d
--- /dev/null
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
@@ -0,0 +1,59 @@
+/*
+ * 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.core.fs;
+
+import java.io.IOException;
+
+/**
+ * {@link RecoverableFsDataOutputStream} with fixed implementation of {@link 
#closeForCommit()} that
+ * is based on using {@link #persist()} to ensure durability and creates the 
{@link
+ * org.apache.flink.core.fs.RecoverableFsDataOutputStream.Committer} from the 
corresponding {@link
+ * org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable}.
+ *
+ * @param  return type of #persist()
+ */
+public abstract class CommitterFromPersistRecoverableFsDataOutputStream<
+RESUME_RECOVERABLE extends RecoverableWriter.ResumeRecoverable>
+extends RecoverableFsDataOutputStream {
+
+/** @see RecoverableFsDataOutputStream#persist() */
+@Override
+public abstract RESUME_RECOVERABLE persist() throws IOException;
+
+/**
+ * @see RecoverableFsDataOutputStream#closeForCommit()
+ * @param recoverable a resume recoverable to create the committer from. 
Typically the parameter
+ * is the return value of {@link #persist()}.
+ * @return the committer created from recoverable.
+ */
+protected abstract Committer createCommitterFromResumeRecoverable(
+RESUME_RECOVERABLE recoverable);
+
+/**
+ * @see RecoverableFsDataOutputStream#closeForCommit()
+ * @implNote Calls persist to ensure durability of the written data and 
creates a committer
+ * object from the return value of {@link #persist()}.
+ */
+@Override
+public final Committer closeForCommit() throws IOException {
+Committer committer = createCommitterFromResumeRecoverable(persist());
+close();
+return committer;
+}
+}
diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
index cc9c88fc4f4..c273c31960e 100644
--- 
a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
@@ -19,9 +19,10 @@
 package org.apache.flink.core.fs.local;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.

(flink) branch master updated: [FLINK-35217] Add missing fsync to #closeForCommit in some subclasses of RecoverableFsDataOutputStream. (#24722)

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

srichter 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 80af4d50231 [FLINK-35217] Add missing fsync to #closeForCommit in some 
subclasses of RecoverableFsDataOutputStream. (#24722)
80af4d50231 is described below

commit 80af4d502318348ba15a8f75a2a622ce9dbdc968
Author: Stefan Richter 
AuthorDate: Tue Apr 30 13:36:11 2024 +0200

[FLINK-35217] Add missing fsync to #closeForCommit in some subclasses of 
RecoverableFsDataOutputStream. (#24722)
---
 ...erFromPersistRecoverableFsDataOutputStream.java |  59 +++
 .../local/LocalRecoverableFsDataOutputStream.java  |  23 ++-
 .../AbstractRecoverableFsDataOutputStreamTest.java |  97 +++
 .../LocalRecoverableFsDataOutputStreamTest.java| 188 +
 .../AzureBlobFsRecoverableDataOutputStream.java|  17 +-
 ...AzureBlobFsRecoverableDataOutputStreamTest.java | 100 +++
 .../BaseHadoopFsRecoverableFsDataOutputStream.java |  12 +-
 .../hdfs/HadoopRecoverableFsDataOutputStream.java  |  17 +-
 .../HadoopRecoverableFsDataOutputStreamTest.java   |  89 ++
 9 files changed, 578 insertions(+), 24 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
new file mode 100644
index 000..12e3fbc7f4d
--- /dev/null
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/CommitterFromPersistRecoverableFsDataOutputStream.java
@@ -0,0 +1,59 @@
+/*
+ * 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.core.fs;
+
+import java.io.IOException;
+
+/**
+ * {@link RecoverableFsDataOutputStream} with fixed implementation of {@link 
#closeForCommit()} that
+ * is based on using {@link #persist()} to ensure durability and creates the 
{@link
+ * org.apache.flink.core.fs.RecoverableFsDataOutputStream.Committer} from the 
corresponding {@link
+ * org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable}.
+ *
+ * @param  return type of #persist()
+ */
+public abstract class CommitterFromPersistRecoverableFsDataOutputStream<
+RESUME_RECOVERABLE extends RecoverableWriter.ResumeRecoverable>
+extends RecoverableFsDataOutputStream {
+
+/** @see RecoverableFsDataOutputStream#persist() */
+@Override
+public abstract RESUME_RECOVERABLE persist() throws IOException;
+
+/**
+ * @see RecoverableFsDataOutputStream#closeForCommit()
+ * @param recoverable a resume recoverable to create the committer from. 
Typically the parameter
+ * is the return value of {@link #persist()}.
+ * @return the committer created from recoverable.
+ */
+protected abstract Committer createCommitterFromResumeRecoverable(
+RESUME_RECOVERABLE recoverable);
+
+/**
+ * @see RecoverableFsDataOutputStream#closeForCommit()
+ * @implNote Calls persist to ensure durability of the written data and 
creates a committer
+ * object from the return value of {@link #persist()}.
+ */
+@Override
+public final Committer closeForCommit() throws IOException {
+Committer committer = createCommitterFromResumeRecoverable(persist());
+close();
+return committer;
+}
+}
diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
index cc9c88fc4f4..c273c31960e 100644
--- 
a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableFsDataOutputStream.java
@@ -19,9 +19,10 @@
 package org.apache.flink.core.fs.local;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import 
org.apache.flink.core.fs.CommitterFromPersistRecoverableF

(flink) branch master updated: [FLINK-34546] Emit span with failure labels on failure in AdaptiveScheduler. (#24498)

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

srichter 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 10bff3dbad1 [FLINK-34546] Emit span with failure labels on failure in 
AdaptiveScheduler. (#24498)
10bff3dbad1 is described below

commit 10bff3dbad103b60915be817a3408820ed09b6cf
Author: Stefan Richter 
AuthorDate: Fri Mar 15 09:36:43 2024 +0100

[FLINK-34546] Emit span with failure labels on failure in 
AdaptiveScheduler. (#24498)
---
 .../failover/ExecutionFailureHandler.java  | 32 ++---
 .../scheduler/adaptive/AdaptiveScheduler.java  | 22 +-
 .../runtime/scheduler/adaptive/Canceling.java  |  4 +-
 .../runtime/scheduler/adaptive/Executing.java  | 10 ++-
 .../flink/runtime/scheduler/adaptive/Failing.java  |  4 +-
 .../adaptive/JobFailureMetricReporter.java | 84 ++
 .../runtime/scheduler/adaptive/Restarting.java |  4 +-
 .../adaptive/StateWithExecutionGraph.java  |  7 +-
 .../scheduler/adaptive/StopWithSavepoint.java  |  9 ++-
 .../failover/ExecutionFailureHandlerTest.java  |  4 +-
 .../scheduler/adaptive/AdaptiveSchedulerTest.java  | 78 +++-
 .../runtime/scheduler/adaptive/ExecutingTest.java  |  3 +-
 .../adaptive/StateWithExecutionGraphTest.java  |  2 +-
 .../scheduler/adaptive/StopWithSavepointTest.java  | 13 +++-
 14 files changed, 228 insertions(+), 48 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
index 3d36a9e6bff..94130bc2f5f 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
@@ -26,13 +26,12 @@ import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.failure.FailureEnricherUtils;
+import org.apache.flink.runtime.scheduler.adaptive.JobFailureMetricReporter;
 import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
 import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex;
 import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
 import org.apache.flink.runtime.throwable.ThrowableClassifier;
 import org.apache.flink.runtime.throwable.ThrowableType;
-import org.apache.flink.traces.Span;
-import org.apache.flink.traces.SpanBuilder;
 import org.apache.flink.util.IterableUtils;
 
 import javax.annotation.Nullable;
@@ -70,8 +69,8 @@ public class ExecutionFailureHandler {
 private final Collection failureEnrichers;
 private final ComponentMainThreadExecutor mainThreadExecutor;
 private final MetricGroup metricGroup;
-
 private final boolean reportEventsAsSpans;
+private final JobFailureMetricReporter jobFailureMetricReporter;
 
 /**
  * Creates the handler to deal with task failures.
@@ -105,6 +104,7 @@ public class ExecutionFailureHandler {
 this.globalFailureCtx = globalFailureCtx;
 this.metricGroup = metricGroup;
 this.reportEventsAsSpans = 
jobMasterConfig.get(TraceOptions.REPORT_EVENTS_AS_SPANS);
+this.jobFailureMetricReporter = new 
JobFailureMetricReporter(metricGroup);
 }
 
 /**
@@ -171,35 +171,15 @@ public class ExecutionFailureHandler {
 failureHandlingResult
 .getFailureLabels()
 .thenAcceptAsync(
-labels -> 
reportFailureHandling(failureHandlingResult, labels),
+labels ->
+jobFailureMetricReporter.reportJobFailure(
+failureHandlingResult, labels),
 mainThreadExecutor);
 }
 
 return failureHandlingResult;
 }
 
-private void reportFailureHandling(
-FailureHandlingResult failureHandlingResult, Map 
failureLabels) {
-
-// Add base attributes
-SpanBuilder spanBuilder =
-Span.builder(ExecutionFailureHandler.class, "JobFailure")
-.setStartTsMillis(failureHandlingResult.getTimestamp())
-.setEndTsMillis(failureHandlingResult.getTimestamp())
-.setAttribute(
-"canRestart", 
String.valueOf(failureHandlingResult.canRestart()))
-.setAttribute(
-"isGlobalFailure",
-
String.valueOf(failureHandlingResult.isGlobalFailure()));

(flink) branch master updated: [FLINK-34546] Emit span with failure labels on failure.

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

srichter 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 7c8e3f5a0c3 [FLINK-34546] Emit span with failure labels on failure.
7c8e3f5a0c3 is described below

commit 7c8e3f5a0c39f9a82c5549925035344c5d27cb98
Author: Stefan Richter 
AuthorDate: Thu Feb 29 10:26:02 2024 +0100

[FLINK-34546] Emit span with failure labels on failure.
---
 .../apache/flink/configuration/TraceOptions.java   | 14 +
 .../failover/ExecutionFailureHandler.java  | 65 +-
 .../flink/runtime/scheduler/DefaultScheduler.java  |  4 +-
 .../failover/ExecutionFailureHandlerTest.java  | 39 -
 4 files changed, 117 insertions(+), 5 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/TraceOptions.java 
b/flink-core/src/main/java/org/apache/flink/configuration/TraceOptions.java
index 1aee746e210..a7e84192dea 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/TraceOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/TraceOptions.java
@@ -56,6 +56,20 @@ public class TraceOptions {
 + " any of the names in the list will be 
started. Otherwise, all reporters that could be found in"
 + " the configuration will be started.");
 
+/**
+ * Temporary option to report events as span. This option will be removed 
once we support
+ * reporting events.
+ */
+@Deprecated
+public static final ConfigOption REPORT_EVENTS_AS_SPANS =
+key("traces.report-events-as-spans")
+.booleanType()
+.defaultValue(false)
+.withDescription(
+"Whether to report events as spans. This is a 
temporary parameter that "
++ "is in place until we have support for 
reporting events. "
++ "In the meantime, this can be activated 
to report them as spans instead.");
+
 /**
  * Returns a view over the given configuration via which options can be 
set/retrieved for the
  * given reporter.
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
index aed330de522..3d36a9e6bff 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/ExecutionFailureHandler.java
@@ -17,8 +17,11 @@
 
 package org.apache.flink.runtime.executiongraph.failover;
 
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.TraceOptions;
 import org.apache.flink.core.failure.FailureEnricher;
 import org.apache.flink.core.failure.FailureEnricher.Context;
+import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
 import org.apache.flink.runtime.executiongraph.Execution;
@@ -28,6 +31,8 @@ import 
org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex;
 import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
 import org.apache.flink.runtime.throwable.ThrowableClassifier;
 import org.apache.flink.runtime.throwable.ThrowableType;
+import org.apache.flink.traces.Span;
+import org.apache.flink.traces.SpanBuilder;
 import org.apache.flink.util.IterableUtils;
 
 import javax.annotation.Nullable;
@@ -47,6 +52,8 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class ExecutionFailureHandler {
 
+public static final String FAILURE_LABEL_ATTRIBUTE_PREFIX = 
"failureLabel.";
+
 private final SchedulingTopology schedulingTopology;
 
 /** Strategy to judge which tasks should be restarted. */
@@ -62,6 +69,9 @@ public class ExecutionFailureHandler {
 private final Context globalFailureCtx;
 private final Collection failureEnrichers;
 private final ComponentMainThreadExecutor mainThreadExecutor;
+private final MetricGroup metricGroup;
+
+private final boolean reportEventsAsSpans;
 
 /**
  * Creates the handler to deal with task failures.
@@ -76,13 +86,15 @@ public class ExecutionFailureHandler {
  * @param globalFailureCtx Global failure Context used by FailureEnrichers
  */
 public ExecutionFailureHandler(
+final Configuration jobMasterConfig,
 final SchedulingTopology schedulingTopology,
 final FailoverStrategy failoverStrategy,
 final RestartBackoffTimeStrategy rest

(flink) 01/02: [hotfix] Fix configuration through TernaryBoolean in EmbeddedRocksDBStateBackend.

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

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

commit 5be8b3e1dfdb5a009677b2e194def2ad3b1122ec
Author: Stefan Richter 
AuthorDate: Tue Feb 27 10:48:40 2024 +0100

[hotfix] Fix configuration through TernaryBoolean in 
EmbeddedRocksDBStateBackend.
---
 .../java/org/apache/flink/util/TernaryBoolean.java | 18 +++
 .../state/EmbeddedRocksDBStateBackend.java | 21 
 .../state/EmbeddedRocksDBStateBackendTest.java | 57 --
 .../state/RocksDBStateBackendConfigTest.java   | 25 ++
 4 files changed, 107 insertions(+), 14 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java 
b/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java
index 4cfd71ac238..4f06c15e9f8 100644
--- a/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java
+++ b/flink-core/src/main/java/org/apache/flink/util/TernaryBoolean.java
@@ -19,6 +19,8 @@
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
 
 import javax.annotation.Nullable;
 
@@ -71,6 +73,22 @@ public enum TernaryBoolean {
 return this == UNDEFINED ? null : (this == TRUE ? Boolean.TRUE : 
Boolean.FALSE);
 }
 
+/**
+ * Merges an existing value with a config, accepting the config's value 
only if the existing
+ * value is undefined.
+ *
+ * @param original the value to merge with the config.
+ * @param configOption the config option to merge with from the config.
+ * @param config the config to merge with.
+ */
+public static TernaryBoolean mergeTernaryBooleanWithConfig(
+TernaryBoolean original, ConfigOption configOption, 
ReadableConfig config) {
+if (original != TernaryBoolean.UNDEFINED) {
+return original;
+}
+return 
TernaryBoolean.fromBoxedBoolean(config.getOptional(configOption).orElse(null));
+}
+
 // 
 
 /**
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
index 1a489fad8c8..baca5f4a013 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
@@ -321,21 +321,20 @@ public class EmbeddedRocksDBStateBackend extends 
AbstractManagedMemoryStateBacke
 "Overlap fraction threshold of restoring should be between 0 
and 1");
 
 incrementalRestoreAsyncCompactAfterRescale =
-original.incrementalRestoreAsyncCompactAfterRescale == 
TernaryBoolean.UNDEFINED
-? TernaryBoolean.fromBoxedBoolean(
-
config.get(INCREMENTAL_RESTORE_ASYNC_COMPACT_AFTER_RESCALE))
-: original.incrementalRestoreAsyncCompactAfterRescale;
+TernaryBoolean.mergeTernaryBooleanWithConfig(
+original.incrementalRestoreAsyncCompactAfterRescale,
+INCREMENTAL_RESTORE_ASYNC_COMPACT_AFTER_RESCALE,
+config);
 
 useIngestDbRestoreMode =
-original.useIngestDbRestoreMode == TernaryBoolean.UNDEFINED
-? 
TernaryBoolean.fromBoxedBoolean(config.get(USE_INGEST_DB_RESTORE_MODE))
-: 
TernaryBoolean.fromBoolean(original.getUseIngestDbRestoreMode());
+TernaryBoolean.mergeTernaryBooleanWithConfig(
+original.useIngestDbRestoreMode, 
USE_INGEST_DB_RESTORE_MODE, config);
 
 rescalingUseDeleteFilesInRange =
-original.rescalingUseDeleteFilesInRange == 
TernaryBoolean.UNDEFINED
-? TernaryBoolean.fromBoxedBoolean(
-
config.get(USE_DELETE_FILES_IN_RANGE_DURING_RESCALING))
-: original.rescalingUseDeleteFilesInRange;
+TernaryBoolean.mergeTernaryBooleanWithConfig(
+original.rescalingUseDeleteFilesInRange,
+USE_DELETE_FILES_IN_RANGE_DURING_RESCALING,
+config);
 
 this.rocksDBMemoryFactory = original.rocksDBMemoryFactory;
 }
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest.java
 
b/flink-state-backends/flink-st

(flink) 02/02: [hotfix] Reduce code duplication and add logging in RocksDBIncrementalCheckpointUtils#deleteRangeData.

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

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

commit e2b6588ef39cbeb1825cacf553a80272ae7b9418
Author: Stefan Richter 
AuthorDate: Tue Feb 27 11:54:28 2024 +0100

[hotfix] Reduce code duplication and add logging in 
RocksDBIncrementalCheckpointUtils#deleteRangeData.
---
 .../state/RocksDBIncrementalCheckpointUtils.java   | 66 +++---
 1 file changed, 46 insertions(+), 20 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
index bdb94466de9..67bf02cd31a 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
@@ -29,21 +29,28 @@ import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.LiveFileMetaData;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nonnegative;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 /** Utils for RocksDB Incremental Checkpoint. */
 public class RocksDBIncrementalCheckpointUtils {
 
+private static final Logger logger =
+LoggerFactory.getLogger(RocksDBIncrementalCheckpointUtils.class);
+
 /**
  * Evaluates state handle's "score" regarding the target range when 
choosing the best state
  * handle to init the initial db for recovery, if the overlap fraction is 
less than
@@ -112,7 +119,7 @@ public class RocksDBIncrementalCheckpointUtils {
  * @param targetKeyGroupRange the target key group range.
  * @param currentKeyGroupRange the key group range of the db instance.
  * @param keyGroupPrefixBytes Number of bytes required to prefix the key 
groups.
- * @param useDeleteFilesInRange Whether to use deleteFilesInRange to clean 
up redundant files.
+ * @param useDeleteFilesInRange whether to call db.deleteFilesInRanges for 
the deleted ranges.
  */
 public static void clipDBWithKeyGroupRange(
 @Nonnull RocksDB db,
@@ -122,31 +129,45 @@ public class RocksDBIncrementalCheckpointUtils {
 @Nonnegative int keyGroupPrefixBytes,
 boolean useDeleteFilesInRange)
 throws RocksDBException {
-List deletedRanges = new ArrayList<>(4);
+
+List deleteFilesRanges = new ArrayList<>(4);
 
 if (currentKeyGroupRange.getStartKeyGroup() < 
targetKeyGroupRange.getStartKeyGroup()) {
-final byte[] beginKeyGroupBytes = new byte[keyGroupPrefixBytes];
-final byte[] endKeyGroupBytes = new byte[keyGroupPrefixBytes];
-CompositeKeySerializationUtils.serializeKeyGroup(
-currentKeyGroupRange.getStartKeyGroup(), 
beginKeyGroupBytes);
-CompositeKeySerializationUtils.serializeKeyGroup(
-targetKeyGroupRange.getStartKeyGroup(), endKeyGroupBytes);
-deletedRanges.add(beginKeyGroupBytes);
-deletedRanges.add(endKeyGroupBytes);
+prepareRangeDeletes(
+keyGroupPrefixBytes,
+currentKeyGroupRange.getStartKeyGroup(),
+targetKeyGroupRange.getStartKeyGroup(),
+deleteFilesRanges);
 }
 
 if (currentKeyGroupRange.getEndKeyGroup() > 
targetKeyGroupRange.getEndKeyGroup()) {
-final byte[] beginKeyGroupBytes = new byte[keyGroupPrefixBytes];
-final byte[] endKeyGroupBytes = new byte[keyGroupPrefixBytes];
-CompositeKeySerializationUtils.serializeKeyGroup(
-targetKeyGroupRange.getEndKeyGroup() + 1, 
beginKeyGroupBytes);
-CompositeKeySerializationUtils.serializeKeyGroup(
-currentKeyGroupRange.getEndKeyGroup() + 1, 
endKeyGroupBytes);
-deletedRanges.add(beginKeyGroupBytes);
-deletedRanges.add(endKeyGroupBytes);
+prepareRangeDeletes(
+keyGroupPrefixBytes,
+targetKeyGroupRange.getEndKeyGroup() + 1,
+currentKeyGroupRange.getEndKeyGroup() + 1,
+deleteFilesRanges);
 }
 
-deleteRangeData(db, columnFamilyHandles, deletedRanges, 
useDeleteFilesIn

(flink) branch master updated (b9069227098 -> e2b6588ef39)

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

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


from b9069227098 [FLINK-34454][doc/annotation] Use claim mode instead of 
restore mode everywhere
 new 5be8b3e1dfd [hotfix] Fix configuration through TernaryBoolean in 
EmbeddedRocksDBStateBackend.
 new e2b6588ef39 [hotfix] Reduce code duplication and add logging in 
RocksDBIncrementalCheckpointUtils#deleteRangeData.

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:
 .../java/org/apache/flink/util/TernaryBoolean.java | 18 ++
 .../state/EmbeddedRocksDBStateBackend.java | 21 ---
 .../state/RocksDBIncrementalCheckpointUtils.java   | 66 +++---
 .../state/EmbeddedRocksDBStateBackendTest.java | 57 ++-
 .../state/RocksDBStateBackendConfigTest.java   | 25 
 5 files changed, 153 insertions(+), 34 deletions(-)



(flink) 01/02: [FLINK-34050][state] Clean up useless files in deleted ranges during restore for RocksdbStateBackend

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

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

commit 534ea31e7ef8c7aa906ed4226f4f55ec0abc3fb2
Author: Jinzhong Li 
AuthorDate: Mon Jan 15 18:00:34 2024 +0800

[FLINK-34050][state] Clean up useless files in deleted ranges during 
restore for RocksdbStateBackend
---
 .../contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java| 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
index 99b97ef5164..6409840ed4d 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
@@ -33,6 +33,7 @@ import javax.annotation.Nonnegative;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -153,13 +154,14 @@ public class RocksDBIncrementalCheckpointUtils {
 byte[] beginKeyBytes,
 byte[] endKeyBytes)
 throws RocksDBException {
-
+List deletedRange = Arrays.asList(beginKeyBytes, endKeyBytes);
 for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) {
 // Using RocksDB's deleteRange will take advantage of delete
 // tombstones, which mark the range as deleted.
 //
 // 
https://github.com/ververica/frocksdb/blob/FRocksDB-6.20.3/include/rocksdb/db.h#L363-L377
 db.deleteRange(columnFamilyHandle, beginKeyBytes, endKeyBytes);
+db.deleteFilesInRanges(columnFamilyHandle, deletedRange, false);
 }
 }
 



(flink) 02/02: [FLINK-34050][state] Introduce parameter protection for DeleteFilesInRange

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

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

commit b048a9cd37d0f202117bfbb74f2991879223c160
Author: Jinzhong Li 
AuthorDate: Mon Feb 26 20:08:56 2024 +0800

[FLINK-34050][state] Introduce parameter protection for DeleteFilesInRange
---
 .../rocksdb_configurable_configuration.html|  6 +++
 .../state/EmbeddedRocksDBStateBackend.java | 22 +++-
 .../state/RocksDBConfigurableOptions.java  |  7 +++
 .../state/RocksDBIncrementalCheckpointUtils.java   | 59 ++
 .../state/RocksDBKeyedStateBackendBuilder.java | 13 -
 .../RocksDBIncrementalRestoreOperation.java|  9 +++-
 .../RocksDBIncrementalCheckpointUtilsTest.java |  5 +-
 .../streaming/state/RocksDBRecoveryTest.java   |  1 +
 8 files changed, 97 insertions(+), 25 deletions(-)

diff --git 
a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html 
b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
index 4ec59a27110..bafad2db137 100644
--- a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
+++ b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
@@ -104,6 +104,12 @@
 MemorySize
 The maximum size of RocksDB's file used for information 
logging. If the log files becomes larger than this, a new file will be created. 
If 0, all logs will be written to one log file. The default maximum file size 
is '25MB'. 
 
+
+
state.backend.rocksdb.rescaling.use-delete-files-in-range
+false
+Boolean
+If true, during rescaling, the deleteFilesInRange API will be 
invoked to clean up the useless files so that local disk space can be reclaimed 
more promptly.
+
 
 
state.backend.rocksdb.restore-overlap-fraction-threshold
 0.0
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
index 0c5e748772e..1a489fad8c8 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
@@ -72,6 +72,7 @@ import java.util.function.Supplier;
 import static org.apache.flink.configuration.description.TextElement.text;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.INCREMENTAL_RESTORE_ASYNC_COMPACT_AFTER_RESCALE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.RESTORE_OVERLAP_FRACTION_THRESHOLD;
+import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.USE_DELETE_FILES_IN_RANGE_DURING_RESCALING;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.USE_INGEST_DB_RESTORE_MODE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.WRITE_BATCH_SIZE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBOptions.CHECKPOINT_TRANSFER_THREAD_NUM;
@@ -184,6 +185,12 @@ public class EmbeddedRocksDBStateBackend extends 
AbstractManagedMemoryStateBacke
  */
 private final TernaryBoolean incrementalRestoreAsyncCompactAfterRescale;
 
+/**
+ * Whether to leverage deleteFilesInRange API to clean up useless rocksdb 
files during
+ * rescaling.
+ */
+private final TernaryBoolean rescalingUseDeleteFilesInRange;
+
 /** Factory for Write Buffer Manager and Block Cache. */
 private RocksDBMemoryFactory rocksDBMemoryFactory;
 // 
@@ -218,6 +225,7 @@ public class EmbeddedRocksDBStateBackend extends 
AbstractManagedMemoryStateBacke
 this.priorityQueueConfig = new RocksDBPriorityQueueConfig();
 this.useIngestDbRestoreMode = TernaryBoolean.UNDEFINED;
 this.incrementalRestoreAsyncCompactAfterRescale = 
TernaryBoolean.UNDEFINED;
+this.rescalingUseDeleteFilesInRange = TernaryBoolean.UNDEFINED;
 }
 
 /**
@@ -323,6 +331,12 @@ public class EmbeddedRocksDBStateBackend extends 
AbstractManagedMemoryStateBacke
 ? 
TernaryBoolean.fromBoxedBoolean(config.get(USE_INGEST_DB_RESTORE_MODE))
 : 
TernaryBoolean.fromBoolean(original.getUseIngestDbRestoreMode());
 
+rescalingUseDeleteFilesInRange =
+original.rescalingUseDeleteFilesInRange == 
TernaryBoolean.UNDEFINED
+? TernaryBoolean.fromBoxedBoolean(
+
config.get

(flink) branch master updated (db3e21f84c2 -> b048a9cd37d)

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

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


from db3e21f84c2 [hotfix][runtime] Refactors suspend and cancel logic
 new 534ea31e7ef [FLINK-34050][state] Clean up useless files in deleted 
ranges during restore for RocksdbStateBackend
 new b048a9cd37d [FLINK-34050][state] Introduce parameter protection for 
DeleteFilesInRange

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:
 .../rocksdb_configurable_configuration.html|  6 +++
 .../state/EmbeddedRocksDBStateBackend.java | 22 -
 .../state/RocksDBConfigurableOptions.java  |  7 +++
 .../state/RocksDBIncrementalCheckpointUtils.java   | 55 +++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 13 -
 .../RocksDBIncrementalRestoreOperation.java|  9 +++-
 .../RocksDBIncrementalCheckpointUtilsTest.java |  5 +-
 .../streaming/state/RocksDBRecoveryTest.java   |  1 +
 8 files changed, 96 insertions(+), 22 deletions(-)



(flink) branch master updated: [hotfix] Remove redundant call to remove().

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

srichter 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 049de6628de [hotfix] Remove redundant call to remove().
049de6628de is described below

commit 049de6628dee1222b4f40eeb3bb5884e1ed9649a
Author: Stefan Richter 
AuthorDate: Tue Feb 20 12:01:03 2024 +0100

[hotfix] Remove redundant call to remove().
---
 .../streaming/state/restore/RocksDBIncrementalRestoreOperation.java  | 5 +
 1 file changed, 1 insertion(+), 4 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
index c96d5d64224..afd19c75c7c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
@@ -600,7 +600,7 @@ public class RocksDBIncrementalRestoreOperation 
implements RocksDBRestoreOper
 keyGroupRange.prettyPrintInterval(),
 operatorIdentifier);
 
-// Choose the best state handle for the initial DB
+// Choose the best state handle for the initial DB and remove it from 
the list
 final IncrementalLocalKeyedStateHandle selectedInitialHandle =
 localKeyedStateHandles.remove(
 
RocksDBIncrementalCheckpointUtils.findTheBestStateHandleForInitial(
@@ -608,9 +608,6 @@ public class RocksDBIncrementalRestoreOperation 
implements RocksDBRestoreOper
 
 Preconditions.checkNotNull(selectedInitialHandle);
 
-// Remove the selected handle from the list so that we don't restore 
it twice.
-localKeyedStateHandles.remove(selectedInitialHandle);
-
 // Init the base DB instance with the initial state
 initBaseDBFromSingleStateHandle(selectedInitialHandle);
 



(flink) 04/05: [FLINK-34199] Add tracing for durations of rescaling/restoring (from local and downloaded remote state).

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

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

commit 16eae412a1219fec1aaf43381fb03709e52407bd
Author: Stefan Richter 
AuthorDate: Mon Jan 22 15:46:32 2024 +0100

[FLINK-34199] Add tracing for durations of rescaling/restoring (from local 
and downloaded remote state).
---
 .../java/org/apache/flink/util/CollectionUtil.java |  42 +++
 .../org/apache/flink/util/CollectionUtilTest.java  |  47 +++
 .../apache/flink/runtime/metrics/MetricNames.java  |   3 +
 .../runtime/state/AbstractChannelStateHandle.java  |   2 +
 .../flink/runtime/state/DirectoryStateHandle.java  |   8 +-
 .../runtime/state/IncrementalKeyedStateHandle.java |  11 ++
 .../apache/flink/runtime/state/KeyGroupRange.java  |   4 +
 .../flink/runtime/state/StateBackendTestBase.java  |  27 +++-
 ...logDelegateEmbeddedRocksDBStateBackendTest.java |   5 +
 .../state/RocksDBKeyedStateBackendBuilder.java |   6 +-
 .../streaming/state/RocksDBStateDownloader.java|  14 +--
 .../RocksDBIncrementalRestoreOperation.java| 137 ++---
 .../state/EmbeddedRocksDBStateBackendTest.java |  40 +-
 .../state/RocksDBStateDownloaderTest.java  |  17 +--
 14 files changed, 280 insertions(+), 83 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java 
b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
index bf1714cb5b8..38adccc2cd9 100644
--- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
@@ -56,10 +56,21 @@ public final class CollectionUtil {
 throw new AssertionError();
 }
 
+/** Returns true if the given collection is null or empty. */
 public static boolean isNullOrEmpty(Collection collection) {
 return collection == null || collection.isEmpty();
 }
 
+/** Returns true if the given collection is empty or contains only null 
elements. */
+public static boolean isEmptyOrAllElementsNull(Collection collection) {
+for (Object o : collection) {
+if (o != null) {
+return false;
+}
+}
+return true;
+}
+
 public static boolean isNullOrEmpty(Map map) {
 return map == null || map.isEmpty();
 }
@@ -214,4 +225,35 @@ public final class CollectionUtil {
 ? (int) Math.ceil(expectedSize / loadFactor)
 : Integer.MAX_VALUE;
 }
+
+/**
+ * Casts the given collection to a subtype. This is an unchecked cast that 
can lead to runtime
+ * exceptions.
+ *
+ * @param collection the collection to cast.
+ * @return the collection unchecked-cast to a subtype.
+ * @param  the subtype to cast to.
+ */
+public static  Collection subTypeCast(Collection 
collection) {
+@SuppressWarnings("unchecked")
+Collection result = (Collection) collection;
+return result;
+}
+
+/**
+ * Casts the given collection to a subtype. This is a checked cast.
+ *
+ * @param collection the collection to cast.
+ * @param subTypeClass the class of the subtype to cast to.
+ * @return the collection checked and cast to a subtype.
+ * @param  the subtype to cast to.
+ */
+public static  Collection checkedSubTypeCast(
+Collection collection, Class subTypeClass) {
+for (Object o : collection) {
+// probe each object, will throw ClassCastException on mismatch.
+subTypeClass.cast(o);
+}
+return subTypeCast(collection);
+}
 }
diff --git 
a/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java 
b/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java
index fb2abfec020..50493316e51 100644
--- a/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java
@@ -22,12 +22,16 @@ import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 
 import static 
org.apache.flink.util.CollectionUtil.HASH_MAP_DEFAULT_LOAD_FACTOR;
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.fail;
 
 /** Tests for java collection utilities. */
 @ExtendWith(TestLoggerExtension.class)
@@ -107,4 +111,47 @@ public class CollectionUtilTest {
 } catch (IllegalArgumentException expected) {
 }
 }
+
+@Test
+public void testIsEmptyOrAllElementsNull() {
+
Assertions.assertTrue(CollectionUtil.isEmptyOrAllElementsNull(Collections.emptyList()));
+Assertions.

(flink) 01/05: [FLINK-31238] Use IngestDB to speed up Rocksdb rescaling recovery (part 1)

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

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

commit 4ddedeb0357eb89bef445ccb2d084c940f2c1c52
Author: 马越 
AuthorDate: Tue Aug 8 14:56:17 2023 +0200

[FLINK-31238] Use IngestDB to speed up Rocksdb rescaling recovery (part 1)
---
 .../flink-statebackend-rocksdb/pom.xml |  23 +++--
 .../state/EmbeddedRocksDBStateBackend.java |  20 +++-
 .../state/RocksDBConfigurableOptions.java  |   7 ++
 .../state/RocksDBIncrementalCheckpointUtils.java   |  56 +++
 .../state/RocksDBKeyedStateBackendBuilder.java |  10 +-
 .../streaming/state/RocksDBOperationUtils.java |  41 +++-
 .../streaming/state/restore/RocksDBHandle.java |  31 ++
 .../RocksDBIncrementalRestoreOperation.java| 107 -
 .../RocksIncrementalCheckpointRescalingTest.java   |  19 +++-
 9 files changed, 298 insertions(+), 16 deletions(-)

diff --git a/flink-state-backends/flink-statebackend-rocksdb/pom.xml 
b/flink-state-backends/flink-statebackend-rocksdb/pom.xml
index 98aa501b1aa..592d29df673 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/pom.xml
+++ b/flink-state-backends/flink-statebackend-rocksdb/pom.xml
@@ -19,8 +19,8 @@ under the License.
 
 -->
 http://maven.apache.org/POM/4.0.0;
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/maven-v4_0_0.xsd;>
+xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
+xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/maven-v4_0_0.xsd;>
 
4.0.0
 
@@ -61,9 +61,9 @@ under the License.

 

-   com.ververica
+   io.github.fredia
frocksdbjni
-   6.20.3-ververica-2.0
+   8.6.7-ververica-test-1.0

 

@@ -102,11 +102,18 @@ under the License.



-   
**/org/apache/flink/contrib/streaming/state/RocksDBTestUtils*
-   
**/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest*
+   
+   
**/org/apache/flink/contrib/streaming/state/RocksDBTestUtils*
+   
+   
+   
**/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackendTest*
+   

-   
**/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest*
-   
**/org/apache/flink/contrib/streaming/state/benchmark/*
+   
+   
**/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest*
+   
+   
**/org/apache/flink/contrib/streaming/state/benchmark/*
+   

META-INF/LICENSE

META-INF/NOTICE

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
index 6d15c53c1a2..5fa5c68acf1 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
@@ -71,6 +71,7 @@ import java.util.function.Supplier;
 
 import static org.apache.flink.configuration.description.TextElement.text;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.RESTORE_OVERLAP_FRACT

(flink) 02/05: [FLINK-31238] Use IngestDB to speed up Rocksdb rescaling recovery (part 2)

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

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

commit 5495a96a38663dc04753cb6db0c100aa2bc7d297
Author: Stefan Richter 
AuthorDate: Fri Jan 5 15:32:32 2024 +0100

[FLINK-31238] Use IngestDB to speed up Rocksdb rescaling recovery (part 2)
---
 .../state/AbstractKeyedStateBackendBuilder.java|   2 +-
 .../state/EmbeddedRocksDBStateBackend.java |  43 +-
 .../state/RocksDBConfigurableOptions.java  |  16 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   | 307 ++-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  13 +-
 .../state/RocksDBKeyedStateBackendBuilder.java |  20 +-
 .../streaming/state/RocksDBOperationUtils.java |  62 +-
 .../streaming/state/RocksDBStateDownloader.java|   4 +-
 .../state/restore/RocksDBFullRestoreOperation.java |   1 +
 .../streaming/state/restore/RocksDBHandle.java |  24 +-
 .../RocksDBHeapTimersFullRestoreOperation.java |   1 +
 .../RocksDBIncrementalRestoreOperation.java| 878 ++---
 .../state/restore/RocksDBNoneRestoreOperation.java |   1 +
 .../state/restore/RocksDBRestoreResult.java|  14 +-
 .../state/EmbeddedRocksDBStateBackendTest.java |  49 +-
 .../streaming/state/RocksDBRecoveryTest.java   | 379 +
 .../state/RocksDBStateBackendConfigTest.java   |  43 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  35 +-
 .../RocksIncrementalCheckpointRescalingTest.java   |   3 +-
 .../test/checkpointing/AutoRescalingITCase.java|  14 +-
 20 files changed, 1482 insertions(+), 427 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackendBuilder.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackendBuilder.java
index 00da919b381..c8fe9ef4652 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackendBuilder.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackendBuilder.java
@@ -34,7 +34,7 @@ import java.util.Collection;
 
 /** An abstract base implementation of the {@link StateBackendBuilder} 
interface. */
 public abstract class AbstractKeyedStateBackendBuilder
-implements StateBackendBuilder {
+implements StateBackendBuilder, 
BackendBuildingException> {
 protected final Logger logger = LoggerFactory.getLogger(getClass());
 
 protected final TaskKvStateRegistry kvStateRegistry;
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
index 5fa5c68acf1..0c5e748772e 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/EmbeddedRocksDBStateBackend.java
@@ -70,6 +70,7 @@ import java.util.UUID;
 import java.util.function.Supplier;
 
 import static org.apache.flink.configuration.description.TextElement.text;
+import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.INCREMENTAL_RESTORE_ASYNC_COMPACT_AFTER_RESCALE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.RESTORE_OVERLAP_FRACTION_THRESHOLD;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.USE_INGEST_DB_RESTORE_MODE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.WRITE_BATCH_SIZE;
@@ -108,8 +109,6 @@ public class EmbeddedRocksDBStateBackend extends 
AbstractManagedMemoryStateBacke
 
 private static final double UNDEFINED_OVERLAP_FRACTION_THRESHOLD = -1;
 
-private static final boolean UNDEFINED_USE_INGEST_DB_RESTORE_MODE = false;
-
 // 
 
 // -- configuration values, set in the application / configuration
@@ -171,9 +170,19 @@ public class EmbeddedRocksDBStateBackend extends 
AbstractManagedMemoryStateBacke
  * The threshold of the overlap fraction between the handle's key-group 
range and target
  * key-group range.
  */
-private double overlapFractionThreshold;
+private final double overlapFractionThreshold;
+
+/**
+ * Whether we use the optimized Ingest/Clip DB method for rescaling 
RocksDB incremental
+ * checkpoints.
+ */
+private final TernaryBoolean useIngestDbRestoreMode;
 
-private boolean useIngestDbRestoreMode;
+/**
+ * Whether we trigger an async compaction after restores for which we 
detect state in the
+ * database (including tombstones) that exceed the proclaimed key-groups 
ra

(flink) 03/05: [FLINK-31238] Deactivate parts of the code until new FRocksDB release is available.

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

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

commit 268a30802edf0c7d1be61e29d47ba5343dfa8a49
Author: Stefan Richter 
AuthorDate: Mon Feb 12 13:35:40 2024 +0100

[FLINK-31238] Deactivate parts of the code until new FRocksDB release is 
available.

Then this commit should be reverted.
---
 .../flink-statebackend-rocksdb/pom.xml |  4 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   | 15 ++--
 .../streaming/state/RocksDBOperationUtils.java | 43 ++
 .../streaming/state/restore/RocksDBHandle.java |  4 +-
 .../RocksDBIncrementalRestoreOperation.java| 16 +---
 5 files changed, 22 insertions(+), 60 deletions(-)

diff --git a/flink-state-backends/flink-statebackend-rocksdb/pom.xml 
b/flink-state-backends/flink-statebackend-rocksdb/pom.xml
index 592d29df673..1a39fc84b7f 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/pom.xml
+++ b/flink-state-backends/flink-statebackend-rocksdb/pom.xml
@@ -61,9 +61,9 @@ under the License.

 

-   io.github.fredia
+   com.ververica
frocksdbjni
-   8.6.7-ververica-test-1.0
+   6.20.3-ververica-2.0

 

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
index 2e7342d4e0f..99b97ef5164 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
@@ -20,17 +20,11 @@ package org.apache.flink.contrib.streaming.state;
 import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
-import org.apache.flink.runtime.state.RegisteredStateMetaInfoBase;
-import org.apache.flink.util.IOUtils;
-import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.function.RunnableWithException;
 
 import 
org.apache.flink.shaded.guava31.com.google.common.primitives.UnsignedBytes;
 
-import org.rocksdb.Checkpoint;
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.CompactRangeOptions;
-import org.rocksdb.ExportImportFilesMetaData;
 import org.rocksdb.LiveFileMetaData;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
@@ -39,16 +33,11 @@ import javax.annotation.Nonnegative;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
-import java.io.File;
-import java.nio.file.Path;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.Optional;
-import java.util.UUID;
 
 /** Utils for RocksDB Incremental Checkpoint. */
 public class RocksDBIncrementalCheckpointUtils {
@@ -218,6 +207,7 @@ public class RocksDBIncrementalCheckpointUtils {
 
 return Optional.of(
 () -> {
+/*
 try (CompactRangeOptions compactionOptions =
 new CompactRangeOptions()
 .setExclusiveManualCompaction(true)
@@ -251,6 +241,7 @@ public class RocksDBIncrementalCheckpointUtils {
 }
 }
 }
+ */
 });
 }
 
@@ -321,6 +312,7 @@ public class RocksDBIncrementalCheckpointUtils {
  * @param resultOutput output parameter for the metadata of the export.
  * @throws RocksDBException on problems inside RocksDB.
  */
+/*
 public static void exportColumnFamilies(
 RocksDB db,
 List columnFamilyHandles,
@@ -357,6 +349,7 @@ public class RocksDBIncrementalCheckpointUtils {
 }
 }
 }
+*/
 
 /** check whether the bytes is before prefixBytes in the character order. 
*/
 public static boolean beforeThePrefixBytes(@Nonnull byte[] bytes, @Nonnull 
byte[] prefixBytes) {
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOperationUtils.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOperationUtils.java
index 1f7bcf5ff1a..c3a9549ccd8 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/st

(flink) 05/05: [FLINK-34199] Add documentation.

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

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

commit 6bf97674b660dbe1cd96b5b0273c3954f2c16fba
Author: Stefan Richter 
AuthorDate: Mon Jan 22 15:50:09 2024 +0100

[FLINK-34199] Add documentation.
---
 docs/content.zh/docs/ops/traces.md | 14 +++---
 docs/content/docs/ops/traces.md| 14 +++---
 .../generated/rocksdb_configurable_configuration.html  | 12 
 3 files changed, 34 insertions(+), 6 deletions(-)

diff --git a/docs/content.zh/docs/ops/traces.md 
b/docs/content.zh/docs/ops/traces.md
index d5025b44d02..8157d1ecddd 100644
--- a/docs/content.zh/docs/ops/traces.md
+++ b/docs/content.zh/docs/ops/traces.md
@@ -97,7 +97,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   
   
 
-  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
+  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
   Checkpoint
   startTs
   Timestamp when the checkpoint has started.
@@ -123,7 +123,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   What was the state of this checkpoint: FAILED or COMPLETED.
 
 
-  JobInitialization
+  JobInitialization
   startTs
   Timestamp when the job initialization has started.
 
@@ -157,7 +157,11 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
 
 
   (Max/Sum)DownloadStateDurationMs(optional - currently only 
supported by RocksDB Incremental)
-  The aggregated (max and sum) across all subtasks duration of 
downloading state files from the DFS.
+  The aggregated (max and sum) duration across all subtasks of 
downloading state files from the DFS.
+
+
+  (Max/Sum)RestoreStateDurationMs(optional - currently only 
supported by RocksDB Incremental)
+  The aggregated (max and sum) duration across all subtasks of 
restoring the state backend from fully localized state, i.e. after all remote 
state was downloaded.
 
 
   (Max/Sum)RestoredStateSizeBytes.[location]
@@ -167,6 +171,10 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
 REMOTE,
 UNKNOWN.
 
+
+  (Max/Sum)RestoreAsyncCompactionDurationMs(optional - 
currently only supported by RocksDB Incremental)
+  The aggregated (max and sum) duration across all subtasks for async 
compaction after incremental restore.
+
   
 
 
diff --git a/docs/content/docs/ops/traces.md b/docs/content/docs/ops/traces.md
index d5025b44d02..8157d1ecddd 100644
--- a/docs/content/docs/ops/traces.md
+++ b/docs/content/docs/ops/traces.md
@@ -97,7 +97,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   
   
 
-  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
+  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
   Checkpoint
   startTs
   Timestamp when the checkpoint has started.
@@ -123,7 +123,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   What was the state of this checkpoint: FAILED or COMPLETED.
 
 
-  JobInitialization
+  JobInitialization
   startTs
   Timestamp when the job initialization has started.
 
@@ -157,7 +157,11 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
 
 
   (Max/Sum)DownloadStateDurationMs(optional - currently only 
supported by RocksDB Incremental)
-  The aggregated (max and sum) across all subtasks duration of 
downloading state files from the DFS.
+  The aggregated (max and sum) duration across all subtasks of 
downloading state files from the DFS.
+
+
+  (Max/Sum)RestoreStateDurationMs(optional - currently only 
supported by RocksDB Incremental)
+  The aggregated (max and sum) duration across all subtasks of 
restoring the state backend from fully localized state, i.e. after all remote 
state was downloaded.
 
 
   (Max/Sum)RestoredStateSizeBytes.[location]
@@ -167,6 +171,10 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
 REMOTE,
 UNKNOWN.
 
+
+  (Max/Sum)RestoreAsyncCompactionDurationMs(optional - 
currently only supported by RocksDB Incremental)
+  The aggregated (max and sum) duration across all subtasks for async 
compaction after incremental restore.
+
   
 
 
diff --git 
a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html 
b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
index 8810b9086d1..4ec59a27110 100644
--- a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
+++ b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
@@ -74,6 +74,12

(flink) branch master updated (5844092408d -> 6bf97674b66)

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

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


from 5844092408d [FLINK-34000] Remove IncrementalGroupAgg Json Plan & IT 
tests
 new 4ddedeb0357 [FLINK-31238] Use IngestDB to speed up Rocksdb rescaling 
recovery (part 1)
 new 5495a96a386 [FLINK-31238] Use IngestDB to speed up Rocksdb rescaling 
recovery (part 2)
 new 268a30802ed [FLINK-31238] Deactivate parts of the code until new 
FRocksDB release is available.
 new 16eae412a12 [FLINK-34199] Add tracing for durations of 
rescaling/restoring (from local and downloaded remote state).
 new 6bf97674b66 [FLINK-34199] Add documentation.

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


Summary of changes:
 docs/content.zh/docs/ops/traces.md |  14 +-
 docs/content/docs/ops/traces.md|  14 +-
 .../rocksdb_configurable_configuration.html|  12 +
 .../java/org/apache/flink/util/CollectionUtil.java |  42 +
 .../org/apache/flink/util/CollectionUtilTest.java  |  47 ++
 .../apache/flink/runtime/metrics/MetricNames.java  |   3 +
 .../runtime/state/AbstractChannelStateHandle.java  |   2 +
 .../state/AbstractKeyedStateBackendBuilder.java|   2 +-
 .../flink/runtime/state/DirectoryStateHandle.java  |   8 +-
 .../runtime/state/IncrementalKeyedStateHandle.java |  11 +
 .../apache/flink/runtime/state/KeyGroupRange.java  |   4 +
 .../flink/runtime/state/StateBackendTestBase.java  |  27 +-
 ...logDelegateEmbeddedRocksDBStateBackendTest.java |   5 +
 .../flink-statebackend-rocksdb/pom.xml |  19 +-
 .../state/EmbeddedRocksDBStateBackend.java |  43 +-
 .../state/RocksDBConfigurableOptions.java  |  19 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   | 296 ++-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  13 +-
 .../state/RocksDBKeyedStateBackendBuilder.java |  34 +-
 .../streaming/state/RocksDBOperationUtils.java |  34 +-
 .../streaming/state/RocksDBStateDownloader.java|  18 +-
 .../state/restore/RocksDBFullRestoreOperation.java |   1 +
 .../streaming/state/restore/RocksDBHandle.java |  29 +
 .../RocksDBHeapTimersFullRestoreOperation.java |   1 +
 .../RocksDBIncrementalRestoreOperation.java| 864 -
 .../state/restore/RocksDBNoneRestoreOperation.java |   1 +
 .../state/restore/RocksDBRestoreResult.java|  14 +-
 .../state/EmbeddedRocksDBStateBackendTest.java |  87 ++-
 .../streaming/state/RocksDBRecoveryTest.java   | 379 +
 .../state/RocksDBStateBackendConfigTest.java   |  43 +
 .../state/RocksDBStateDownloaderTest.java  |  17 +-
 .../contrib/streaming/state/RocksDBTestUtils.java  |  35 +-
 .../RocksIncrementalCheckpointRescalingTest.java   |  22 +-
 .../test/checkpointing/AutoRescalingITCase.java|  14 +-
 34 files changed, 1849 insertions(+), 325 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBRecoveryTest.java



(flink) branch master updated (201571b486f -> bdf31327775)

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

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


from 201571b486f [FLINK-27992] Set ALWAYS chaining strategy for CepOperator 
in ExecMatch
 new f1e355437eb [FLINK-34134] Minor cleanup of unused parameters in 
RocksDB backend.
 new 22241080a80 [FLINK-34134] Introduce size/location statistics reporting 
to StateObject interface.
 new 210ab633350 [FLINK-34134] Add methods to determine local directory 
sizes to FileUtils.
 new 8e0e2ca97c9 [FLINK-34134] Implement handle-specific size/location 
reporting methods.
 new 9dc0f6f658b [FLINK-34134] Collect and report size/location statistics 
during task restore.
 new bdf31327775 [FLINK-34134] Add documentation.

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:
 docs/content.zh/docs/ops/traces.md | 12 +++-
 docs/content/docs/ops/traces.md| 12 +++-
 .../main/java/org/apache/flink/util/FileUtils.java | 24 +++
 .../java/org/apache/flink/util/FileUtilsTest.java  | 13 
 .../KubernetesStateHandleStore.java|  5 ++
 .../flink/runtime/checkpoint/OperatorState.java| 18 ++---
 .../runtime/checkpoint/OperatorSubtaskState.java   | 60 
 .../runtime/checkpoint/StateObjectCollection.java  | 35 -
 .../apache/flink/runtime/checkpoint/TaskState.java | 18 ++---
 .../runtime/checkpoint/TaskStateSnapshot.java  | 17 +++--
 .../apache/flink/runtime/metrics/MetricNames.java  |  1 +
 .../runtime/state/AbstractChannelStateHandle.java  | 19 +++--
 .../flink/runtime/state/ChainedStateHandle.java| 23 +++---
 .../flink/runtime/state/DirectoryStateHandle.java  | 25 +--
 .../state/IncrementalLocalKeyedStateHandle.java|  6 ++
 .../state/IncrementalRemoteKeyedStateHandle.java   | 25 ---
 .../flink/runtime/state/KeyGroupsStateHandle.java  | 11 +++
 .../runtime/state/OperatorStreamStateHandle.java   |  5 ++
 .../state/RetrievableStreamStateHandle.java|  5 ++
 .../flink/runtime/state/SnapshotDirectory.java |  4 +-
 .../apache/flink/runtime/state/StateObject.java| 57 +++
 .../changelog/ChangelogStateBackendHandle.java |  5 ++
 .../inmemory/InMemoryChangelogStateHandle.java |  5 ++
 .../runtime/state/filesystem/FileStateHandle.java  | 14 
 .../state/memory/ByteStreamStateHandle.java|  5 ++
 .../IncrementalLocalKeyedStateHandleTest.java  | 84 ++
 .../IncrementalRemoteKeyedStateHandleTest.java | 19 +
 .../runtime/state/KeyGroupsStateHandleTest.java| 21 ++
 .../state/filesystem/FileStateHandleTest.java  | 37 ++
 .../state/memory/ByteStreamStateHandleTest.java| 19 +
 .../state/RocksDBKeyedStateBackendBuilder.java |  5 --
 .../streaming/state/StateHandleDownloadSpec.java   |  2 +-
 .../snapshot/RocksIncrementalSnapshotStrategy.java |  9 ++-
 .../RocksIncrementalSnapshotStrategyTest.java  | 11 ++-
 .../api/operators/BackendRestorerProcedure.java|  9 ++-
 .../operators/StreamTaskStateInitializerImpl.java  | 62 +---
 .../operators/BackendRestorerProcedureTest.java| 11 ++-
 .../StreamTaskStateInitializerImplTest.java| 63 ++--
 38 files changed, 627 insertions(+), 149 deletions(-)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandleTest.java



(flink) 05/06: [FLINK-34134] Collect and report size/location statistics during task restore.

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

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

commit 9dc0f6f658be4e8d1845857bb16ac5155ccb0873
Author: Stefan Richter 
AuthorDate: Tue Jan 16 17:22:02 2024 +0100

[FLINK-34134] Collect and report size/location statistics during task 
restore.
---
 .../apache/flink/runtime/metrics/MetricNames.java  |  1 +
 .../api/operators/BackendRestorerProcedure.java|  9 +++-
 .../operators/StreamTaskStateInitializerImpl.java  | 62 +
 .../operators/BackendRestorerProcedureTest.java| 11 ++--
 .../StreamTaskStateInitializerImplTest.java| 63 +++---
 5 files changed, 125 insertions(+), 21 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java
index b0e5be64592..4aa375020c9 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java
@@ -69,6 +69,7 @@ public class MetricNames {
 public static final String INITIALIZE_STATE_DURATION = 
"InitializeStateDurationMs";
 public static final String GATE_RESTORE_DURATION = "GateRestoreDurationMs";
 public static final String DOWNLOAD_STATE_DURATION = 
"DownloadStateDurationMs";
+public static final String RESTORED_STATE_SIZE = "RestoredStateSizeBytes";
 
 public static final String START_WORKER_FAILURE_RATE = "startWorkFailure" 
+ SUFFIX_RATE;
 
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java
index 5a0cea94927..83f6cdcc41f 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java
@@ -91,7 +91,9 @@ public class BackendRestorerProcedure> 
restoreOptions)
+public T createAndRestore(
+@Nonnull List> restoreOptions,
+@Nonnull StateObject.StateObjectSizeStatsCollector stats)
 throws Exception {
 
 if (restoreOptions.isEmpty()) {
@@ -132,7 +134,10 @@ public class BackendRestorerProcedure handle.collectSizeStats(stats));
+return successfullyRestored;
 } catch (Exception ex) {
 
 collectedException = ExceptionUtils.firstOrSuppressed(ex, 
collectedException);
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java
index 737a75f0627..9cd11a09a81 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java
@@ -29,6 +29,7 @@ import 
org.apache.flink.runtime.checkpoint.StateObjectCollection;
 import org.apache.flink.runtime.checkpoint.SubTaskInitializationMetricsBuilder;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricNames;
 import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
 import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
@@ -42,6 +43,7 @@ import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.StatePartitionStreamProvider;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.TaskStateManager;
@@ -65,7 +67,9 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
+import java.util.Objects;
 import java.util.OptionalLong;
+import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
 import static 
org.apache.flink.runtime.state.StateBackendLoader.loadStateBackendFromKeyedStateHandles;
@@ -166,6 +170,9 @@ public class StreamTaskStateInitializerImpl implements 
StreamTaskStateInitialize
 CloseableIterable rawOperatorStateInputs 
= null;
 InternalTimeServiceManager timeServiceManager;
 
+final StateObject.StateObjectSizeStatsCollector statsCollector =
+

(flink) 01/06: [FLINK-34134] Minor cleanup of unused parameters in RocksDB backend.

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

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

commit f1e355437eb8ae5a0cdad0caccac26faa5c022c6
Author: Stefan Richter 
AuthorDate: Mon Jan 15 13:22:41 2024 +0100

[FLINK-34134] Minor cleanup of unused parameters in RocksDB backend.
---
 .../streaming/state/RocksDBKeyedStateBackendBuilder.java  |  5 -
 .../state/snapshot/RocksIncrementalSnapshotStrategy.java  |  9 ++---
 .../state/snapshot/RocksIncrementalSnapshotStrategyTest.java  | 11 +--
 3 files changed, 11 insertions(+), 14 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
index f33ce8f0fba..3f67a8b5f85 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
@@ -358,10 +358,8 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
 // init snapshot strategy after db is assured to be initialized
 checkpointStrategy =
 initializeSavepointAndCheckpointStrategies(
-cancelStreamRegistryForBackend,
 rocksDBResourceGuard,
 kvStateInformation,
-registeredPQStates,
 keyGroupPrefixBytes,
 db,
 backendUID,
@@ -523,10 +521,8 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
 }
 
 private RocksDBSnapshotStrategyBase 
initializeSavepointAndCheckpointStrategies(
-CloseableRegistry cancelStreamRegistry,
 ResourceGuard rocksDBResourceGuard,
 LinkedHashMap 
kvStateInformation,
-LinkedHashMap> 
registeredPQStates,
 int keyGroupPrefixBytes,
 RocksDB db,
 UUID backendUID,
@@ -547,7 +543,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
 keyGroupRange,
 keyGroupPrefixBytes,
 localRecoveryConfig,
-cancelStreamRegistry,
 instanceBasePath,
 backendUID,
 materializedSstFiles,
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
index 9eb66a5ee8c..436a0f2ec1c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
@@ -97,7 +97,6 @@ public class RocksIncrementalSnapshotStrategy
 @Nonnull KeyGroupRange keyGroupRange,
 @Nonnegative int keyGroupPrefixBytes,
 @Nonnull LocalRecoveryConfig localRecoveryConfig,
-@Nonnull CloseableRegistry cancelStreamRegistry,
 @Nonnull File instanceBasePath,
 @Nonnull UUID backendUID,
 @Nonnull SortedMap> 
uploadedStateHandles,
@@ -358,7 +357,9 @@ public class RocksIncrementalSnapshotStrategy
 snapshotCloseableRegistry,
 tmpResourcesRegistry);
 uploadedSize +=
-sstFilesUploadResult.stream().mapToLong(e -> 
e.getStateSize()).sum();
+sstFilesUploadResult.stream()
+.mapToLong(HandleAndLocalPath::getStateSize)
+.sum();
 sstFiles.addAll(sstFilesUploadResult);
 
 List miscFilesUploadResult =
@@ -369,7 +370,9 @@ public class RocksIncrementalSnapshotStrategy
 snapshotCloseableRegistry,
 tmpResourcesRegistry);
 uploadedSize +=
-miscFilesUploadResult.stream().mapToLong(e -> 
e.getStateSize()).sum();
+miscFilesUploadResult.stream()
+.mapToLong(HandleAndLocalPath::getStateSize)
+

(flink) 04/06: [FLINK-34134] Implement handle-specific size/location reporting methods.

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

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

commit 8e0e2ca97c9225d4ce3faa6781a0e3a524645cd4
Author: Stefan Richter 
AuthorDate: Tue Jan 16 16:09:44 2024 +0100

[FLINK-34134] Implement handle-specific size/location reporting methods.
---
 .../KubernetesStateHandleStore.java|  5 ++
 .../flink/runtime/checkpoint/OperatorState.java| 18 ++---
 .../runtime/checkpoint/OperatorSubtaskState.java   | 60 
 .../runtime/checkpoint/StateObjectCollection.java  | 35 -
 .../apache/flink/runtime/checkpoint/TaskState.java | 18 ++---
 .../runtime/checkpoint/TaskStateSnapshot.java  | 17 +++--
 .../runtime/state/AbstractChannelStateHandle.java  | 19 +++--
 .../flink/runtime/state/ChainedStateHandle.java| 23 +++---
 .../flink/runtime/state/DirectoryStateHandle.java  | 25 +--
 .../state/IncrementalLocalKeyedStateHandle.java|  6 ++
 .../state/IncrementalRemoteKeyedStateHandle.java   | 25 ---
 .../flink/runtime/state/KeyGroupsStateHandle.java  | 11 +++
 .../runtime/state/OperatorStreamStateHandle.java   |  5 ++
 .../state/RetrievableStreamStateHandle.java|  5 ++
 .../flink/runtime/state/SnapshotDirectory.java |  4 +-
 .../changelog/ChangelogStateBackendHandle.java |  5 ++
 .../inmemory/InMemoryChangelogStateHandle.java |  5 ++
 .../runtime/state/filesystem/FileStateHandle.java  | 14 
 .../state/memory/ByteStreamStateHandle.java|  5 ++
 .../IncrementalLocalKeyedStateHandleTest.java  | 84 ++
 .../IncrementalRemoteKeyedStateHandleTest.java | 19 +
 .../runtime/state/KeyGroupsStateHandleTest.java| 21 ++
 .../state/filesystem/FileStateHandleTest.java  | 37 ++
 .../state/memory/ByteStreamStateHandleTest.java| 19 +
 .../streaming/state/StateHandleDownloadSpec.java   |  2 +-
 25 files changed, 377 insertions(+), 110 deletions(-)

diff --git 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java
 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java
index 2a99cf6e2f1..790a0441dbf 100644
--- 
a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java
+++ 
b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java
@@ -137,6 +137,11 @@ public class KubernetesStateHandleStore
 return inner.getStateSize();
 }
 
+@Override
+public void collectSizeStats(StateObjectSizeStatsCollector collector) {
+inner.collectSizeStats(collector);
+}
+
 RetrievableStateHandle getInner() {
 return inner;
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
index 2b6c4e06197..d377ca5ec7b 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import static org.apache.flink.util.Preconditions.checkState;
 
@@ -204,16 +205,17 @@ public class OperatorState implements 
CompositeStateHandle {
 
 @Override
 public long getStateSize() {
-long result = coordinatorState == null ? 0L : 
coordinatorState.getStateSize();
+return 
streamAllSubHandles().mapToLong(StateObject::getStateSize).sum();
+}
 
-for (int i = 0; i < parallelism; i++) {
-OperatorSubtaskState operatorSubtaskState = 
operatorSubtaskStates.get(i);
-if (operatorSubtaskState != null) {
-result += operatorSubtaskState.getStateSize();
-}
-}
+@Override
+public void collectSizeStats(StateObjectSizeStatsCollector collector) {
+streamAllSubHandles().forEach(handle -> 
handle.collectSizeStats(collector));
+}
 
-return result;
+private Stream streamAllSubHandles() {
+return Stream.concat(Stream.of(coordinatorState), 
operatorSubtaskStates.values().stream())
+.filter(Objects::nonNull);
 }
 
 @Override
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
index 4fede9683e5..a2f30e9f64a 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
@@ -19,6 +19,7 @@
 p

(flink) 02/06: [FLINK-34134] Introduce size/location statistics reporting to StateObject interface.

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

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

commit 22241080a8013296a95c2b18cc17d5148befad4f
Author: Stefan Richter 
AuthorDate: Tue Jan 16 15:22:19 2024 +0100

[FLINK-34134] Introduce size/location statistics reporting to StateObject 
interface.
---
 .../apache/flink/runtime/state/StateObject.java| 57 ++
 1 file changed, 57 insertions(+)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateObject.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateObject.java
index 8eaa1d68921..5aadebc5cd7 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateObject.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateObject.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.state;
 
 import java.io.Serializable;
+import java.util.EnumMap;
 
 /**
  * Base of all handles that represent checkpointed state in some form. The 
object may hold the
@@ -63,4 +64,60 @@ public interface StateObject extends Serializable {
  * @return Size of the state in bytes.
  */
 long getStateSize();
+
+/**
+ * Collects statistics about state size and location from the state object.
+ *
+ * @implNote default implementation reports {@link 
StateObject#getStateSize()} as size and
+ * {@link StateObjectLocation#UNKNOWN} as location.
+ * @param collector the statistics collector.
+ */
+default void collectSizeStats(StateObjectSizeStatsCollector collector) {
+collector.add(StateObjectLocation.UNKNOWN, getStateSize());
+}
+
+/** Enum for state locations. */
+enum StateObjectLocation {
+LOCAL_MEMORY,
+LOCAL_DISK,
+REMOTE,
+UNKNOWN,
+}
+
+/**
+ * Collector for size and location stats from a state object via {@link
+ * StateObject#collectSizeStats(StateObjectSizeStatsCollector)}.
+ */
+final class StateObjectSizeStatsCollector {
+private final EnumMap stats;
+
+private StateObjectSizeStatsCollector() {
+stats = new EnumMap<>(StateObjectLocation.class);
+}
+
+public void add(StateObjectLocation key, long value) {
+stats.compute(
+key,
+(k, v) -> {
+if (v != null) {
+return v + value;
+} else {
+return value;
+}
+});
+}
+
+public EnumMap getStats() {
+return stats;
+}
+
+public static StateObjectSizeStatsCollector create() {
+return new StateObjectSizeStatsCollector();
+}
+
+@Override
+public String toString() {
+return "StateObjectSizeStatsCollector{" + "stats=" + stats + '}';
+}
+}
 }



(flink) 06/06: [FLINK-34134] Add documentation.

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

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

commit bdf313277754ebcab4d6ccb6aba9d49860933322
Author: Stefan Richter 
AuthorDate: Wed Jan 17 14:55:20 2024 +0100

[FLINK-34134] Add documentation.
---
 docs/content.zh/docs/ops/traces.md | 12 ++--
 docs/content/docs/ops/traces.md| 12 ++--
 2 files changed, 20 insertions(+), 4 deletions(-)

diff --git a/docs/content.zh/docs/ops/traces.md 
b/docs/content.zh/docs/ops/traces.md
index 3fffd7ce430..d5025b44d02 100644
--- a/docs/content.zh/docs/ops/traces.md
+++ b/docs/content.zh/docs/ops/traces.md
@@ -97,7 +97,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   
   
 
-  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
+  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
   Checkpoint
   startTs
   Timestamp when the checkpoint has started.
@@ -123,7 +123,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   What was the state of this checkpoint: FAILED or COMPLETED.
 
 
-  JobInitialization
+  JobInitialization
   startTs
   Timestamp when the job initialization has started.
 
@@ -159,6 +159,14 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   (Max/Sum)DownloadStateDurationMs(optional - currently only 
supported by RocksDB Incremental)
   The aggregated (max and sum) across all subtasks duration of 
downloading state files from the DFS.
 
+
+  (Max/Sum)RestoredStateSizeBytes.[location]
+  The aggregated (max and sum) across all subtasks size of restored 
state by location. Possible locations are defined in Enum 
StateObjectSizeStatsCollector as 
+LOCAL_MEMORY,
+LOCAL_DISK,
+REMOTE,
+UNKNOWN.
+
   
 
 
diff --git a/docs/content/docs/ops/traces.md b/docs/content/docs/ops/traces.md
index 3fffd7ce430..d5025b44d02 100644
--- a/docs/content/docs/ops/traces.md
+++ b/docs/content/docs/ops/traces.md
@@ -97,7 +97,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   
   
 
-  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
+  org.apache.flink.runtime.checkpoint.CheckpointStatsTracker
   Checkpoint
   startTs
   Timestamp when the checkpoint has started.
@@ -123,7 +123,7 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   What was the state of this checkpoint: FAILED or COMPLETED.
 
 
-  JobInitialization
+  JobInitialization
   startTs
   Timestamp when the job initialization has started.
 
@@ -159,6 +159,14 @@ Flink reports a single span trace for the whole checkpoint 
and job initializatio
   (Max/Sum)DownloadStateDurationMs(optional - currently only 
supported by RocksDB Incremental)
   The aggregated (max and sum) across all subtasks duration of 
downloading state files from the DFS.
 
+
+  (Max/Sum)RestoredStateSizeBytes.[location]
+  The aggregated (max and sum) across all subtasks size of restored 
state by location. Possible locations are defined in Enum 
StateObjectSizeStatsCollector as 
+LOCAL_MEMORY,
+LOCAL_DISK,
+REMOTE,
+UNKNOWN.
+
   
 
 



(flink) 03/06: [FLINK-34134] Add methods to determine local directory sizes to FileUtils.

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

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

commit 210ab633350c77b026f883f72d3884eda20a03ae
Author: Stefan Richter 
AuthorDate: Tue Jan 16 15:36:09 2024 +0100

[FLINK-34134] Add methods to determine local directory sizes to FileUtils.
---
 .../main/java/org/apache/flink/util/FileUtils.java | 24 ++
 .../java/org/apache/flink/util/FileUtilsTest.java  | 13 
 2 files changed, 37 insertions(+)

diff --git a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java 
b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
index e33e69e9c8d..a75df3d643b 100644
--- a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java
@@ -580,6 +580,30 @@ public final class FileUtils {
 return filterFileVisitor.getFiles();
 }
 
+/**
+ * Computes the sum of sizes of all files in the directory and it's 
subdirectories.
+ *
+ * @param path the root path from which to start the calculation.
+ * @param options visitation options for the directory traversal.
+ * @return sum of sizes of all files in the directory and it's 
subdirectories.
+ * @throws IOException if the size cannot be determined.
+ */
+public static long getDirectoryFilesSize(java.nio.file.Path path, 
FileVisitOption... options)
+throws IOException {
+
+if (path == null) {
+return 0L;
+}
+
+try (Stream pathStream = Files.walk(path, 
options)) {
+return pathStream
+.map(java.nio.file.Path::toFile)
+.filter(File::isFile)
+.mapToLong(File::length)
+.sum();
+}
+}
+
 /**
  * Absolutize the given path if it is relative.
  *
diff --git a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java 
b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
index afd1b70c941..0a9953c060b 100644
--- a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java
@@ -24,6 +24,7 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.testutils.junit.utils.TempDirUtils;
 
+import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -363,6 +364,18 @@ public class FileUtilsTest {
 assertThat(targetPath).isEqualTo(dirInLinked2);
 }
 
+@Test
+void testGetDirectorySize() throws Exception {
+final File parent = TempDirUtils.newFolder(temporaryFolder);
+
+// Empty directory should have size 0
+Assertions.assertEquals(0, 
FileUtils.getDirectoryFilesSize(parent.toPath()));
+
+// Expected size: (20*5^0 + 20*5^1 + 20*5^2 + 20*5^3) * 1 byte = 3120 
bytes
+generateRandomDirs(parent, 20, 5, 3);
+Assertions.assertEquals(3120, 
FileUtils.getDirectoryFilesSize(parent.toPath()));
+}
+
 // 
 //  Utilities
 // 



(flink) 04/05: [FLINK-33341][state] Add support for rescaling from local keyed state to RockDBIncrementalRestoreOperation.

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

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

commit 05cdb299e26db931a328bdc4e536dfb7effdb905
Author: Stefan Richter 
AuthorDate: Wed Oct 25 11:50:55 2023 +0200

[FLINK-33341][state] Add support for rescaling from local keyed state to 
RockDBIncrementalRestoreOperation.
---
 .../state/RocksDBIncrementalCheckpointUtils.java   |   8 +-
 .../streaming/state/StateHandleDownloadSpec.java   |  12 ++
 .../RocksDBIncrementalRestoreOperation.java| 233 ++---
 3 files changed, 129 insertions(+), 124 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
index 54121709876..a835d10c481 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBIncrementalCheckpointUtils.java
@@ -175,14 +175,14 @@ public class RocksDBIncrementalCheckpointUtils {
  * @return The best candidate or null if no candidate was a good fit.
  */
 @Nullable
-public static KeyedStateHandle chooseTheBestStateHandleForInitial(
-@Nonnull Collection restoreStateHandles,
+public static  T 
chooseTheBestStateHandleForInitial(
+@Nonnull Collection restoreStateHandles,
 @Nonnull KeyGroupRange targetKeyGroupRange,
 double overlapFractionThreshold) {
 
-KeyedStateHandle bestStateHandle = null;
+T bestStateHandle = null;
 Score bestScore = Score.MIN;
-for (KeyedStateHandle rawStateHandle : restoreStateHandles) {
+for (T rawStateHandle : restoreStateHandles) {
 Score handleScore =
 stateHandleEvaluator(
 rawStateHandle, targetKeyGroupRange, 
overlapFractionThreshold);
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java
index 93a33fdc6fa..5f37f84921f 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/StateHandleDownloadSpec.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.contrib.streaming.state;
 
+import org.apache.flink.runtime.state.DirectoryStateHandle;
+import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle;
 import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle;
 
 import java.nio.file.Path;
@@ -46,4 +48,14 @@ public class StateHandleDownloadSpec {
 public Path getDownloadDestination() {
 return downloadDestination;
 }
+
+public IncrementalLocalKeyedStateHandle 
createLocalStateHandleForDownloadedState() {
+return new IncrementalLocalKeyedStateHandle(
+stateHandle.getBackendIdentifier(),
+stateHandle.getCheckpointId(),
+new DirectoryStateHandle(downloadDestination),
+stateHandle.getKeyGroupRange(),
+stateHandle.getMetaDataStateHandle(),
+stateHandle.getSharedState());
+}
 }
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
index a62bbb4a70b..11d4756ae13 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBIncrementalRestoreOperation.java
@@ -35,7 +35,6 @@ import 
org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.state.BackendBuildingException;
 import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
-import org.apache.flink.runtime.state.DirectoryStateHandle;
 import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
 import 
org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath;
 import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle

(flink) 03/05: [FLINK-33341][state] Refactoring: consolidate equals/hashCode/toString for incremental state handle classes.

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

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

commit 0be231972d5d687924db22ae81d6a10049f7c43e
Author: Stefan Richter 
AuthorDate: Fri Oct 27 11:01:49 2023 +0200

[FLINK-33341][state] Refactoring: consolidate equals/hashCode/toString for 
incremental state handle classes.
---
 .../state/AbstractIncrementalStateHandle.java  | 36 +++
 .../state/IncrementalLocalKeyedStateHandle.java| 35 +--
 .../state/IncrementalRemoteKeyedStateHandle.java   | 69 ++
 3 files changed, 43 insertions(+), 97 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
index 8c7ea74c33c..85f12329f57 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.state;
 import javax.annotation.Nonnull;
 
 import java.util.List;
+import java.util.Objects;
 import java.util.UUID;
 
 /** Abstract superclass for all {@link IncrementalKeyedStateHandle}. */
@@ -103,4 +104,39 @@ public abstract class AbstractIncrementalStateHandle 
implements IncrementalKeyed
 ? null
 : this;
 }
+
+@Override
+public boolean equals(Object o) {
+if (this == o) {
+return true;
+}
+if (o == null || getClass() != o.getClass()) {
+return false;
+}
+AbstractIncrementalStateHandle that = (AbstractIncrementalStateHandle) 
o;
+return Objects.equals(stateHandleId, that.stateHandleId);
+}
+
+@Override
+public int hashCode() {
+return stateHandleId.hashCode();
+}
+
+@Override
+public String toString() {
+return "AbstractIncrementalStateHandle{"
++ "checkpointId="
++ checkpointId
++ ", backendIdentifier="
++ backendIdentifier
++ ", keyGroupRange="
++ keyGroupRange
++ ", sharedState="
++ sharedState
++ ", metaStateHandle="
++ metaStateHandle
++ ", stateHandleId="
++ stateHandleId
++ '}';
+}
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
index f854c111c6e..ac457f00622 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java
@@ -94,44 +94,13 @@ public class IncrementalLocalKeyedStateHandle extends 
AbstractIncrementalStateHa
 return directoryStateHandle.getStateSize() + 
metaStateHandle.getStateSize();
 }
 
-@Override
-public int hashCode() {
-int result = directoryStateHandle.hashCode();
-result = 31 * result + getKeyGroupRange().hashCode();
-result = 31 * result + getMetaDataStateHandle().hashCode();
-return result;
-}
-
-@Override
-public boolean equals(Object o) {
-if (this == o) {
-return true;
-}
-if (o == null || getClass() != o.getClass()) {
-return false;
-}
-if (!super.equals(o)) {
-return false;
-}
-
-IncrementalLocalKeyedStateHandle that = 
(IncrementalLocalKeyedStateHandle) o;
-
-return getKeyGroupRange().equals(that.keyGroupRange)
-&& 
getMetaDataStateHandle().equals(that.getMetaDataStateHandle());
-}
-
 @Override
 public String toString() {
 return "IncrementalLocalKeyedStateHandle{"
-+ "metaDataState="
-+ metaStateHandle
-+ "} "
-+ "DirectoryKeyedStateHandle{"
 + "directoryStateHandle="
 + directoryStateHandle
-+ ", keyGroupRange="
-+ keyGroupRange
-+ '}';
++ "} "
++ super.toString();
 }
 
 @Override
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRemoteKeyedStateHandle.java
index 86a5b59c168..41b9a0466cc 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalRem

(flink) 02/05: [FLINK-33341][state] Refactoring: introduce common superclass for all IncrementalKeyedStateHandle.

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

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

commit 2d74ee3966525be5ab3770a653c96a7d04d5f05a
Author: Stefan Richter 
AuthorDate: Thu Oct 19 14:31:10 2023 +0200

[FLINK-33341][state] Refactoring: introduce common superclass for all 
IncrementalKeyedStateHandle.
---
 .../metadata/MetadataV2V3SerializerBase.java   |   2 +-
 .../state/AbstractIncrementalStateHandle.java  | 106 ++
 .../runtime/state/DirectoryKeyedStateHandle.java   | 124 -
 .../runtime/state/IncrementalKeyedStateHandle.java |   3 +
 .../state/IncrementalLocalKeyedStateHandle.java| 122 ++--
 .../state/IncrementalRemoteKeyedStateHandle.java   |  71 ++--
 .../changelog/ChangelogStateBackendHandle.java |   2 +-
 .../checkpoint/CheckpointCoordinatorTest.java  |   2 +-
 .../flink/runtime/state/ChangelogTestUtils.java|   2 +-
 .../IncrementalRemoteKeyedStateHandleTest.java |  18 +--
 .../RocksDBIncrementalRestoreOperation.java|   6 +-
 11 files changed, 192 insertions(+), 266 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
index db3b5dd22d0..bd91e8e1fbb 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
@@ -341,7 +341,7 @@ public abstract class MetadataV2V3SerializerBase {
 
dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
 dos.writeLong(incrementalKeyedStateHandle.getCheckpointedSize());
 
-
serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), 
dos);
+
serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaDataStateHandle(),
 dos);
 
 
serializeHandleAndLocalPathList(incrementalKeyedStateHandle.getSharedState(), 
dos);
 
serializeHandleAndLocalPathList(incrementalKeyedStateHandle.getPrivateState(), 
dos);
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
new file mode 100644
index 000..8c7ea74c33c
--- /dev/null
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.state;
+
+import javax.annotation.Nonnull;
+
+import java.util.List;
+import java.util.UUID;
+
+/** Abstract superclass for all {@link IncrementalKeyedStateHandle}. */
+public abstract class AbstractIncrementalStateHandle implements 
IncrementalKeyedStateHandle {
+private static final long serialVersionUID = 1L;
+
+/** The checkpoint Id. */
+protected final long checkpointId;
+
+/**
+ * UUID to identify the backend which created this state handle. This is 
in creating the key for
+ * the {@link SharedStateRegistry}.
+ */
+protected final UUID backendIdentifier;
+
+/** The key-group range covered by this state handle. */
+protected final KeyGroupRange keyGroupRange;
+
+/** Shared state in the incremental checkpoint. */
+protected final List sharedState;
+
+/** Primary meta data state of the incremental checkpoint. */
+protected final StreamStateHandle metaStateHandle;
+
+/** Unique id for this state handle. */
+protected final StateHandleID stateHandleId;
+
+public AbstractIncrementalStateHandle(
+UUID backendIdentifier,
+KeyGroupRange keyGroupRange,
+long checkpointId,
+List sharedState,
+StreamStateHandle metaStateHandle,
+StateHandleID stateHandleId) {
+this.checkpointId = checkpointId;
+this.keyGroupR

(flink) branch master updated (e914eb7fc3f -> a4ad86fb083)

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

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


from e914eb7fc3f [FLINK-33371] Make TestValues sinks return results as Rows
 new 98e4610f09f [FLINK-33246][tests] Add AutoRescalingITCase.
 new 2d74ee39665 [FLINK-33341][state] Refactoring: introduce common 
superclass for all IncrementalKeyedStateHandle.
 new 0be231972d5 [FLINK-33341][state] Refactoring: consolidate 
equals/hashCode/toString for incremental state handle classes.
 new 05cdb299e26 [FLINK-33341][state] Add support for rescaling from local 
keyed state to RockDBIncrementalRestoreOperation.
 new a4ad86fb083 [FLINK-33341][state] Add support for rescaling from local 
keyed state to PrioritizedOperatorSubtaskState.

The 5 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:
 .../PrioritizedOperatorSubtaskState.java   | 132 -
 .../metadata/MetadataV2V3SerializerBase.java   |   2 +-
 .../state/AbstractIncrementalStateHandle.java  | 142 ++
 .../runtime/state/DirectoryKeyedStateHandle.java   | 124 -
 .../runtime/state/IncrementalKeyedStateHandle.java |   3 +
 .../state/IncrementalLocalKeyedStateHandle.java| 101 ++--
 .../state/IncrementalRemoteKeyedStateHandle.java   | 136 +
 .../changelog/ChangelogStateBackendHandle.java |   2 +-
 .../checkpoint/CheckpointCoordinatorTest.java  |   2 +-
 .../PrioritizedOperatorSubtaskStateTest.java   | 184 ++-
 .../runtime/checkpoint/StateHandleDummyUtil.java   |  29 +-
 .../flink/runtime/state/ChangelogTestUtils.java|   2 +-
 .../IncrementalRemoteKeyedStateHandleTest.java |  18 +-
 .../flink/runtime/testutils/CommonTestUtils.java   |  27 +
 .../state/RocksDBIncrementalCheckpointUtils.java   |   8 +-
 .../streaming/state/StateHandleDownloadSpec.java   |  12 +
 .../RocksDBIncrementalRestoreOperation.java| 235 +
 ...scalingITCase.java => AutoRescalingITCase.java} | 549 +
 .../UpdateJobResourceRequirementsITCase.java   |   2 +-
 19 files changed, 894 insertions(+), 816 deletions(-)
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractIncrementalStateHandle.java
 delete mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java
 copy 
flink-tests/src/test/java/org/apache/flink/test/checkpointing/{RescalingITCase.java
 => AutoRescalingITCase.java} (62%)



(flink) 01/05: [FLINK-33246][tests] Add AutoRescalingITCase.

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

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

commit 98e4610f09f35a942e55472b5d358ebe113b0dba
Author: Stefan Richter 
AuthorDate: Mon Oct 9 14:23:47 2023 +0200

[FLINK-33246][tests] Add AutoRescalingITCase.
---
 .../flink/runtime/testutils/CommonTestUtils.java   |  27 +
 .../test/checkpointing/AutoRescalingITCase.java| 968 +
 .../UpdateJobResourceRequirementsITCase.java   |   2 +-
 3 files changed, 996 insertions(+), 1 deletion(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
index a101a453ff0..da016150500 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.testutils;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobStatus;
 import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
 import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.AccessExecutionGraph;
@@ -366,6 +367,32 @@ public class CommonTestUtils {
 });
 }
 
+/** Wait for on more completed checkpoint. */
+public static void waitForOneMoreCheckpoint(JobID jobID, MiniCluster 
miniCluster)
+throws Exception {
+final long[] currentCheckpoint = new long[] {-1L};
+waitUntilCondition(
+() -> {
+AccessExecutionGraph graph = 
miniCluster.getExecutionGraph(jobID).get();
+CheckpointStatsSnapshot snapshot = 
graph.getCheckpointStatsSnapshot();
+if (snapshot != null) {
+long currentCount = 
snapshot.getCounts().getNumberOfCompletedCheckpoints();
+if (currentCheckpoint[0] < 0L) {
+currentCheckpoint[0] = currentCount;
+} else {
+return currentCount > currentCheckpoint[0];
+}
+} else if (graph.getState().isGloballyTerminalState()) {
+checkState(
+graph.getFailureInfo() != null,
+"Job terminated before taking required 
checkpoint.",
+graph.getState());
+throw graph.getFailureInfo().getException();
+}
+return false;
+});
+}
+
 /**
  * @return the path as {@link java.net.URI} to the latest checkpoint.
  * @throws FlinkJobNotFoundException if job not found
diff --git 
a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java
 
b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java
new file mode 100644
index 000..403449a388b
--- /dev/null
+++ 
b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AutoRescalingITCase.java
@@ -0,0 +1,968 @@
+/*
+ * 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.test.checkpointing;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import

(flink) 05/05: [FLINK-33341][state] Add support for rescaling from local keyed state to PrioritizedOperatorSubtaskState.

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

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

commit a4ad86fb083f90503938a9c3d816cdda9dc22427
Author: Stefan Richter 
AuthorDate: Fri Oct 13 16:27:39 2023 +0200

[FLINK-33341][state] Add support for rescaling from local keyed state to 
PrioritizedOperatorSubtaskState.
---
 .../PrioritizedOperatorSubtaskState.java   | 132 +--
 .../PrioritizedOperatorSubtaskStateTest.java   | 184 ++---
 .../runtime/checkpoint/StateHandleDummyUtil.java   |  29 +++-
 3 files changed, 305 insertions(+), 40 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
index ef9bcd0440b..e41bcfe7338 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java
@@ -31,10 +31,14 @@ import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import java.util.Optional;
 import java.util.function.BiFunction;
 import java.util.function.Function;
+import java.util.stream.Collectors;
 
 /**
  * This class is a wrapper over multiple alternative {@link 
OperatorSubtaskState} that are (partial)
@@ -286,14 +290,14 @@ public class PrioritizedOperatorSubtaskState {
 }
 
 return new PrioritizedOperatorSubtaskState(
-resolvePrioritizedAlternatives(
+computePrioritizedAlternatives(
 jobManagerState.getManagedKeyedState(),
 managedKeyedAlternatives,
-
eqStateApprover(KeyedStateHandle::getKeyGroupRange)),
-resolvePrioritizedAlternatives(
+KeyedStateHandle::getKeyGroupRange),
+computePrioritizedAlternatives(
 jobManagerState.getRawKeyedState(),
 rawKeyedAlternatives,
-
eqStateApprover(KeyedStateHandle::getKeyGroupRange)),
+KeyedStateHandle::getKeyGroupRange),
 resolvePrioritizedAlternatives(
 jobManagerState.getManagedOperatorState(),
 managedOperatorAlternatives,
@@ -313,22 +317,121 @@ public class PrioritizedOperatorSubtaskState {
 restoredCheckpointId);
 }
 
+/**
+ * This method creates an alternative recovery option by replacing as 
much job manager state
+ * with higher prioritized (=local) alternatives as possible.
+ *
+ * @param jobManagerState the state that the task got assigned from 
the job manager (this
+ * state lives in remote storage).
+ * @param alternativesByPriority local alternatives to the job manager 
state, ordered by
+ * priority.
+ * @param identityExtractor function to extract an identifier from a 
state object.
+ * @return prioritized state alternatives.
+ * @param  the type of the state objects we process.
+ * @param  the type of object that represents the id the 
state object type.
+ */
+
+List> 
computePrioritizedAlternatives(
+StateObjectCollection jobManagerState,
+List> 
alternativesByPriority,
+Function identityExtractor) {
+
+if (alternativesByPriority != null
+&& !alternativesByPriority.isEmpty()
+&& jobManagerState.hasState()) {
+
+Optional> 
mergedAlternative =
+tryComputeMixedLocalAndRemoteAlternative(
+jobManagerState, alternativesByPriority, 
identityExtractor);
+
+// Return the mix of local/remote state as first and pure 
remote state as second
+// alternative (in case that we fail to recover from the local 
state, e.g. because
+// of corruption).
+if (mergedAlternative.isPresent()) {
+return Arrays.asList(mergedAlternative.get(), 
jobManagerState);
+}
+}
+
+return Collections.singletonList(jobManagerState);
+}
+
+/**
+ * This method creates an alternative recovery option by replacing as 
much job manager state
+ * with higher prioritized (=local) alternatives as possible. Returns 
empty Optional if the
+ * JM state is empty or 

[flink] branch master updated (9e6f8e8e127 -> d11cc32948b)

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

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


from 9e6f8e8e127 [FLINK-32680][streaming] Fix the messed up JobVertex name 
when the source nodes chained with a MultipleInput node.
 add d11cc32948b [FLINK-32681] Wait for RocksDBStateDownloader threads on 
failure to ensure proper cleanup.

No new revisions were added by this update.

Summary of changes:
 .../apache/flink/core/fs/CloseableRegistry.java|  26 +
 .../flink/core/fs/CloseableRegistryTest.java   | 108 +
 .../streaming/state/RocksDBStateDownloader.java|  50 --
 .../state/RocksDBStateDownloaderTest.java  |   6 +-
 4 files changed, 158 insertions(+), 32 deletions(-)



[flink] branch master updated (9d7b1663296 -> 1ef99736979)

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

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


from 9d7b1663296 [FLINK-32495][connectors/common] Limit the 
CoordinatorExecutorThreadFactory to be a one-off
 add ab9445aca56 [FLINK-32410] Replace new HashMap<>(size) with 
CollectionUtil.newHashMapWithExpectedSize(size)
 add 4532e41e8b1 [FLINK-32410] Replace new HashSet<>(size) with 
CollectionUtil.newHashSetWithExpectedSize(size)
 add 4c5bca08a7b [FLINK-32410] Replace new LinkedHashSet<>(size) with 
CollectionUtil.newLinkedHashSetWithExpectedSize(size)
 add 1ef99736979 [FLINK-32410] Replace new LinkedHashMap<>(size) with 
CollectionUtil.newLinkedHashMapWithExpectedSize(size)

No new revisions were added by this update.

Summary of changes:
 .../flink/connector/file/table/PartitionWriterTest.java   |  4 +++-
 .../flink/connectors/hive/util/HivePartitionUtils.java|  3 ++-
 .../table/functions/hive/conversion/HiveInspectors.java   |  6 +++---
 .../planner/delegation/hive/HiveParserCalcitePlanner.java |  3 ++-
 .../hive/copy/HiveParserBaseSemanticAnalyzer.java | 11 +++
 .../flink/api/common/accumulators/AccumulatorHelper.java  |  6 --
 .../flink/api/common/typeutils/base/MapSerializer.java|  5 +++--
 .../apache/flink/api/java/typeutils/TupleTypeInfo.java|  4 ++--
 .../flink/api/java/typeutils/runtime/PojoSerializer.java  |  8 +---
 .../java/typeutils/runtime/PojoSerializerSnapshot.java|  9 +
 .../typeutils/runtime/PojoSerializerSnapshotData.java |  7 ---
 .../flink/api/java/typeutils/runtime/ValueSerializer.java |  4 +++-
 .../api/java/typeutils/runtime/kryo/KryoSerializer.java   |  7 +--
 .../org/apache/flink/configuration/Configuration.java |  4 +++-
 .../main/java/org/apache/flink/util/CollectionUtil.java   |  4 ++--
 .../java/org/apache/flink/util/InstantiationUtil.java |  6 --
 .../java/org/apache/flink/util/LinkedOptionalMap.java |  8 +---
 .../java/org/apache/flink/util/CollectionUtilTest.java| 14 +++---
 .../docs/configuration/ConfigOptionsDocGenerator.java |  4 ++--
 .../tests/artificialstate/ArtificialKeyedStateMapper.java |  5 +++--
 .../flink/table/tpcds/schema/TpcdsSchemaProvider.java |  5 +++--
 .../flink/formats/avro/RowDataToAvroConverters.java   |  5 +++--
 .../flink/api/java/utils/MultipleParameterTool.java   |  6 --
 .../org/apache/flink/api/java/utils/ParameterTool.java|  6 --
 .../src/main/java/org/apache/flink/cep/nfa/NFA.java   |  3 ++-
 .../flink/cep/nfa/sharedbuffer/SharedBufferAccessor.java  |  4 +++-
 .../state/api/runtime/metadata/SavepointMetadata.java |  4 ++--
 .../state/api/runtime/metadata/SavepointMetadataV2.java   |  4 ++--
 .../org/apache/flink/runtime/rpc/akka/AkkaRpcService.java |  4 ++--
 .../flink/runtime/checkpoint/CheckpointStatsHistory.java  |  5 +++--
 .../org/apache/flink/runtime/checkpoint/Checkpoints.java  |  4 +++-
 .../apache/flink/runtime/checkpoint/OperatorState.java|  4 ++--
 .../flink/runtime/checkpoint/PendingCheckpoint.java   |  9 +++--
 .../apache/flink/runtime/checkpoint/RescaleMappings.java  |  5 +++--
 .../checkpoint/RoundRobinOperatorStateRepartitioner.java  | 15 +++
 .../runtime/checkpoint/StateAssignmentOperation.java  |  9 ++---
 .../org/apache/flink/runtime/checkpoint/TaskState.java|  4 ++--
 .../flink/runtime/checkpoint/TaskStateAssignment.java | 15 +--
 .../flink/runtime/checkpoint/TaskStateSnapshot.java   |  3 ++-
 .../runtime/checkpoint/metadata/MetadataSerializers.java  |  6 --
 .../checkpoint/metadata/MetadataV2V3SerializerBase.java   |  8 +---
 .../dispatcher/DefaultJobManagerRunnerRegistry.java   |  4 ++--
 .../org/apache/flink/runtime/dispatcher/Dispatcher.java   |  4 +++-
 .../execution/librarycache/BlobLibraryCacheManager.java   |  3 ++-
 .../runtime/executiongraph/DefaultExecutionGraph.java |  7 ---
 .../apache/flink/runtime/executiongraph/Execution.java|  6 --
 .../io/network/partition/ResultPartitionManager.java  |  6 --
 .../io/network/partition/consumer/SingleInputGate.java|  4 ++--
 .../slotpool/LocationPreferenceSlotSelectionStrategy.java |  7 ---
 .../org/apache/flink/runtime/memory/MemoryManager.java|  4 ++--
 .../org/apache/flink/runtime/metrics/ReporterSetup.java   |  5 +++--
 .../apache/flink/runtime/metrics/scope/ScopeFormat.java   |  3 ++-
 .../flink/runtime/operators/sort/SpillChannelManager.java |  5 +++--
 .../resourcemanager/DefaultJobLeaderIdService.java|  4 ++--
 .../flink/runtime/resourcemanager/ResourceManager.java| 10 +-
 .../slotmanager/DeclarativeSlotManager.java   |  3 ++-
 .../apache/flink/runtime/rest/handler/HandlerRequest.java |  5 +++--
 .../flink/runtime/rest/handler/job/JobDetailsHandler.java | 10 ++--

[flink] branch master updated: [FLINK-32441] Fix DefaultSchedulerTest#testTriggerCheckpointAndCompletedAfterStore (#22879)

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

srichter 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 0c787f522ed [FLINK-32441] Fix 
DefaultSchedulerTest#testTriggerCheckpointAndCompletedAfterStore (#22879)
0c787f522ed is described below

commit 0c787f522ed5983a902554a00edaa81bdb622480
Author: Stefan Richter 
AuthorDate: Tue Jun 27 14:49:51 2023 +0200

[FLINK-32441] Fix 
DefaultSchedulerTest#testTriggerCheckpointAndCompletedAfterStore (#22879)
---
 .../java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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 1635cb41d79..81fa4ea31ac 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
@@ -891,7 +891,7 @@ public class DefaultSchedulerTest extends TestLogger {
 final long checkpointId =
 
checkpointCoordinator.getPendingCheckpoints().keySet().iterator().next();
 OneShotLatch latch = new OneShotLatch();
-executor.execute(
+mainThreadExecutor.execute(
 () -> {
 try {
 final AcknowledgeCheckpoint acknowledgeCheckpoint =



[flink] branch master updated: [FLINK-32347][checkpoint] Exceptions from the CompletedCheckpointStore are not registered by the CheckpointFailureManager. (#22793)

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

srichter 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 602261648db [FLINK-32347][checkpoint] Exceptions from the 
CompletedCheckpointStore are not registered by the CheckpointFailureManager. 
(#22793)
602261648db is described below

commit 602261648dbc387c51ae113139486b2d1f0935e2
Author: Stefan Richter 
AuthorDate: Mon Jun 26 19:22:10 2023 +0200

[FLINK-32347][checkpoint] Exceptions from the CompletedCheckpointStore are 
not registered by the CheckpointFailureManager. (#22793)

Currently if an error occurs while saving a completed checkpoint in the 
CompletedCheckpointStore, CheckpointCoordinator doesn't call 
CheckpointFailureManager to handle the error. Such behavior leads to the fact, 
that errors from CompletedCheckpointStore don't increase the failed checkpoints 
count and 'execution.checkpointing.tolerable-failed-checkpoints' option does 
not limit the number of errors of this kind in any way.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 33 
 .../checkpoint/CheckpointCoordinatorTest.java  | 91 ++
 2 files changed, 110 insertions(+), 14 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 585a35ffb1c..41902b4af39 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -1330,6 +1330,7 @@ public class CheckpointCoordinator {
 }
 
 private void reportCompletedCheckpoint(CompletedCheckpoint 
completedCheckpoint) {
+
failureManager.handleCheckpointSuccess(completedCheckpoint.getCheckpointID());
 CompletedCheckpointStats completedCheckpointStats = 
completedCheckpoint.getStatistic();
 if (completedCheckpointStats != null) {
 LOG.trace(
@@ -1403,7 +1404,6 @@ public class CheckpointCoordinator {
 pendingCheckpoint.finalizeCheckpoint(
 checkpointsCleaner, this::scheduleTriggerRequest, 
executor);
 
-
failureManager.handleCheckpointSuccess(pendingCheckpoint.getCheckpointID());
 return completedCheckpoint;
 } catch (Exception e1) {
 // abort the current pending checkpoint if we fails to finalize 
the pending
@@ -1467,23 +1467,28 @@ public class CheckpointCoordinator {
 
checkpointsCleaner.cleanCheckpointOnFailedStoring(completedCheckpoint, 
executor);
 }
 
-reportFailedCheckpoint(checkpointId, exception);
+final CheckpointException checkpointException =
+new CheckpointException(
+"Could not complete the pending checkpoint " + 
checkpointId + '.',
+
CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE,
+exception);
+reportFailedCheckpoint(pendingCheckpoint, checkpointException);
 sendAbortedMessages(tasksToAbort, checkpointId, 
completedCheckpoint.getTimestamp());
-throw new CheckpointException(
-"Could not complete the pending checkpoint " + 
checkpointId + '.',
-CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE,
-exception);
+throw checkpointException;
 }
 }
 
-private void reportFailedCheckpoint(long checkpointId, Exception 
exception) {
-PendingCheckpointStats pendingCheckpointStats =
-statsTracker.getPendingCheckpointStats(checkpointId);
-if (pendingCheckpointStats != null) {
-statsTracker.reportFailedCheckpoint(
-pendingCheckpointStats.toFailedCheckpoint(
-System.currentTimeMillis(), exception));
-}
+private void reportFailedCheckpoint(
+PendingCheckpoint pendingCheckpoint, CheckpointException 
exception) {
+
+failureManager.handleCheckpointException(
+pendingCheckpoint,
+pendingCheckpoint.getProps(),
+exception,
+null,
+job,
+getStatsCallback(pendingCheckpoint),
+statsTracker);
 }
 
 void scheduleTriggerRequest() {
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
index 660671dfa8f..93d565644cb 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
+++ 
b/flink-runti

[flink] branch master updated: [FLINK-32345][state] Improve parallel download of RocksDB incremental state. (#22788)

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

srichter 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 2ed858318f3 [FLINK-32345][state] Improve parallel download of RocksDB 
incremental state. (#22788)
2ed858318f3 is described below

commit 2ed858318f38eb9913e5f4b3019be6b6d0a8e6fb
Author: Stefan Richter 
AuthorDate: Wed Jun 21 12:17:08 2023 +0200

[FLINK-32345][state] Improve parallel download of RocksDB incremental 
state. (#22788)

* [FLINK-32345] Improve parallel download of RocksDB incremental state.

This commit improves RocksDBStateDownloader to support parallelized state 
download across multiple state types and across multiple state handles. This 
can improve our download times for scale-in.
---
 .../java/org/apache/flink/util/CollectionUtil.java |  81 +++
 .../org/apache/flink/util/CollectionUtilTest.java  |  47 ++
 .../state/RocksDBIncrementalCheckpointUtils.java   |   2 +-
 .../streaming/state/RocksDBStateDownloader.java| 121 +---
 .../streaming/state/StateHandleDownloadSpec.java   |  49 +++
 .../RocksDBIncrementalRestoreOperation.java| 157 -
 .../state/RocksDBStateDownloaderTest.java  | 132 +
 7 files changed, 438 insertions(+), 151 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java 
b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
index 8c96e3e3554..18f4c4313c4 100644
--- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java
@@ -19,6 +19,7 @@
 package org.apache.flink.util;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
 
 import javax.annotation.Nullable;
 
@@ -27,7 +28,10 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -45,6 +49,9 @@ public final class CollectionUtil {
 /** A safe maximum size for arrays in the JVM. */
 public static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
 
+/** The default load factor for hash maps create with this util class. */
+static final float HASH_MAP_DEFAULT_LOAD_FACTOR = 0.75f;
+
 private CollectionUtil() {
 throw new AssertionError();
 }
@@ -133,4 +140,78 @@ public final class CollectionUtil {
 }
 return Collections.unmodifiableMap(map);
 }
+
+/**
+ * Creates a new {@link HashMap} of the expected size, i.e. a hash map 
that will not rehash if
+ * expectedSize many keys are inserted, considering the load factor.
+ *
+ * @param expectedSize the expected size of the created hash map.
+ * @return a new hash map instance with enough capacity for the expected 
size.
+ * @param  the type of keys maintained by this map.
+ * @param  the type of mapped values.
+ */
+public static  HashMap newHashMapWithExpectedSize(int 
expectedSize) {
+return new HashMap<>(
+computeRequiredCapacity(expectedSize, 
HASH_MAP_DEFAULT_LOAD_FACTOR),
+HASH_MAP_DEFAULT_LOAD_FACTOR);
+}
+
+/**
+ * Creates a new {@link LinkedHashMap} of the expected size, i.e. a hash 
map that will not
+ * rehash if expectedSize many keys are inserted, considering the load 
factor.
+ *
+ * @param expectedSize the expected size of the created hash map.
+ * @return a new hash map instance with enough capacity for the expected 
size.
+ * @param  the type of keys maintained by this map.
+ * @param  the type of mapped values.
+ */
+public static  LinkedHashMap 
newLinkedHashMapWithExpectedSize(int expectedSize) {
+return new LinkedHashMap<>(
+computeRequiredCapacity(expectedSize, 
HASH_MAP_DEFAULT_LOAD_FACTOR),
+HASH_MAP_DEFAULT_LOAD_FACTOR);
+}
+
+/**
+ * Creates a new {@link HashSet} of the expected size, i.e. a hash set 
that will not rehash if
+ * expectedSize many unique elements are inserted, considering the load 
factor.
+ *
+ * @param expectedSize the expected size of the created hash map.
+ * @return a new hash map instance with enough capacity for the expected 
size.
+ * @param  the type of elements stored by this set.
+ */
+public static  HashSet newHashSetWithExpectedSize(int expectedSize) {
+return new HashSet<>(
+computeRequiredCapacity(expectedSize, 
HASH_MAP_DEFAULT_LOAD_FACTOR),
+HASH_MAP_DEFAULT_LOAD_FACTOR);
+}
+
+/**
+ * Creates a new {@link LinkedHashSet} of

[flink] branch master updated: [FLINK-32326][state] Disable WAL in RocksDBWriteBatchWrapper by default. (#22771)

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

srichter 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 45ba7ee87ca [FLINK-32326][state] Disable WAL in 
RocksDBWriteBatchWrapper by default. (#22771)
45ba7ee87ca is described below

commit 45ba7ee87caee63a0babfd421b7c5eabaa779baa
Author: Stefan Richter 
AuthorDate: Wed Jun 14 19:23:48 2023 +0200

[FLINK-32326][state] Disable WAL in RocksDBWriteBatchWrapper by default. 
(#22771)

* [FLINK-32326][state] Disable WAL in RocksDBWriteBatchWrapper by default.

Disables WAL by default in RocksDBWriteBatchWrapper for the case that now 
explicit WriteOption is passed in. This is the case in all restore operations 
and can impact the performance.

* [hotfix][state] Replace deprecated API call WriteBatch::remove with 
WriteBatch::delete.
---
 .../streaming/state/RocksDBWriteBatchWrapper.java  | 40 ++
 .../state/RocksDBWriteBatchWrapperTest.java| 37 
 2 files changed, 63 insertions(+), 14 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBWriteBatchWrapper.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBWriteBatchWrapper.java
index 3906c74972c..354009e335c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBWriteBatchWrapper.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBWriteBatchWrapper.java
@@ -32,6 +32,9 @@ import javax.annotation.Nonnegative;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * It's a wrapper class around RocksDB's {@link WriteBatch} for writing in 
bulk.
  *
@@ -55,6 +58,9 @@ public class RocksDBWriteBatchWrapper implements 
AutoCloseable {
 
 @Nonnegative private final long batchSize;
 
+/** List of all objects that we need to close in close(). */
+private final List toClose;
+
 public RocksDBWriteBatchWrapper(@Nonnull RocksDB rocksDB, long 
writeBatchSize) {
 this(rocksDB, null, 500, writeBatchSize);
 }
@@ -79,9 +85,9 @@ public class RocksDBWriteBatchWrapper implements 
AutoCloseable {
 Preconditions.checkArgument(batchSize >= 0, "Max batch size have to be 
no negative.");
 
 this.db = rocksDB;
-this.options = options;
 this.capacity = capacity;
 this.batchSize = batchSize;
+this.toClose = new ArrayList<>(2);
 if (this.batchSize > 0) {
 this.batch =
 new WriteBatch(
@@ -89,6 +95,15 @@ public class RocksDBWriteBatchWrapper implements 
AutoCloseable {
 } else {
 this.batch = new WriteBatch(this.capacity * PER_RECORD_BYTES);
 }
+this.toClose.add(this.batch);
+if (options != null) {
+this.options = options;
+} else {
+// Use default write options with disabled WAL
+this.options = new WriteOptions().setDisableWAL(true);
+// We own this object, so we must ensure that we close it.
+this.toClose.add(this.options);
+}
 }
 
 public void put(@Nonnull ColumnFamilyHandle handle, @Nonnull byte[] key, 
@Nonnull byte[] value)
@@ -102,33 +117,30 @@ public class RocksDBWriteBatchWrapper implements 
AutoCloseable {
 public void remove(@Nonnull ColumnFamilyHandle handle, @Nonnull byte[] key)
 throws RocksDBException {
 
-batch.remove(handle, key);
+batch.delete(handle, key);
 
 flushIfNeeded();
 }
 
 public void flush() throws RocksDBException {
-if (options != null) {
-db.write(options, batch);
-} else {
-// use the default WriteOptions, if wasn't provided.
-try (WriteOptions writeOptions = new WriteOptions()) {
-db.write(writeOptions, batch);
-}
-}
+db.write(options, batch);
 batch.clear();
 }
 
-public WriteOptions getOptions() {
+@VisibleForTesting
+WriteOptions getOptions() {
 return options;
 }
 
 @Override
 public void close() throws RocksDBException {
-if (batch.count() != 0) {
-flush();
+try {
+if (batch.count() != 0) {
+flush();
+}
+} finally {
+IOUtils.closeAllQuietly(toClose);
 }
-IOUtils.closeQuietly(batch);
 }
 
 private void flushIfNeeded() throws RocksDBException {
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBWriteBat

[flink] branch release-1.17 updated: [FLINK-31963][state] Fix rescaling bug in recovery from unaligned checkpoints. (#22584) (#22594)

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

srichter 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 8d8a486aaa8 [FLINK-31963][state] Fix rescaling bug in recovery from 
unaligned checkpoints. (#22584) (#22594)
8d8a486aaa8 is described below

commit 8d8a486aaa8360d6beabacc6980280c96bf900ea
Author: Stefan Richter 
AuthorDate: Wed May 17 12:05:04 2023 +0200

[FLINK-31963][state] Fix rescaling bug in recovery from unaligned 
checkpoints. (#22584) (#22594)

This commit fixes problems in StateAssignmentOperation for unaligned 
checkpoints with stateless operators that have upstream operators with output 
partition state or downstream operators with input channel state.

(cherry picked from commit 354c0f455b92c083299d8028f161f0dd113ab614)
---
 .../checkpoint/StateAssignmentOperation.java   |  28 ++--
 .../runtime/checkpoint/TaskStateAssignment.java|  19 ++-
 .../checkpoint/StateAssignmentOperationTest.java   | 178 -
 .../checkpointing/UnalignedCheckpointITCase.java   |  18 ++-
 .../UnalignedCheckpointRescaleITCase.java  | 137 ++--
 .../checkpointing/UnalignedCheckpointTestBase.java |  32 +++-
 6 files changed, 335 insertions(+), 77 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
index 681e0b18df1..e476c6b65ec 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
@@ -136,19 +136,24 @@ public class StateAssignmentOperation {
 
 // repartition state
 for (TaskStateAssignment stateAssignment : vertexAssignments.values()) 
{
-if (stateAssignment.hasNonFinishedState) {
+if (stateAssignment.hasNonFinishedState
+// FLINK-31963: We need to run repartitioning for 
stateless operators that have
+// upstream output or downstream input states.
+|| stateAssignment.hasUpstreamOutputStates()
+|| stateAssignment.hasDownstreamInputStates()) {
 assignAttemptState(stateAssignment);
 }
 }
 
 // actually assign the state
 for (TaskStateAssignment stateAssignment : vertexAssignments.values()) 
{
-// If upstream has output states, even the empty task state should 
be assigned for the
-// current task in order to notify this task that the old states 
will send to it which
-// likely should be filtered.
+// If upstream has output states or downstream has input states, 
even the empty task
+// state should be assigned for the current task in order to 
notify this task that the
+// old states will send to it which likely should be filtered.
 if (stateAssignment.hasNonFinishedState
 || stateAssignment.isFullyFinished
-|| stateAssignment.hasUpstreamOutputStates()) {
+|| stateAssignment.hasUpstreamOutputStates()
+|| stateAssignment.hasDownstreamInputStates()) {
 assignTaskStateToExecutionJobVertices(stateAssignment);
 }
 }
@@ -345,9 +350,10 @@ public class StateAssignmentOperation {
 newParallelism)));
 }
 
-public > void 
reDistributeResultSubpartitionStates(
-TaskStateAssignment assignment) {
-if (!assignment.hasOutputState) {
+public void reDistributeResultSubpartitionStates(TaskStateAssignment 
assignment) {
+// FLINK-31963: We can skip this phase if there is no output state AND 
downstream has no
+// input states
+if (!assignment.hasOutputState && 
!assignment.hasDownstreamInputStates()) {
 return;
 }
 
@@ -394,7 +400,9 @@ public class StateAssignmentOperation {
 }
 
 public void reDistributeInputChannelStates(TaskStateAssignment 
stateAssignment) {
-if (!stateAssignment.hasInputState) {
+// FLINK-31963: We can skip this phase only if there is no input state 
AND upstream has no
+// output states
+if (!stateAssignment.hasInputState && 
!stateAssignment.hasUpstreamOutputStates()) {
 return;
 }
 
@@ -435,7 +443,7 @@ public class StateAssignmentOperation {
 : getPartitionState(
 inputOperatorState, 
InputChannelInfo::getGateIdx, gateIndex);
 final MappingBasedRepartitioner 
repartitioner =
-new MappingBasedRepartitioner(mapping);
+

[flink] branch release-1.16 updated (ffa58e1de3d -> 2203bc3bdc9)

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

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


from ffa58e1de3d [FLINK-31418][network][tests] Fix unstable test case 
SortMergeResultPartitionReadSchedulerTest.testRequestBufferTimeout
 add 2203bc3bdc9 [FLINK-31963][state] Fix rescaling bug in recovery from 
unaligned checkpoints. (#22584) (#22595)

No new revisions were added by this update.

Summary of changes:
 .../checkpoint/StateAssignmentOperation.java   |  28 ++--
 .../runtime/checkpoint/TaskStateAssignment.java|  19 ++-
 .../checkpoint/StateAssignmentOperationTest.java   | 178 -
 .../checkpointing/UnalignedCheckpointITCase.java   |  18 ++-
 .../UnalignedCheckpointRescaleITCase.java  | 137 ++--
 .../checkpointing/UnalignedCheckpointTestBase.java |  32 +++-
 6 files changed, 335 insertions(+), 77 deletions(-)



[flink] branch master updated: [FLINK-31963][state] Fix rescaling bug in recovery from unaligned checkpoints. (#22584)

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

srichter 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 354c0f455b9 [FLINK-31963][state] Fix rescaling bug in recovery from 
unaligned checkpoints. (#22584)
354c0f455b9 is described below

commit 354c0f455b92c083299d8028f161f0dd113ab614
Author: Stefan Richter 
AuthorDate: Tue May 16 13:06:05 2023 +0200

[FLINK-31963][state] Fix rescaling bug in recovery from unaligned 
checkpoints. (#22584)

This commit fixes problems in StateAssignmentOperation for unaligned 
checkpoints with stateless operators that have upstream operators with output 
partition state or downstream operators with input channel state.
---
 .../checkpoint/StateAssignmentOperation.java   |  28 ++--
 .../runtime/checkpoint/TaskStateAssignment.java|  19 ++-
 .../checkpoint/StateAssignmentOperationTest.java   | 178 -
 .../checkpointing/UnalignedCheckpointITCase.java   |  18 ++-
 .../UnalignedCheckpointRescaleITCase.java  | 137 ++--
 .../checkpointing/UnalignedCheckpointTestBase.java |  32 +++-
 6 files changed, 335 insertions(+), 77 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
index 681e0b18df1..e476c6b65ec 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
@@ -136,19 +136,24 @@ public class StateAssignmentOperation {
 
 // repartition state
 for (TaskStateAssignment stateAssignment : vertexAssignments.values()) 
{
-if (stateAssignment.hasNonFinishedState) {
+if (stateAssignment.hasNonFinishedState
+// FLINK-31963: We need to run repartitioning for 
stateless operators that have
+// upstream output or downstream input states.
+|| stateAssignment.hasUpstreamOutputStates()
+|| stateAssignment.hasDownstreamInputStates()) {
 assignAttemptState(stateAssignment);
 }
 }
 
 // actually assign the state
 for (TaskStateAssignment stateAssignment : vertexAssignments.values()) 
{
-// If upstream has output states, even the empty task state should 
be assigned for the
-// current task in order to notify this task that the old states 
will send to it which
-// likely should be filtered.
+// If upstream has output states or downstream has input states, 
even the empty task
+// state should be assigned for the current task in order to 
notify this task that the
+// old states will send to it which likely should be filtered.
 if (stateAssignment.hasNonFinishedState
 || stateAssignment.isFullyFinished
-|| stateAssignment.hasUpstreamOutputStates()) {
+|| stateAssignment.hasUpstreamOutputStates()
+|| stateAssignment.hasDownstreamInputStates()) {
 assignTaskStateToExecutionJobVertices(stateAssignment);
 }
 }
@@ -345,9 +350,10 @@ public class StateAssignmentOperation {
 newParallelism)));
 }
 
-public > void 
reDistributeResultSubpartitionStates(
-TaskStateAssignment assignment) {
-if (!assignment.hasOutputState) {
+public void reDistributeResultSubpartitionStates(TaskStateAssignment 
assignment) {
+// FLINK-31963: We can skip this phase if there is no output state AND 
downstream has no
+// input states
+if (!assignment.hasOutputState && 
!assignment.hasDownstreamInputStates()) {
 return;
 }
 
@@ -394,7 +400,9 @@ public class StateAssignmentOperation {
 }
 
 public void reDistributeInputChannelStates(TaskStateAssignment 
stateAssignment) {
-if (!stateAssignment.hasInputState) {
+// FLINK-31963: We can skip this phase only if there is no input state 
AND upstream has no
+// output states
+if (!stateAssignment.hasInputState && 
!stateAssignment.hasUpstreamOutputStates()) {
 return;
 }
 
@@ -435,7 +443,7 @@ public class StateAssignmentOperation {
 : getPartitionState(
 inputOperatorState, 
InputChannelInfo::getGateIdx, gateIndex);
 final MappingBasedRepartitioner 
repartitioner =
-new MappingBasedRepartitioner(mapping);
+new MappingBasedRepartitioner<>(mapping);
 

[flink] branch master updated (de7440e -> 3d51e6c3d)

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

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


from de7440e  [FLINK-15658][table-planner-blink] Fix duplicate field names 
exception when join on the same key multiple times (#11011)
 add 3d51e6c3d [hotfix] [javadoc] RocksFullSnapshotStrategy remark 
DESCRIPTION correctly

No new revisions were added by this update.

Summary of changes:
 .../contrib/streaming/state/snapshot/RocksFullSnapshotStrategy.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)



[flink] branch master updated (61d2525 -> c75af84)

2019-07-22 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from 61d2525  [FLINK-13323][table-runtime-blink] Add tests for complex data 
formats
 add 23f726c  [FLINK-13326] Add closeInterruptibly()
 add 3c38352  [FLINK-13326] Use ResourceGuard in NonClosingCheckpointStream
 add c75af84  [FLINK-13326] Support async rawState checkpointing

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/flink/util/ResourceGuard.java  |  39 ++--
 .../state/KeyedStateCheckpointOutputStream.java|   4 +-
 .../state/NonClosingCheckpointOutputStream.java|  24 -
 .../state/OperatorStateCheckpointOutputStream.java |   2 +-
 .../state/StateSnapshotContextSynchronousImpl.java | 108 ++---
 .../api/operators/AbstractStreamOperator.java  |  19 ++--
 .../api/operators/AbstractStreamOperatorTest.java  |   4 -
 .../StateSnapshotContextSynchronousImplTest.java   |  43 +++-
 8 files changed, 187 insertions(+), 56 deletions(-)



[flink] branch release-1.9 updated: [FLINK-13249][runtime] Fix handling of partition producer responses by running them with the task's executor

2019-07-18 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.9 by this push:
 new 3eff638  [FLINK-13249][runtime] Fix handling of partition producer 
responses by running them with the task's executor
3eff638 is described below

commit 3eff6387b5f6716dee5c17b71b10c08760b946cc
Author: Stefan Richter 
AuthorDate: Thu Jul 18 10:37:22 2019 +0200

[FLINK-13249][runtime] Fix handling of partition producer responses by 
running them with the task's executor

Fixes the problem in FLINK-13249 by ensuring that processing the partition 
producer response is not blocking any netty thread, but is always executed by 
the task's executor.

(cherry picked from commit 23bd23b325f15c726fcc48c76eb252fa2ed2fe59)
---
 .../partition/PartitionProducerStateProvider.java|  9 +
 .../io/network/partition/consumer/SingleInputGate.java   |  6 +++---
 .../java/org/apache/flink/runtime/taskmanager/Task.java  | 16 ++--
 .../partition/consumer/SingleInputGateBuilder.java   |  6 +-
 .../org/apache/flink/runtime/taskmanager/TaskTest.java   |  8 
 5 files changed, 23 insertions(+), 22 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
index 8bbdaa5..5785095 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
@@ -22,7 +22,7 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.types.Either;
 
-import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
 
 /**
  * Request execution state of partition producer, the response accepts state 
check callbacks.
@@ -34,11 +34,12 @@ public interface PartitionProducerStateProvider {
 * @param intermediateDataSetId ID of the parent intermediate data set.
 * @param resultPartitionId ID of the result partition to check. This
 * identifies the producing execution and partition.
-* @return a future with response handle.
+* @param responseConsumer consumer for the response handle.
 */
-   CompletableFuture 
requestPartitionProducerState(
+   void requestPartitionProducerState(
IntermediateDataSetID intermediateDataSetId,
-   ResultPartitionID resultPartitionId);
+   ResultPartitionID resultPartitionId,
+   Consumer responseConsumer);
 
/**
 * Result of state query, accepts state check callbacks.
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
index bd75262..534078d9 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
@@ -601,8 +601,8 @@ public class SingleInputGate extends InputGate {
void triggerPartitionStateCheck(ResultPartitionID partitionId) {
partitionProducerStateProvider.requestPartitionProducerState(
consumedResultId,
-   partitionId)
-   .thenAccept(responseHandle -> {
+   partitionId,
+   ((PartitionProducerStateProvider.ResponseHandle 
responseHandle) -> {
boolean isProducingState = new 
RemoteChannelStateChecker(partitionId, owningTaskName)

.isProducerReadyOrAbortConsumption(responseHandle);
if (isProducingState) {
@@ -612,7 +612,7 @@ public class SingleInputGate extends InputGate {

responseHandle.failConsumption(t);
}
}
-   });
+   }));
}
 
private void queueChannel(InputChannel channel) {
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index d4e1d8a..12049f2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -99,6 +99,7 @@ 

[flink] branch master updated: [FLINK-13249][runtime] Fix handling of partition producer responses b… (#9138)

2019-07-18 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 23bd23b  [FLINK-13249][runtime] Fix handling of partition producer 
responses b… (#9138)
23bd23b is described below

commit 23bd23b325f15c726fcc48c76eb252fa2ed2fe59
Author: Stefan Richter 
AuthorDate: Thu Jul 18 10:37:22 2019 +0200

[FLINK-13249][runtime] Fix handling of partition producer responses b… 
(#9138)

* [FLINK-13249][runtime] Fix handling of partition producer responses by 
running them with the task's executor

* Review comments
---
 .../partition/PartitionProducerStateProvider.java|  9 +
 .../io/network/partition/consumer/SingleInputGate.java   |  6 +++---
 .../java/org/apache/flink/runtime/taskmanager/Task.java  | 16 ++--
 .../partition/consumer/SingleInputGateBuilder.java   |  6 +-
 .../org/apache/flink/runtime/taskmanager/TaskTest.java   |  8 
 5 files changed, 23 insertions(+), 22 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
index 8bbdaa5..5785095 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider.java
@@ -22,7 +22,7 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.types.Either;
 
-import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
 
 /**
  * Request execution state of partition producer, the response accepts state 
check callbacks.
@@ -34,11 +34,12 @@ public interface PartitionProducerStateProvider {
 * @param intermediateDataSetId ID of the parent intermediate data set.
 * @param resultPartitionId ID of the result partition to check. This
 * identifies the producing execution and partition.
-* @return a future with response handle.
+* @param responseConsumer consumer for the response handle.
 */
-   CompletableFuture 
requestPartitionProducerState(
+   void requestPartitionProducerState(
IntermediateDataSetID intermediateDataSetId,
-   ResultPartitionID resultPartitionId);
+   ResultPartitionID resultPartitionId,
+   Consumer responseConsumer);
 
/**
 * Result of state query, accepts state check callbacks.
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
index bd75262..534078d9 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
@@ -601,8 +601,8 @@ public class SingleInputGate extends InputGate {
void triggerPartitionStateCheck(ResultPartitionID partitionId) {
partitionProducerStateProvider.requestPartitionProducerState(
consumedResultId,
-   partitionId)
-   .thenAccept(responseHandle -> {
+   partitionId,
+   ((PartitionProducerStateProvider.ResponseHandle 
responseHandle) -> {
boolean isProducingState = new 
RemoteChannelStateChecker(partitionId, owningTaskName)

.isProducerReadyOrAbortConsumption(responseHandle);
if (isProducingState) {
@@ -612,7 +612,7 @@ public class SingleInputGate extends InputGate {

responseHandle.failConsumption(t);
}
}
-   });
+   }));
}
 
private void queueChannel(InputChannel channel) {
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index d4e1d8a..12049f2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -99,6 +99,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReferenceFieldU

[flink] branch release-1.9 updated: [FLINK-13256] Ensure periodical checkpointing continues when regional failover aborts pending checkpoints

2019-07-18 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.9 by this push:
 new b7bfafc  [FLINK-13256] Ensure periodical checkpointing continues when 
regional failover aborts pending checkpoints
b7bfafc is described below

commit b7bfafca14cd6995a6a59d68d14b4bc3cf91cfb5
Author: Yun Tang 
AuthorDate: Thu Jul 18 09:58:21 2019 +0200

[FLINK-13256] Ensure periodical checkpointing continues when regional 
failover aborts pending checkpoints

This closes #9128.

(cherry picked from commit 1ec34249a0303ae64d049d177057ef9b6c413ab5)
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 139 ---
 .../executiongraph/failover/FailoverRegion.java|   2 +-
 .../FailoverStrategyCheckpointCoordinatorTest.java | 186 +
 3 files changed, 264 insertions(+), 63 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 3dc5c1d..682685c 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -478,32 +478,9 @@ public class CheckpointCoordinator {
throw new 
CheckpointException(CheckpointFailureReason.ALREADY_QUEUED);
}
 
-   // if too many checkpoints are currently in 
progress, we need to mark that a request is queued
-   if (pendingCheckpoints.size() >= 
maxConcurrentCheckpointAttempts) {
-   triggerRequestQueued = true;
-   if (currentPeriodicTrigger != null) {
-   
currentPeriodicTrigger.cancel(false);
-   currentPeriodicTrigger = null;
-   }
-   throw new 
CheckpointException(CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS);
-   }
+   checkConcurrentCheckpoints();
 
-   // make sure the minimum interval between 
checkpoints has passed
-   final long earliestNext = 
lastCheckpointCompletionNanos + minPauseBetweenCheckpointsNanos;
-   final long durationTillNextMillis = 
(earliestNext - System.nanoTime()) / 1_000_000;
-
-   if (durationTillNextMillis > 0) {
-   if (currentPeriodicTrigger != null) {
-   
currentPeriodicTrigger.cancel(false);
-   currentPeriodicTrigger = null;
-   }
-   // Reassign the new trigger to the 
currentPeriodicTrigger
-   currentPeriodicTrigger = 
timer.scheduleAtFixedRate(
-   new ScheduledTrigger(),
-   durationTillNextMillis, 
baseInterval, TimeUnit.MILLISECONDS);
-
-   throw new 
CheckpointException(CheckpointFailureReason.MINIMUM_TIME_BETWEEN_CHECKPOINTS);
-   }
+   checkMinPauseBetweenCheckpoints();
}
}
 
@@ -623,32 +600,9 @@ public class CheckpointCoordinator {
throw new 
CheckpointException(CheckpointFailureReason.ALREADY_QUEUED);
}
 
-   if (pendingCheckpoints.size() 
>= maxConcurrentCheckpointAttempts) {
-   triggerRequestQueued = 
true;
-   if 
(currentPeriodicTrigger != null) {
-   
currentPeriodicTrigger.cancel(false);
-   
currentPeriodicTrigger = null;
-   }
-   throw new 
CheckpointException(CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS);
-   }
-
-   // make sure the minimum 
interval between checkpoints has passed
-   final long ear

[flink] branch master updated: [FLINK-13256] Ensure periodical checkpointing continues when regional failover aborts pending checkpoints

2019-07-18 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 1ec3424  [FLINK-13256] Ensure periodical checkpointing continues when 
regional failover aborts pending checkpoints
1ec3424 is described below

commit 1ec34249a0303ae64d049d177057ef9b6c413ab5
Author: Yun Tang 
AuthorDate: Thu Jul 18 15:58:21 2019 +0800

[FLINK-13256] Ensure periodical checkpointing continues when regional 
failover aborts pending checkpoints

This closes #9128.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 139 ---
 .../executiongraph/failover/FailoverRegion.java|   2 +-
 .../FailoverStrategyCheckpointCoordinatorTest.java | 186 +
 3 files changed, 264 insertions(+), 63 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 3dc5c1d..682685c 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -478,32 +478,9 @@ public class CheckpointCoordinator {
throw new 
CheckpointException(CheckpointFailureReason.ALREADY_QUEUED);
}
 
-   // if too many checkpoints are currently in 
progress, we need to mark that a request is queued
-   if (pendingCheckpoints.size() >= 
maxConcurrentCheckpointAttempts) {
-   triggerRequestQueued = true;
-   if (currentPeriodicTrigger != null) {
-   
currentPeriodicTrigger.cancel(false);
-   currentPeriodicTrigger = null;
-   }
-   throw new 
CheckpointException(CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS);
-   }
+   checkConcurrentCheckpoints();
 
-   // make sure the minimum interval between 
checkpoints has passed
-   final long earliestNext = 
lastCheckpointCompletionNanos + minPauseBetweenCheckpointsNanos;
-   final long durationTillNextMillis = 
(earliestNext - System.nanoTime()) / 1_000_000;
-
-   if (durationTillNextMillis > 0) {
-   if (currentPeriodicTrigger != null) {
-   
currentPeriodicTrigger.cancel(false);
-   currentPeriodicTrigger = null;
-   }
-   // Reassign the new trigger to the 
currentPeriodicTrigger
-   currentPeriodicTrigger = 
timer.scheduleAtFixedRate(
-   new ScheduledTrigger(),
-   durationTillNextMillis, 
baseInterval, TimeUnit.MILLISECONDS);
-
-   throw new 
CheckpointException(CheckpointFailureReason.MINIMUM_TIME_BETWEEN_CHECKPOINTS);
-   }
+   checkMinPauseBetweenCheckpoints();
}
}
 
@@ -623,32 +600,9 @@ public class CheckpointCoordinator {
throw new 
CheckpointException(CheckpointFailureReason.ALREADY_QUEUED);
}
 
-   if (pendingCheckpoints.size() 
>= maxConcurrentCheckpointAttempts) {
-   triggerRequestQueued = 
true;
-   if 
(currentPeriodicTrigger != null) {
-   
currentPeriodicTrigger.cancel(false);
-   
currentPeriodicTrigger = null;
-   }
-   throw new 
CheckpointException(CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS);
-   }
-
-   // make sure the minimum 
interval between checkpoints has passed
-   final long earliestNext = 
lastCheckpointCompletionNanos + minPauseBetweenCheckpo

[flink] 03/03: [FLINK-12804] Change mailbox implementation from bounded to unbounded

2019-07-12 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 6cf98b671d889f1003172bc656ded69d1aab9235
Author: Stefan Richter 
AuthorDate: Wed Jul 3 13:32:46 2019 +0200

[FLINK-12804] Change mailbox implementation from bounded to unbounded

This closes #8692.
---
 .../flink/streaming/runtime/tasks/StreamTask.java  |   2 +-
 .../streaming/runtime/tasks/mailbox/Mailbox.java   |  37 +---
 .../runtime/tasks/mailbox/MailboxImpl.java | 201 +
 .../runtime/tasks/mailbox/MailboxSender.java   |  13 +-
 .../tasks/mailbox/execution/MailboxExecutor.java   |  17 +-
 .../execution/MailboxExecutorServiceImpl.java  |  20 --
 .../tasks/mailbox/execution/MailboxProcessor.java  |  56 ++
 .../execution/SuspendedMailboxDefaultAction.java   |   2 +-
 .../runtime/tasks/mailbox/MailboxImplTest.java |  88 ++---
 .../execution/MailboxExecutorServiceImplTest.java  |   6 +-
 .../mailbox/execution/MailboxProcessorTest.java|  13 +-
 .../mailbox/execution/TestMailboxExecutor.java |   6 -
 12 files changed, 92 insertions(+), 369 deletions(-)

diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index a626bba..52e2011 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -484,7 +484,7 @@ public abstract class StreamTask>
cancelTask();
}
finally {
-   mailboxProcessor.cancelMailboxExecution();
+   mailboxProcessor.allActionsCompleted();
cancelables.close();
}
}
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java
index 9b2d95f..deb69e0 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java
@@ -23,7 +23,7 @@ import javax.annotation.Nonnull;
 import java.util.List;
 
 /**
- * A mailbox is basically a blocking queue for inter-thread message exchange 
in form of {@link Runnable} objects between
+ * A mailbox is basically a queue for inter-thread message exchange in form of 
{@link Runnable} objects between
  * multiple producer threads and a single consumer. This has a lifecycle of 
closed -> open -> (quiesced) -> closed.
  */
 public interface Mailbox extends MailboxReceiver, MailboxSender {
@@ -56,36 +56,12 @@ public interface Mailbox extends MailboxReceiver, 
MailboxSender {
List close();
 
/**
-* The effect of this is that all pending letters in the mailbox are 
dropped and the given priorityLetter
-* is enqueued to the head of the mailbox. Dropped letters are 
returned. This method should only be invoked
-* by code that has ownership of the mailbox object and only rarely 
used, e.g. to submit special events like
-* shutting down the mailbox loop.
-*
-* @param priorityLetter action to enqueue atomically after the mailbox 
was cleared.
-* @throws MailboxStateException if the mailbox is quiesced or closed.
-*/
-   @Nonnull
-   List clearAndPut(@Nonnull Runnable priorityLetter) throws 
MailboxStateException;
-
-   /**
-* Adds the given action to the head of the mailbox. This method will 
block if the mailbox is full and
-* should therefore only be called from outside the mailbox main-thread 
to avoid deadlocks.
-*
-* @param priorityLetter action to enqueue to the head of the mailbox.
-* @throws InterruptedException on interruption.
-* @throws MailboxStateException if the mailbox is quiesced or closed.
-*/
-   void putFirst(@Nonnull Runnable priorityLetter) throws 
InterruptedException, MailboxStateException;
-
-   /**
-* Adds the given action to the head of the mailbox if the mailbox is 
not full. Returns true if the letter
-* was successfully added to the mailbox.
+* Adds the given action to the head of the mailbox.
 *
 * @param priorityLetter action to enqueue to the head of the mailbox.
-* @return true if the letter was successfully added.
 * @throws MailboxStateException if the mailbox is quiesced or closed.
 */
-   boolean tryPutFirst(@Nonnull Runnable priorityLetter) throws 
MailboxStateException;
+   void putFirst(@Nonnull Runnable priorityLetter) throws 
MailboxSt

[flink] 02/03: [FLINK-12804] Introduce mailbox-based ExecutorService

2019-07-12 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 5b24aa0302074a5eaf6c1b47c2764e9883b674ab
Author: Stefan Richter 
AuthorDate: Fri Jun 21 18:07:03 2019 +0200

[FLINK-12804] Introduce mailbox-based ExecutorService
---
 .../flink/state/api/output/BoundedStreamTask.java  |   3 +-
 .../flink/runtime/concurrent/FutureUtils.java  |  26 ++
 .../streaming/runtime/tasks/SourceStreamTask.java  |  66 ++---
 .../runtime/tasks/StreamIterationHead.java |   3 +-
 .../flink/streaming/runtime/tasks/StreamTask.java  |  94 ++-
 .../streaming/runtime/tasks/mailbox/Mailbox.java   |  78 +-
 .../runtime/tasks/mailbox/MailboxImpl.java | 195 --
 .../runtime/tasks/mailbox/MailboxReceiver.java |   6 +-
 .../runtime/tasks/mailbox/MailboxSender.java   |   6 +-
 .../{Mailbox.java => MailboxStateException.java}   |  29 +-
 .../DefaultActionContext.java} |  24 +-
 .../MailboxDefaultAction.java} |  19 +-
 .../tasks/mailbox/execution/MailboxExecutor.java   |  86 ++
 .../MailboxExecutorService.java}   |  19 +-
 .../execution/MailboxExecutorServiceImpl.java  | 157 +++
 .../tasks/mailbox/execution/MailboxProcessor.java  | 295 +
 .../SuspendedMailboxDefaultAction.java}|  16 +-
 .../tasks/StreamTaskSelectiveReadingTest.java  |   3 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   3 +-
 .../streaming/runtime/tasks/StreamTaskTest.java|  13 +-
 .../runtime/tasks/SynchronousCheckpointITCase.java |   5 +-
 .../runtime/tasks/SynchronousCheckpointTest.java   |   3 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |   3 +-
 .../runtime/tasks/mailbox/MailboxImplTest.java | 185 -
 .../execution/MailboxExecutorServiceImplTest.java  | 178 +
 .../mailbox/execution/MailboxProcessorTest.java| 265 ++
 .../mailbox/execution/TestMailboxExecutor.java |  70 +
 .../flink/streaming/util/MockStreamTask.java   |   3 +-
 .../jobmaster/JobMasterStopWithSavepointIT.java|   5 +-
 29 files changed, 1636 insertions(+), 222 deletions(-)

diff --git 
a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java
 
b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java
index db663da..2ce9626 100644
--- 
a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java
+++ 
b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java
@@ -28,6 +28,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import 
org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.OutputTag;
 import org.apache.flink.util.Preconditions;
@@ -76,7 +77,7 @@ class BoundedStreamTask & Bo
}
 
@Override
-   protected void performDefaultAction(ActionContext context) throws 
Exception {
+   protected void performDefaultAction(DefaultActionContext context) 
throws Exception {
if (input.hasNext()) {
reuse.replace(input.next());
headOperator.setKeyContextElement1(reuse);
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
index c1613c5..1db1d2a 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java
@@ -28,9 +28,12 @@ import org.apache.flink.util.function.SupplierWithException;
 
 import akka.dispatch.OnComplete;
 
+import javax.annotation.Nonnull;
+
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
@@ -995,4 +998,27 @@ public class FutureUtils {
}
});
}
+
+   /**
+* Cancels all instances of {@link java.util.concurrent.Future} in the 
given list of runnables without interrupting.
+* This method will suppress unexpected exceptions until the whole list 
is processed and then rethrow.
+*
+* @param runnables list of {@link Runnable} candidates to cancel.
+*/
+   public stat

[flink] branch master updated (3059a2f -> 6cf98b6)

2019-07-12 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from 3059a2f  [FLINK-13133] [pubsub] Fix small error in PubSub 
documentation relating to PubSubSink serializer and emulator settings
 new acfbc83  [hotfix] Remove dangerous waiting methods from mailbox
 new 5b24aa0  [FLINK-12804] Introduce mailbox-based ExecutorService
 new 6cf98b6  [FLINK-12804] Change mailbox implementation from bounded to 
unbounded

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


Summary of changes:
 .../flink/state/api/output/BoundedStreamTask.java  |   3 +-
 .../flink/runtime/concurrent/FutureUtils.java  |  26 ++
 .../streaming/runtime/tasks/SourceStreamTask.java  |  66 ++---
 .../runtime/tasks/StreamIterationHead.java |   3 +-
 .../flink/streaming/runtime/tasks/StreamTask.java  |  94 ++-
 .../streaming/runtime/tasks/mailbox/Mailbox.java   |  49 +++-
 .../runtime/tasks/mailbox/MailboxImpl.java | 220 
 .../runtime/tasks/mailbox/MailboxReceiver.java |  12 +-
 .../runtime/tasks/mailbox/MailboxSender.java   |  20 +-
 .../tasks/mailbox/MailboxStateException.java   |  20 +-
 .../mailbox/execution/DefaultActionContext.java|  43 +---
 .../mailbox/execution/MailboxDefaultAction.java|  19 +-
 .../tasks/mailbox/execution/MailboxExecutor.java   |  71 ++
 .../mailbox/execution/MailboxExecutorService.java  |  23 +-
 .../execution/MailboxExecutorServiceImpl.java  | 137 ++
 .../tasks/mailbox/execution/MailboxProcessor.java  | 279 +
 .../execution/SuspendedMailboxDefaultAction.java   |  12 +-
 .../tasks/StreamTaskSelectiveReadingTest.java  |   3 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   3 +-
 .../streaming/runtime/tasks/StreamTaskTest.java|  13 +-
 .../runtime/tasks/SynchronousCheckpointITCase.java |   5 +-
 .../runtime/tasks/SynchronousCheckpointTest.java   |   3 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |   3 +-
 .../runtime/tasks/mailbox/MailboxImplTest.java | 182 ++
 .../execution/MailboxExecutorServiceImplTest.java  | 178 +
 .../mailbox/execution/MailboxProcessorTest.java| 266 
 .../mailbox/execution/TestMailboxExecutor.java |  49 ++--
 .../flink/streaming/util/MockStreamTask.java   |   3 +-
 .../jobmaster/JobMasterStopWithSavepointIT.java|   5 +-
 29 files changed, 1410 insertions(+), 400 deletions(-)
 copy 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlConversionException.java
 => 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java
 (62%)
 copy flink-core/src/main/java/org/apache/flink/util/CloseableIterable.java => 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/DefaultActionContext.java
 (51%)
 copy 
flink-runtime/src/main/java/org/apache/flink/runtime/state/OwnedTaskLocalStateStore.java
 => 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxDefaultAction.java
 (59%)
 create mode 100644 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java
 copy 
flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/IntType.java
 => 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorService.java
 (69%)
 create mode 100644 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorServiceImpl.java
 create mode 100644 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java
 copy flink-runtime/src/main/java/org/apache/flink/runtime/state/Keyed.java => 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/SuspendedMailboxDefaultAction.java
 (75%)
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorServiceImplTest.java
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessorTest.java
 copy 
flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java
 => 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TestMailboxExecutor.java
 (52%)



[flink] 01/03: [hotfix] Remove dangerous waiting methods from mailbox

2019-07-12 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit acfbc83cf17f94a27c1aa71c331ee95382f55596
Author: Stefan Richter 
AuthorDate: Wed Jun 26 12:21:33 2019 +0200

[hotfix] Remove dangerous waiting methods from mailbox
---
 .../flink/streaming/runtime/tasks/StreamTask.java  |  2 +-
 .../runtime/tasks/mailbox/MailboxImpl.java | 26 ---
 .../runtime/tasks/mailbox/MailboxReceiver.java |  6 -
 .../runtime/tasks/mailbox/MailboxSender.java   |  7 --
 .../runtime/tasks/mailbox/MailboxImplTest.java | 29 ++
 5 files changed, 8 insertions(+), 62 deletions(-)

diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 2f4dd6a..8ba2a44 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -1339,7 +1339,7 @@ public abstract class StreamTask>
 */
public final class ActionContext {
 
-   private final Runnable actionUnavailableLetter = 
ThrowingRunnable.unchecked(mailbox::waitUntilHasMail);
+   private final Runnable actionUnavailableLetter = 
ThrowingRunnable.unchecked(() -> mailbox.takeMail().run());
 
/**
 * This method must be called to end the stream task when all 
actions for the tasks have been performed.
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImpl.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImpl.java
index 411efd1..e9bd346 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImpl.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImpl.java
@@ -126,19 +126,6 @@ public class MailboxImpl implements Mailbox {
}
}
 
-   @Override
-   public void waitUntilHasMail() throws InterruptedException {
-   final ReentrantLock lock = this.lock;
-   lock.lockInterruptibly();
-   try {
-   while (isEmpty()) {
-   notEmpty.await();
-   }
-   } finally {
-   lock.unlock();
-   }
-   }
-

//--
 
@Override
@@ -171,19 +158,6 @@ public class MailboxImpl implements Mailbox {
}
}
 
-   @Override
-   public void waitUntilHasCapacity() throws InterruptedException {
-   final ReentrantLock lock = this.lock;
-   lock.lockInterruptibly();
-   try {
-   while (isFull()) {
-   notFull.await();
-   }
-   } finally {
-   lock.unlock();
-   }
-   }
-

//--
 
private void putInternal(Runnable letter) {
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java
index 189687e..2d2f112 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java
@@ -50,10 +50,4 @@ public interface MailboxReceiver {
 */
@Nonnull
Runnable takeMail() throws InterruptedException;
-
-   /**
-* This method blocks if the mailbox is empty until mail becomes 
available.
-* @throws InterruptedException on interruption.
-*/
-   void waitUntilHasMail() throws InterruptedException;
 }
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java
index 1829125..36d10a1 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java
@@ -42,11 +42,4 @@ public interface MailboxSender {
 * @throws InterruptedException on interruption.
 */
void putMail(@N

[flink] branch master updated: [FLINK-13063] Temporary fix for AsyncWaitOperator consistency problems

2019-07-09 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 c773ce5  [FLINK-13063] Temporary fix for AsyncWaitOperator consistency 
problems
c773ce5 is described below

commit c773ce5ff1c7cfb4ee2537c873b24cde4b9a060a
Author: Stefan Richter 
AuthorDate: Tue Jul 9 14:07:24 2019 +0200

[FLINK-13063] Temporary fix for AsyncWaitOperator consistency problems

The current implementation of AsyncWaitOperator can violate exactly-once 
and at-least once guarantees in some common scenarios. This commit provides a 
temporary fix by preventing the operator to be chained after other operators.

This closes #9034.
---
 docs/dev/stream/operators/asyncio.md   |  8 ++
 .../api/operators/async/AsyncWaitOperator.java |  5 +-
 .../api/operators/async/AsyncWaitOperatorTest.java | 86 +++---
 3 files changed, 88 insertions(+), 11 deletions(-)

diff --git a/docs/dev/stream/operators/asyncio.md 
b/docs/dev/stream/operators/asyncio.md
index e92e7a9..c9a9b2c 100644
--- a/docs/dev/stream/operators/asyncio.md
+++ b/docs/dev/stream/operators/asyncio.md
@@ -265,5 +265,13 @@ For example, the following patterns result in a blocking 
`asyncInvoke(...)` func
   - Using a database client whose lookup/query method call blocks until the 
result has been received back
 
   - Blocking/waiting on the future-type objects returned by an asynchronous 
client inside the `asyncInvoke(...)` method
+  
+**The operator for AsyncFunction (AsyncWaitOperator) must currently be at the 
head of operator chains for consistency reasons**
+
+For the reasons given in issue `FLINK-13063`, we currently must break operator 
chains for the `AsyncWaitOperator` to prevent 
+potential consistency problems. This is a change to the previous behavior that 
supported chaining. User that
+require the old behavior and accept potential violations of the consistency 
guarantees can instantiate and add the 
+`AsyncWaitOperator` manually to the job graph and set the chaining strategy 
back to chaining via 
+`AsyncWaitOperator#setChainingStrategy(ChainingStrategy.ALWAYS)`.
 
 {% top %}
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 2555c3b..f875775 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
@@ -118,7 +118,10 @@ public class AsyncWaitOperator
int capacity,
AsyncDataStream.OutputMode outputMode) {
super(asyncFunction);
-   chainingStrategy = ChainingStrategy.ALWAYS;
+
+   // TODO this is a temporary fix for the problems described 
under FLINK-13063 at the cost of breaking chains for
+   //  AsyncOperators.
+   setChainingStrategy(ChainingStrategy.HEAD);
 
Preconditions.checkArgument(capacity > 0, "The number of 
concurrent async operation should be greater than 0.");
this.capacity = capacity;
diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
index 8fdad8f..05df362 100644
--- 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
+++ 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java
@@ -21,7 +21,10 @@ package org.apache.flink.streaming.api.operators.async;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.OneShotLatch;
@@ -40,12 +43,14 @@ import 
org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.runtime.state.TestTaskStateManager;
 import org.apache.flink.streaming.api.datastream.AsyncDataStream;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOper

[flink] branch master updated: [FLINK-12730][runtime] Unify BitSet implementations in flink-runtime

2019-07-09 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 14c4b23  [FLINK-12730][runtime] Unify BitSet implementations in 
flink-runtime
14c4b23 is described below

commit 14c4b23ca8744f83864a8c91ff0c1b88af52e532
Author: liyafan82 <42827532+liyafa...@users.noreply.github.com>
AuthorDate: Tue Jul 9 19:08:08 2019 +0800

[FLINK-12730][runtime] Unify BitSet implementations in flink-runtime

This closes #8613.
---
 .../flink/runtime/operators/util/BitSet.java   | 15 ++--
 .../flink/runtime/operators/util/BloomFilter.java  | 79 --
 .../flink/runtime/operators/util/BitSetTest.java   | 19 +-
 .../runtime/operators/util/BloomFilterTest.java|  7 +-
 4 files changed, 27 insertions(+), 93 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BitSet.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BitSet.java
index b6e1e07..df13caf 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BitSet.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BitSet.java
@@ -32,7 +32,6 @@ public class BitSet {
// The BitSet bit size.
private int bitLength;
 
-   private final int BYTE_POSITION_MASK = 0xfff8;
private final int BYTE_INDEX_MASK = 0x0007;
 
public BitSet(int byteSize) {
@@ -58,7 +57,7 @@ public class BitSet {
public void set(int index) {
Preconditions.checkArgument(index < bitLength && index >= 0);
 
-   int byteIndex = (index & BYTE_POSITION_MASK) >>> 3;
+   int byteIndex = index >>> 3;
byte current = memorySegment.get(offset + byteIndex);
current |= (1 << (index & BYTE_INDEX_MASK));
memorySegment.put(offset + byteIndex, current);
@@ -73,7 +72,7 @@ public class BitSet {
public boolean get(int index) {
Preconditions.checkArgument(index < bitLength && index >= 0);

-   int byteIndex = (index & BYTE_POSITION_MASK) >>> 3;
+   int byteIndex = index >>> 3;
byte current = memorySegment.get(offset + byteIndex);
return (current & (1 << (index & BYTE_INDEX_MASK))) != 0;
}
@@ -89,8 +88,14 @@ public class BitSet {
 * Clear the bit set.
 */
public void clear() {
-   for (int i = 0; i < byteLength; i++) {
-   memorySegment.put(offset + i, (byte) 0);
+   int index = 0;
+   while (index + 8 <= byteLength) {
+   memorySegment.putLong(offset + index, 0L);
+   index += 8;
+   }
+   while (index < byteLength) {
+   memorySegment.put(offset + index, (byte) 0);
+   index += 1;
}
}
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
index 5f09391..5abf5a7 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
@@ -143,83 +143,4 @@ public class BloomFilter {
output.append(bitSet);
return output.toString();
}
-   
-   /**
-* Bare metal bit set implementation. For performance reasons, this 
implementation does not check
-* for index bounds nor expand the bit set size if the specified index 
is greater than the size.
-*/
-   public class BitSet {
-   private MemorySegment memorySegment;
-   // MemorySegment byte array offset.
-   private int offset;
-   // MemorySegment byte size.
-   private int length;
-   private final int LONG_POSITION_MASK = 0xffc0;
-   
-   public BitSet(int byteSize) {
-   checkArgument(byteSize > 0, "bits size should be 
greater than 0.");
-   checkArgument(byteSize << 29 == 0, "bytes size should 
be integral multiple of long size(8 Bytes).");
-   this.length = byteSize;
-   }
-   
-   public void setMemorySegment(MemorySegment memorySegment, int 
offset) {
-   this.memorySegment = memorySegment;
-   this.offset = offset;
-   }
-   
-   /**
-* Sets the bit at 

[flink] branch master updated: [FLINK-12364] Introduce CheckpointFailureManager for centralized checkpoint failure handling

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

srichter 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 8c57e5a  [FLINK-12364] Introduce CheckpointFailureManager for 
centralized checkpoint failure handling
8c57e5a is described below

commit 8c57e5aed871b42d51e6218620ce7bd7991338c0
Author: vinoyang 
AuthorDate: Wed Jun 19 17:00:59 2019 +0800

[FLINK-12364] Introduce CheckpointFailureManager for centralized checkpoint 
failure handling
---
 flink-end-to-end-tests/test-scripts/common.sh  |   2 +-
 .../jobmanager/JMXJobManagerMetricTest.java|   3 +-
 .../runtime/checkpoint/CheckpointCoordinator.java  |  74 ++--
 .../checkpoint/CheckpointFailureManager.java   | 135 +++
 .../checkpoint/CheckpointFailureReason.java|  18 +-
 .../runtime/checkpoint/CheckpointIDCounter.java|   7 +
 .../runtime/checkpoint/PendingCheckpoint.java  |   2 +-
 .../checkpoint/StandaloneCheckpointIDCounter.java  |   5 +
 .../checkpoint/ZooKeeperCheckpointIDCounter.java   |  21 +-
 .../decline/AlignmentLimitExceededException.java   |  33 --
 .../decline/CheckpointDeclineException.java|  35 --
 ...pointDeclineOnCancellationBarrierException.java |  32 --
 .../CheckpointDeclineSubsumedException.java|  32 --
 ...kpointDeclineTaskNotCheckpointingException.java |  32 --
 .../CheckpointDeclineTaskNotReadyException.java|  32 --
 .../decline/InputEndOfStreamException.java |  32 --
 .../runtime/executiongraph/ExecutionGraph.java |  32 +-
 .../executiongraph/ExecutionGraphBuilder.java  |   9 +-
 .../tasks/CheckpointCoordinatorConfiguration.java  |  22 +-
 .../messages/checkpoint/DeclineCheckpoint.java |  16 +-
 .../flink/runtime/taskexecutor/TaskExecutor.java   |   7 +-
 .../exceptions/CheckpointException.java|  41 --
 .../org/apache/flink/runtime/taskmanager/Task.java |   7 +-
 .../CheckpointCoordinatorFailureTest.java  |  14 +-
 .../CheckpointCoordinatorMasterHooksTest.java  |  18 +-
 .../checkpoint/CheckpointCoordinatorTest.java  | 418 -
 .../checkpoint/CheckpointFailureManagerTest.java   | 117 ++
 .../checkpoint/CheckpointIDCounterTest.java|   6 +
 .../CheckpointSettingsSerializableTest.java|   3 +-
 .../checkpoint/CheckpointStateRestoreTest.java |  43 ++-
 .../checkpoint/CheckpointStatsTrackerTest.java |   3 +-
 .../ExecutionGraphCheckpointCoordinatorTest.java   |  25 +-
 .../executiongraph/ArchivedExecutionGraphTest.java |  11 +-
 ...ncurrentFailoverStrategyExecutionGraphTest.java |  12 +-
 .../ExecutionGraphDeploymentTest.java  |   3 +-
 .../runtime/executiongraph/FailoverRegionTest.java |  18 +-
 .../flink/runtime/jobgraph/JobGraphTest.java   |   7 +-
 .../tasks/JobCheckpointingSettingsTest.java|   3 +-
 .../flink/runtime/jobmaster/JobMasterTest.java |   3 +-
 .../api/environment/CheckpointConfig.java  |  27 +-
 .../api/graph/StreamingJobGraphGenerator.java  |   5 +-
 .../flink/streaming/runtime/io/BarrierBuffer.java  |  31 +-
 .../flink/streaming/runtime/io/BarrierTracker.java |   6 +-
 .../io/BarrierBufferAlignmentLimitTest.java|   8 +-
 .../runtime/io/BarrierBufferTestBase.java  |  63 +++-
 .../tasks/StreamTaskCancellationBarrierTest.java   |  11 +-
 .../jobmaster/JobMasterStopWithSavepointIT.java|   3 +-
 .../jobmaster/JobMasterTriggerSavepointITCase.java |   3 +-
 .../test/streaming/runtime/IterateITCase.java  |   4 +-
 49 files changed, 943 insertions(+), 551 deletions(-)

diff --git a/flink-end-to-end-tests/test-scripts/common.sh 
b/flink-end-to-end-tests/test-scripts/common.sh
index 361e49a..4eac9f4 100644
--- a/flink-end-to-end-tests/test-scripts/common.sh
+++ b/flink-end-to-end-tests/test-scripts/common.sh
@@ -361,7 +361,7 @@ function check_logs_for_exceptions {
| grep -v "java.io.InvalidClassException: 
org.apache.flink.formats.avro.typeutils.AvroSerializer" \
| grep -v "Caused by: java.lang.Exception: JobManager is shutting down" \
| grep -v "java.lang.Exception: Artificial failure" \
-   | grep -v "org.apache.flink.runtime.checkpoint.decline" \
+   | grep -v "org.apache.flink.runtime.checkpoint.CheckpointException" \
| grep -v "org.elasticsearch.ElasticsearchException" \
| grep -v "Elasticsearch exception" \
| grep -ic "exception" || true)
diff --git 
a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
 
b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
index b946896..7156adb 100644
--- 
a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
+++ 
b/flink-metrics/flink-metrics-jmx/src/test

[flink] 01/02: [FLINK-12478, FLINK-12480][runtime] Introduce mailbox to StreamTask main-loop.

2019-05-22 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 022f6cceef65859bc6f172151d09140038297f69
Author: Stefan Richter 
AuthorDate: Fri May 10 11:20:02 2019 +0200

[FLINK-12478, FLINK-12480][runtime] Introduce mailbox to StreamTask 
main-loop.

This closes #8409.
This closes #8431.

This also decomposes monolithic run-loops in StreamTask implementations 
into step-wise calls.
---
 .../runtime/tasks/OneInputStreamTask.java  |  12 +-
 .../streaming/runtime/tasks/SourceStreamTask.java  |   5 +-
 .../runtime/tasks/StreamIterationHead.java | 105 -
 .../flink/streaming/runtime/tasks/StreamTask.java  |  82 ++-
 .../runtime/tasks/TwoInputStreamTask.java  |  13 +-
 .../streaming/runtime/tasks/mailbox/Mailbox.java   |  36 
 .../runtime/tasks/mailbox/MailboxImpl.java | 236 +
 .../runtime/tasks/mailbox/MailboxReceiver.java |  59 ++
 .../runtime/tasks/mailbox/MailboxSender.java   |  52 +
 ...heckpointExceptionHandlerConfigurationTest.java |   4 +-
 .../tasks/StreamTaskCancellationBarrierTest.java   |   4 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   3 +-
 .../streaming/runtime/tasks/StreamTaskTest.java|  21 +-
 .../runtime/tasks/SynchronousCheckpointITCase.java |   3 +-
 .../runtime/tasks/SynchronousCheckpointTest.java   |   3 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |   4 +-
 .../runtime/tasks/mailbox/MailboxImplTest.java | 170 +++
 .../flink/streaming/util/MockStreamTask.java   |   4 +-
 .../jobmaster/JobMasterStopWithSavepointIT.java|  10 +-
 19 files changed, 726 insertions(+), 100 deletions(-)

diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
index 7498518..7b82d8f 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
@@ -39,8 +39,6 @@ public class OneInputStreamTask extends 
StreamTask inputProcessor;
 
-   private volatile boolean running = true;
-
private final WatermarkGauge inputWatermarkGauge = new WatermarkGauge();
 
/**
@@ -98,12 +96,9 @@ public class OneInputStreamTask extends 
StreamTask inputProcessor = 
this.inputProcessor;
-
-   while (running && inputProcessor.processInput()) {
-   // all the work happens in the "processInput" method
+   protected void performDefaultAction(ActionContext context) throws 
Exception {
+   if (!inputProcessor.processInput()) {
+   context.allActionsCompleted();
}
}
 
@@ -116,6 +111,5 @@ public class OneInputStreamTask extends 
StreamTask, OP extends S
}
 
@Override
-   protected void run() throws Exception {
+   protected void performDefaultAction(ActionContext context) throws 
Exception {
+   // Against the usual contract of this method, this 
implementation is not step-wise but blocking instead for
+   // compatibility reasons with the current source interface 
(source functions run as a loop, not in steps).
headOperator.run(getCheckpointLock(), 
getStreamStatusMaintainer());
+   context.allActionsCompleted();
}
 
@Override
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
index ecef7f0..d25bd23 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
@@ -24,6 +24,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.io.BlockingQueueBroker;
 import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,88 +42,72 @@ public class StreamIterationHead extends 
OneInputStreamTask {
 
private static final Logger LOG = 
LoggerFactory.getLogger(StreamIterationHead.class);
 
-   private volatile boolean running = true;
+   private RecordWriterOutput[] streamOutputs;
+
+   private final BlockingQueue> dataChannel;
+   private final String brokerID;
+   private final long iterationWaitTime;
+   private final boolean shouldWait;
 
publi

[flink] 02/02: [FLINK-12483][runtime] Support (legacy) SourceFunction as special case in the mailbox model for stream tasks.

2019-05-22 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit ead9139680ea82c4fdfd1e9d19baf4d4a08ec845
Author: Stefan Richter 
AuthorDate: Tue May 14 15:33:48 2019 +0200

[FLINK-12483][runtime] Support (legacy) SourceFunction as special case in 
the mailbox model for stream tasks.

This closes #8442.
---
 .../streaming/runtime/tasks/SourceStreamTask.java  | 66 +-
 1 file changed, 64 insertions(+), 2 deletions(-)

diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
index fd50a1a..e604f2c 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
@@ -45,6 +45,8 @@ import org.apache.flink.util.FlinkException;
 public class SourceStreamTask, OP extends 
StreamSource>
extends StreamTask {
 
+   private static final Runnable SOURCE_POISON_LETTER = () -> {};
+
private volatile boolean externallyInducedCheckpoints;
 
public SourceStreamTask(Environment env) {
@@ -101,12 +103,43 @@ public class SourceStreamTask, OP extends S
protected void performDefaultAction(ActionContext context) throws 
Exception {
// Against the usual contract of this method, this 
implementation is not step-wise but blocking instead for
// compatibility reasons with the current source interface 
(source functions run as a loop, not in steps).
-   headOperator.run(getCheckpointLock(), 
getStreamStatusMaintainer());
+   final LegacySourceFunctionThread sourceThread = new 
LegacySourceFunctionThread();
+   sourceThread.start();
+
+   // We run an alternative mailbox loop that does not involve 
default actions and synchronizes around actions.
+   try {
+   runAlternativeMailboxLoop();
+   } catch (Exception mailboxEx) {
+   // We cancel the source function if some runtime 
exception escaped the mailbox.
+   if (!isCanceled()) {
+   cancelTask();
+   }
+   throw mailboxEx;
+   }
+
+   sourceThread.join();
+   sourceThread.checkThrowSourceExecutionException();
+
context.allActionsCompleted();
}
 
+   private void runAlternativeMailboxLoop() throws InterruptedException {
+
+   while (true) {
+
+   Runnable letter = mailbox.takeMail();
+   if (letter == SOURCE_POISON_LETTER) {
+   break;
+   }
+
+   synchronized (getCheckpointLock()) {
+   letter.run();
+   }
+   }
+   }
+
@Override
-   protected void cancelTask() throws Exception {
+   protected void cancelTask() {
if (headOperator != null) {
headOperator.cancel();
}
@@ -133,4 +166,33 @@ public class SourceStreamTask, OP extends S
}
}
}
+
+   /**
+* Runnable that executes the the source function in the head operator.
+*/
+   private class LegacySourceFunctionThread extends Thread {
+
+   private Throwable sourceExecutionThrowable;
+
+   LegacySourceFunctionThread() {
+   this.sourceExecutionThrowable = null;
+   }
+
+   @Override
+   public void run() {
+   try {
+   headOperator.run(getCheckpointLock(), 
getStreamStatusMaintainer());
+   } catch (Throwable t) {
+   sourceExecutionThrowable = t;
+   } finally {
+   mailbox.clearAndPut(SOURCE_POISON_LETTER);
+   }
+   }
+
+   void checkThrowSourceExecutionException() throws Exception {
+   if (sourceExecutionThrowable != null) {
+   throw new Exception(sourceExecutionThrowable);
+   }
+   }
+   }
 }



[flink] branch master updated (3813bb9 -> ead9139)

2019-05-22 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from 3813bb9  [FLINK-12241][hive] Support Flink functions in catalog 
function APIs of HiveCatalog
 new 022f6cc  [FLINK-12478, FLINK-12480][runtime] Introduce mailbox to 
StreamTask main-loop.
 new ead9139  [FLINK-12483][runtime] Support (legacy) SourceFunction as 
special case in the mailbox model for stream tasks.

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


Summary of changes:
 .../runtime/tasks/OneInputStreamTask.java  |  12 +-
 .../streaming/runtime/tasks/SourceStreamTask.java  |  71 ++-
 .../runtime/tasks/StreamIterationHead.java | 105 -
 .../flink/streaming/runtime/tasks/StreamTask.java  |  82 ++-
 .../runtime/tasks/TwoInputStreamTask.java  |  13 +-
 .../streaming/runtime/tasks/mailbox/Mailbox.java   |  23 +-
 .../runtime/tasks/mailbox/MailboxImpl.java | 236 +
 .../runtime/tasks/mailbox/MailboxReceiver.java |  59 ++
 .../runtime/tasks/mailbox/MailboxSender.java   |  52 +
 ...heckpointExceptionHandlerConfigurationTest.java |   4 +-
 .../tasks/StreamTaskCancellationBarrierTest.java   |   4 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   3 +-
 .../streaming/runtime/tasks/StreamTaskTest.java|  21 +-
 .../runtime/tasks/SynchronousCheckpointITCase.java |   3 +-
 .../runtime/tasks/SynchronousCheckpointTest.java   |   3 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |   4 +-
 .../runtime/tasks/mailbox/MailboxImplTest.java | 170 +++
 .../flink/streaming/util/MockStreamTask.java   |   4 +-
 .../jobmaster/JobMasterStopWithSavepointIT.java|  10 +-
 19 files changed, 762 insertions(+), 117 deletions(-)
 copy 
flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoWriter.java
 => 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java
 (60%)
 create mode 100644 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImpl.java
 create mode 100644 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java
 create mode 100644 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxImplTest.java



[flink] branch release-1.6 updated: [FLINK-12296][StateBackend] Fix local state directory collision with state loss for chained keyed operators

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

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


The following commit(s) were added to refs/heads/release-1.6 by this push:
 new 0dda6fe  [FLINK-12296][StateBackend] Fix local state directory 
collision with state loss for chained keyed operators
0dda6fe is described below

commit 0dda6fe9dff4f667b110cda39bfe9738ba615b24
Author: Congxian Qiu 
AuthorDate: Mon May 13 16:29:34 2019 +0800

[FLINK-12296][StateBackend] Fix local state directory collision with state 
loss for chained keyed operators

This closes #8338.
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  16 +-
 .../tasks/OneInputStreamTaskTestHarness.java   |  43 +++-
 .../runtime/tasks/StreamConfigChainer.java |  23 +-
 .../runtime/tasks/StreamMockEnvironment.java   |   8 +-
 .../runtime/tasks/StreamTaskTestHarness.java   |  21 +-
 .../state/StatefulOperatorChainedTaskTest.java | 259 +
 6 files changed, 357 insertions(+), 13 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 17ba985..62c540f 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -270,6 +270,9 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
/** Shared wrapper for batch writes to the RocksDB instance. */
private RocksDBWriteBatchWrapper writeBatchWrapper;
 
+   /** The local directory name of the current snapshot strategy. */
+   private final String localDirectoryName;
+
public RocksDBKeyedStateBackend(
String operatorIdentifier,
ClassLoader userCodeClassLoader,
@@ -319,6 +322,7 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.restoredKvStateMetaInfos = new HashMap<>();
this.materializedSstFiles = new TreeMap<>();
this.backendUID = UUID.randomUUID();
+   this.localDirectoryName = 
this.backendUID.toString().replaceAll("[\\-]", "");
 
this.snapshotStrategy = enableIncrementalCheckpointing ?
new IncrementalSnapshotStrategy() :
@@ -1977,17 +1981,17 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
LocalRecoveryDirectoryProvider 
directoryProvider = localRecoveryConfig.getLocalStateDirectoryProvider();
File directory = 
directoryProvider.subtaskSpecificCheckpointDirectory(checkpointId);
 
-   if (directory.exists()) {
-   FileUtils.deleteDirectory(directory);
-   }
-
-   if (!directory.mkdirs()) {
+   if (!directory.exists() && !directory.mkdirs()) 
{
throw new IOException("Local state base 
directory for checkpoint " + checkpointId +
" already exists: " + 
directory);
}
 
// introduces an extra directory because 
RocksDB wants a non-existing directory for native checkpoints.
-   File rdbSnapshotDir = new File(directory, 
"rocks_db");
+   // append localDirectoryName here to solve 
directory collision problem when two stateful operators chained in one task.
+   File rdbSnapshotDir = new File(directory, 
localDirectoryName);
+   if (rdbSnapshotDir.exists()) {
+   
FileUtils.deleteDirectory(rdbSnapshotDir);
+   }
Path path = new Path(rdbSnapshotDir.toURI());
// create a "permanent" snapshot directory 
because local recovery is active.
snapshotDirectory = 
SnapshotDirectory.permanent(path);
diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
index 89a4f81..0a7efda 100644
--- 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestH

[flink] branch release-1.7 updated: [FLINK-12296][StateBackend] Fix local state directory collision with state loss for chained keyed operators (#8323)

2019-05-02 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.7 by this push:
 new 1ce2efd  [FLINK-12296][StateBackend] Fix local state directory 
collision with state loss for chained keyed operators (#8323)
1ce2efd is described below

commit 1ce2efd7a38d091fc004a8dba034ece0bcc42385
Author: Congxian Qiu 
AuthorDate: Thu May 2 16:12:31 2019 +0800

[FLINK-12296][StateBackend] Fix local state directory collision with state 
loss for chained keyed operators (#8323)

- Change
Will change the local data path from

`.../local_state_root/allocatio_id/job_id/jobvertext_id_subtask_id/chk_id/rocksdb`
to

`.../local_state_root/allocatio_id/job_id/jobvertext_id_subtask_id/chk_id/backend_id`

When preparing the local directory Flink deletes the local directory for 
each subtask if it already exists,
If more than one stateful operators chained in a single task, they'll share 
the same local directory path,
then the local directory will be deleted unexpectedly, and the we'll get 
data loss.
---
 .../snapshot/RocksIncrementalSnapshotStrategy.java |  16 +-
 .../tasks/OneInputStreamTaskTestHarness.java   |  42 +++-
 .../runtime/tasks/StreamConfigChainer.java |  23 +-
 .../runtime/tasks/StreamMockEnvironment.java   |   8 +-
 .../runtime/tasks/StreamTaskTestHarness.java   |  20 ++
 .../state/StatefulOperatorChainedTaskTest.java | 257 +
 6 files changed, 355 insertions(+), 11 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
index 37f1850..e877cc4 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java
@@ -106,6 +106,9 @@ public class RocksIncrementalSnapshotStrategy extends 
RocksDBSnapshotStrategy
/** The identifier of the last completed checkpoint. */
private long lastCompletedCheckpointId;
 
+   /** The local directory name of the current snapshot strategy. */
+   private final String localDirectoryName;
+
public RocksIncrementalSnapshotStrategy(
@Nonnull RocksDB db,
@Nonnull ResourceGuard rocksDBResourceGuard,
@@ -135,6 +138,7 @@ public class RocksIncrementalSnapshotStrategy extends 
RocksDBSnapshotStrategy
this.backendUID = backendUID;
this.materializedSstFiles = materializedSstFiles;
this.lastCompletedCheckpointId = lastCompletedCheckpointId;
+   this.localDirectoryName = 
backendUID.toString().replaceAll("[\\-]", "");
}
 
@Nonnull
@@ -182,17 +186,17 @@ public class RocksIncrementalSnapshotStrategy extends 
RocksDBSnapshotStrategy
LocalRecoveryDirectoryProvider directoryProvider = 
localRecoveryConfig.getLocalStateDirectoryProvider();
File directory = 
directoryProvider.subtaskSpecificCheckpointDirectory(checkpointId);
 
-   if (directory.exists()) {
-   FileUtils.deleteDirectory(directory);
-   }
-
-   if (!directory.mkdirs()) {
+   if (!directory.exists() && !directory.mkdirs()) {
throw new IOException("Local state base 
directory for checkpoint " + checkpointId +
" already exists: " + directory);
}
 
// introduces an extra directory because RocksDB wants 
a non-existing directory for native checkpoints.
-   File rdbSnapshotDir = new File(directory, "rocks_db");
+   // append localDirectoryName here to solve directory 
collision problem when two stateful operators chained in one task.
+   File rdbSnapshotDir = new File(directory, 
localDirectoryName);
+   if (rdbSnapshotDir.exists()) {
+   FileUtils.deleteDirectory(rdbSnapshotDir);
+   }
Path path = new Path(rdbSnapshotDir.toURI());
// create a "permanent" snapshot directory because 
local recovery is active.
try {
diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runti

[flink] branch release-1.8 updated: [hotfix] Fix compile error from rebase of FLINK-12296]

2019-04-30 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.8 by this push:
 new 4d1605b  [hotfix] Fix compile error from rebase of FLINK-12296]
4d1605b is described below

commit 4d1605bf52ec03b5e01d0bb950f279a3e6da9471
Author: Stefan Richter 
AuthorDate: Tue Apr 30 11:26:34 2019 +0200

[hotfix] Fix compile error from rebase of FLINK-12296]
---
 .../org/apache/flink/test/state/StatefulOperatorChainedTaskTest.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/flink-tests/src/test/java/org/apache/flink/test/state/StatefulOperatorChainedTaskTest.java
 
b/flink-tests/src/test/java/org/apache/flink/test/state/StatefulOperatorChainedTaskTest.java
index 5651929..c3e1ba9 100644
--- 
a/flink-tests/src/test/java/org/apache/flink/test/state/StatefulOperatorChainedTaskTest.java
+++ 
b/flink-tests/src/test/java/org/apache/flink/test/state/StatefulOperatorChainedTaskTest.java
@@ -176,7 +176,7 @@ public class StatefulOperatorChainedTaskTest {
 
testHarness.getTaskStateManager().setWaitForReportLatch(new 
OneShotLatch());
 
-   while (!streamTask.triggerCheckpoint(checkpointMetaData, 
CheckpointOptions.forCheckpointWithDefaultLocation(), false)) {}
+   while (!streamTask.triggerCheckpoint(checkpointMetaData, 
CheckpointOptions.forCheckpointWithDefaultLocation())) {}
 

testHarness.getTaskStateManager().getWaitForReportLatch().await();
long reportedCheckpointId = 
testHarness.getTaskStateManager().getReportedCheckpointId();



[flink] branch release-1.8 updated: [FLINK-12296][StateBackend] Fix local state directory collision with state loss for chained keyed operators

2019-04-30 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.8 by this push:
 new 531d727  [FLINK-12296][StateBackend] Fix local state directory 
collision with state loss for chained keyed operators
531d727 is described below

commit 531d727f9b32c310d8d63b253019b8cc4a23a3eb
Author: klion26 
AuthorDate: Wed Apr 24 04:52:03 2019 +0200

[FLINK-12296][StateBackend] Fix local state directory collision with state 
loss for chained keyed operators

- Change
Will change the local data path from

`.../local_state_root/allocatio_id/job_id/jobvertext_id_subtask_id/chk_id/rocksdb`
to

`.../local_state_root/allocatio_id/job_id/jobvertext_id_subtask_id/chk_id/operator_id`

When preparing the local directory Flink deletes the local directory for 
each subtask if it already exists,
If more than one stateful operators chained in a single task, they'll share 
the same local directory path,
then the local directory will be deleted unexpectedly, and the we'll get 
data loss.

This closes #8263.

(cherry picked from commit ee60846dc588b1a832a497ff9522d7a3a282c350)
---
 .../CheckpointStreamWithResultProviderTest.java|   3 +
 .../state/StateSnapshotCompressionTest.java|   2 +-
 .../ttl/mock/MockKeyedStateBackendBuilder.java |   1 +
 .../runtime/state/ttl/mock/MockStateBackend.java   |   2 +-
 .../state/RocksDBKeyedStateBackendBuilder.java |   1 +
 .../snapshot/RocksIncrementalSnapshotStrategy.java |  17 +-
 .../tasks/OneInputStreamTaskTestHarness.java   |  50 +++-
 .../runtime/tasks/StreamConfigChainer.java |  23 +-
 .../runtime/tasks/StreamMockEnvironment.java   |   8 +-
 .../runtime/tasks/StreamTaskTestHarness.java   |  21 +-
 .../state/StatefulOperatorChainedTaskTest.java | 260 +
 11 files changed, 369 insertions(+), 19 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
index 2af25d9..57653e2 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
@@ -35,6 +35,9 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 
+/**
+ * Test for CheckpointStreamWithResultProvider.
+ */
 public class CheckpointStreamWithResultProviderTest extends TestLogger {
 
private static TemporaryFolder temporaryFolder;
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
index a10be26..de687ff 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.state;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
@@ -34,6 +33,7 @@ import 
org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory;
 import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 import org.apache.flink.util.TestLogger;
 
+import org.apache.commons.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
index efef923..2196dc9 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
@@ -30,6 +30,7 @@ import 
org.apache.flink.runtime.state.StreamCompressionDecorator;
 import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 
 import javax.annotation.Nonnull;
+
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
index bdf07bf..f50f1b6 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
@@ -35,8 +35,8

[flink] branch master updated: [FLINK-12296][StateBackend] Fix local state directory collision with state loss for chained keyed operators

2019-04-30 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 ee60846  [FLINK-12296][StateBackend] Fix local state directory 
collision with state loss for chained keyed operators
ee60846 is described below

commit ee60846dc588b1a832a497ff9522d7a3a282c350
Author: klion26 
AuthorDate: Wed Apr 24 10:52:03 2019 +0800

[FLINK-12296][StateBackend] Fix local state directory collision with state 
loss for chained keyed operators

- Change
Will change the local data path from

`.../local_state_root/allocatio_id/job_id/jobvertext_id_subtask_id/chk_id/rocksdb`
to

`.../local_state_root/allocatio_id/job_id/jobvertext_id_subtask_id/chk_id/operator_id`

When preparing the local directory Flink deletes the local directory for 
each subtask if it already exists,
If more than one stateful operators chained in a single task, they'll share 
the same local directory path,
then the local directory will be deleted unexpectedly, and the we'll get 
data loss.

This closes #8263.
---
 .../CheckpointStreamWithResultProviderTest.java|   3 +
 .../state/StateSnapshotCompressionTest.java|   2 +-
 .../ttl/mock/MockKeyedStateBackendBuilder.java |   1 +
 .../runtime/state/ttl/mock/MockStateBackend.java   |   2 +-
 .../state/RocksDBKeyedStateBackendBuilder.java |   1 +
 .../snapshot/RocksIncrementalSnapshotStrategy.java |  17 +-
 .../tasks/OneInputStreamTaskTestHarness.java   |  50 +++-
 .../runtime/tasks/StreamConfigChainer.java |  23 +-
 .../runtime/tasks/StreamMockEnvironment.java   |   8 +-
 .../runtime/tasks/StreamTaskTestHarness.java   |  21 +-
 .../state/StatefulOperatorChainedTaskTest.java | 260 +
 11 files changed, 369 insertions(+), 19 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
index 2af25d9..57653e2 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java
@@ -35,6 +35,9 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 
+/**
+ * Test for CheckpointStreamWithResultProvider.
+ */
 public class CheckpointStreamWithResultProviderTest extends TestLogger {
 
private static TemporaryFolder temporaryFolder;
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
index a10be26..de687ff 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.state;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
@@ -34,6 +33,7 @@ import 
org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory;
 import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 import org.apache.flink.util.TestLogger;
 
+import org.apache.commons.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
index 3ffe183..8ec9b4d 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
@@ -31,6 +31,7 @@ import 
org.apache.flink.runtime.state.heap.InternalKeyContextImpl;
 import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 
 import javax.annotation.Nonnull;
+
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
index bdf07bf..f50f1b6 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockStateBackend.java
@@ -35,8 +35,8 @@ import 
org.apache.flink.runtime.state.CheckpointStorageLocationReference;
 import

[flink] branch release-1.8 updated: [FLINK-12350] [State Backends] RocksDBStateBackendTest doesn't cover the incremental checkpoint code path

2019-04-29 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.8 by this push:
 new 8020787  [FLINK-12350] [State Backends] RocksDBStateBackendTest 
doesn't cover the incremental checkpoint code path
8020787 is described below

commit 80207878251843a2263dcbf25eb21e0d6538048c
Author: Yu Li 
AuthorDate: Mon Apr 29 10:21:24 2019 +0200

[FLINK-12350] [State Backends] RocksDBStateBackendTest doesn't cover the 
incremental checkpoint code path

This closes #8297.

(cherry picked from commit 9aeb4e5cb38079cdfbcc25f8c3966b368287825d)
---
 .../flink/contrib/streaming/state/RocksDBStateBackendTest.java| 8 ++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
index 819c864..ceb781c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
@@ -217,7 +217,9 @@ public class RocksDBStateBackendTest extends 
StateBackendTestBase stubState1 =
new ValueStateDescriptor<>("StubState-1", 
StringSerializer.INSTANCE);
test.createInternalState(StringSerializer.INSTANCE, 
stubState1);



[flink] branch master updated: [FLINK-12350] [State Backends] RocksDBStateBackendTest doesn't cover the incremental checkpoint code path

2019-04-29 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 9aeb4e5  [FLINK-12350] [State Backends] RocksDBStateBackendTest 
doesn't cover the incremental checkpoint code path
9aeb4e5 is described below

commit 9aeb4e5cb38079cdfbcc25f8c3966b368287825d
Author: Yu Li 
AuthorDate: Mon Apr 29 16:21:24 2019 +0800

[FLINK-12350] [State Backends] RocksDBStateBackendTest doesn't cover the 
incremental checkpoint code path

This closes #8297.
---
 .../flink/contrib/streaming/state/RocksDBStateBackendTest.java| 8 ++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
index 819c864..ceb781c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
@@ -217,7 +217,9 @@ public class RocksDBStateBackendTest extends 
StateBackendTestBase stubState1 =
new ValueStateDescriptor<>("StubState-1", 
StringSerializer.INSTANCE);
test.createInternalState(StringSerializer.INSTANCE, 
stubState1);



[flink] branch master updated: [FLINK-11167] Optimize RocksDBListState#put by removing the clear before every put operation

2019-04-29 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 03faeb9  [FLINK-11167] Optimize RocksDBListState#put by removing the 
clear before every put operation
03faeb9 is described below

commit 03faeb9bfbc388666cf62dc2e972ee21c1370031
Author: Congxian Qiu 
AuthorDate: Mon Apr 29 16:15:49 2019 +0800

[FLINK-11167] Optimize RocksDBListState#put by removing the clear before 
every put operation

This closes #7421.

Differential Revision: https://aone.alibaba-inc.com/code/D816268
---
 .../org/apache/flink/contrib/streaming/state/RocksDBListState.java| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
index c18adb1..03a68b2 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
@@ -209,8 +209,6 @@ class RocksDBListState
public void updateInternal(List values) {
Preconditions.checkNotNull(values, "List of values to add 
cannot be null.");
 
-   clear();
-
if (!values.isEmpty()) {
try {
backend.db.put(
@@ -221,6 +219,8 @@ class RocksDBListState
} catch (IOException | RocksDBException e) {
throw new FlinkRuntimeException("Error while 
updating data to RocksDB", e);
}
+   } else {
+   clear();
}
}
 



[flink] branch master updated: [FLINK-10724] Refactor failure handling in check point coordinator

2019-04-29 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 c4b0e8f  [FLINK-10724] Refactor failure handling in check point 
coordinator
c4b0e8f is described below

commit c4b0e8f68c5c4bb2ba60b358df92ee5db1d857df
Author: vinoyang 
AuthorDate: Mon Apr 29 15:56:04 2019 +0800

[FLINK-10724] Refactor failure handling in check point coordinator

This closes #7571.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  | 89 +++--
 .../runtime/checkpoint/CheckpointException.java| 30 +--
 ...ineReason.java => CheckpointFailureReason.java} | 28 ++-
 .../checkpoint/CheckpointTriggerException.java | 42 --
 .../checkpoint/CheckpointTriggerResult.java| 92 --
 .../runtime/checkpoint/PendingCheckpoint.java  | 51 +++-
 .../executiongraph/failover/FailoverRegion.java|  5 +-
 .../apache/flink/runtime/jobmaster/JobMaster.java  | 12 +--
 .../checkpoint/CheckpointCoordinatorTest.java  | 38 +
 .../runtime/checkpoint/PendingCheckpointTest.java  | 30 +++
 .../jobmaster/JobMasterTriggerSavepointITCase.java |  4 +-
 .../test/streaming/runtime/TimestampITCase.java|  4 +-
 12 files changed, 160 insertions(+), 265 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index e6cc5d3..c7f59a7 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -338,7 +338,7 @@ public class CheckpointCoordinator {
 
// clear and discard all pending checkpoints
for (PendingCheckpoint pending : 
pendingCheckpoints.values()) {
-   pending.abortError(new 
Exception("Checkpoint Coordinator is shutting down"));
+   
pending.abort(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN);
}
pendingCheckpoints.clear();
 
@@ -405,17 +405,17 @@ public class CheckpointCoordinator {
 
checkNotNull(checkpointProperties);
 
-   CheckpointTriggerResult triggerResult = triggerCheckpoint(
-   timestamp,
-   checkpointProperties,
-   targetLocation,
-   false,
-   advanceToEndOfEventTime);
-
-   if (triggerResult.isSuccess()) {
-   return 
triggerResult.getPendingCheckpoint().getCompletionFuture();
-   } else {
-   Throwable cause = new 
CheckpointTriggerException("Failed to trigger savepoint.", 
triggerResult.getFailureReason());
+   try {
+   PendingCheckpoint pendingCheckpoint = triggerCheckpoint(
+   timestamp,
+   checkpointProperties,
+   targetLocation,
+   false,
+   advanceToEndOfEventTime);
+
+   return pendingCheckpoint.getCompletionFuture();
+   } catch (CheckpointException e) {
+   Throwable cause = new CheckpointException("Failed to 
trigger savepoint.", e.getCheckpointFailureReason());
return FutureUtils.completedExceptionally(cause);
}
}
@@ -431,16 +431,21 @@ public class CheckpointCoordinator {
 * @return true if triggering the checkpoint succeeded.
 */
public boolean triggerCheckpoint(long timestamp, boolean isPeriodic) {
-   return triggerCheckpoint(timestamp, checkpointProperties, null, 
isPeriodic, false).isSuccess();
+   try {
+   triggerCheckpoint(timestamp, checkpointProperties, 
null, isPeriodic, false);
+   return true;
+   } catch (CheckpointException e) {
+   return false;
+   }
}
 
@VisibleForTesting
-   public CheckpointTriggerResult triggerCheckpoint(
+   public PendingCheckpoint triggerCheckpoint(
long timestamp,
CheckpointProperties props,
@Nullable String externalSavepointLocation,
boolean isPeriodic,
-   boolean advanceToEndOfTime) {
+   boolean advanceToEndO

[flink] branch master updated: [FLINK-12212][docs] Clarify that operator state is checkpointed asynchronously

2019-04-17 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 046d752  [FLINK-12212][docs] Clarify that operator state is 
checkpointed asynchronously
046d752 is described below

commit 046d752c6e41872fbb83163b8c5862c7b8855f5d
Author: Konstantin Knauf 
AuthorDate: Wed Apr 17 16:40:04 2019 +0200

[FLINK-12212][docs] Clarify that operator state is checkpointed 
asynchronously

This closes #8185.
---
 docs/ops/state/large_state_tuning.md | 20 
 1 file changed, 8 insertions(+), 12 deletions(-)

diff --git a/docs/ops/state/large_state_tuning.md 
b/docs/ops/state/large_state_tuning.md
index effea99..9dae3b6 100644
--- a/docs/ops/state/large_state_tuning.md
+++ b/docs/ops/state/large_state_tuning.md
@@ -100,22 +100,18 @@ number of network buffers used per outgoing/incoming 
channel is limited and thus
 may be configured without affecting checkpoint times
 (see [network buffer 
configuration](../config.html#configuring-the-network-buffers)).
 
-## Make state checkpointing Asynchronous where possible
+## Asynchronous Checkpointing
 
 When state is *asynchronously* snapshotted, the checkpoints scale better than 
when the state is *synchronously* snapshotted.
-Especially in more complex streaming applications with multiple joins, 
Co-functions, or windows, this may have a profound
+Especially in more complex streaming applications with multiple joins, 
co-functions, or windows, this may have a profound
 impact.
 
-To get state to be snapshotted asynchronously, applications have to do two 
things:
+For state to be snapshotted asynchronsously, you need to use a state backend 
which supports asynchronous snapshotting.
+Starting from Flink 1.3, both RocksDB-based as well as heap-based state 
backends (`filesystem`) support asynchronous
+snapshotting and use it by default. This applies to to both managed operator 
state as well as managed keyed state (incl. timers state).
 
-  1. Use state that is [managed by Flink](../../dev/stream/state/state.html): 
Managed state means that Flink provides the data
- structure in which the state is stored. Currently, this is true for 
*keyed state*, which is abstracted behind the
- interfaces like `ValueState`, `ListState`, `ReducingState`, ...
-
-  2. Use a state backend that supports asynchronous snapshots. In Flink 1.2, 
only the RocksDB state backend uses
- fully asynchronous snapshots. Starting from Flink 1.3, heap-based state 
backends also support asynchronous snapshots.
-
-The above two points imply that large state should generally be kept as keyed 
state, not as operator state.
+Note *The combination RocksDB state 
backend with heap-based timers currently does NOT support asynchronous 
snapshots for the timers state.
+Other state like keyed state is still snapshotted asynchronously. Please note 
that this is not a regression from previous versions and will be resolved with 
`FLINK-10026`.*
 
 ## Tuning RocksDB
 
@@ -150,7 +146,7 @@ timers, while storing timers inside RocksDB offers higher 
scalability as the num
 When using RockDB as state backend, the type of timer storage can be selected 
through Flink's configuration via option key 
`state.backend.rocksdb.timer-service.factory`.
 Possible choices are `heap` (to store timers on the heap, default) and 
`rocksdb` (to store timers in RocksDB).
 
-Note *The combination RocksDB state 
backend / with incremental checkpoint / with heap-based timers currently does 
NOT support asynchronous snapshots for the timers state.
+Note *The combination RocksDB state 
backend with heap-based timers currently does NOT support asynchronous 
snapshots for the timers state.
 Other state like keyed state is still snapshotted asynchronously. Please note 
that this is not a regression from previous versions and will be resolved with 
`FLINK-10026`.*
 
 **Predefined Options**



[flink] branch master updated: [FLINK-10712] Support state restore for RestartPipelinedRegionStrategy

2019-04-17 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 c3a9293  [FLINK-10712] Support state restore for 
RestartPipelinedRegionStrategy
c3a9293 is described below

commit c3a929346f9993d5177978053fc0bedd482faf7c
Author: Yun Tang 
AuthorDate: Wed Apr 17 19:48:40 2019 +0800

[FLINK-10712] Support state restore for RestartPipelinedRegionStrategy

This closes #7813.
---
 .../fs/RollingSinkFaultToleranceITCase.java|  12 +-
 .../BucketingSinkFaultToleranceITCase.java |  12 +-
 .../runtime/checkpoint/CheckpointCoordinator.java  |  15 +-
 .../flink/runtime/executiongraph/Execution.java|   2 -
 .../executiongraph/failover/FailoverRegion.java|  28 +-
 .../failover/RestartPipelinedRegionStrategy.java   |  18 +-
 ...ncurrentFailoverStrategyExecutionGraphTest.java |   4 +-
 .../runtime/executiongraph/FailoverRegionTest.java | 219 +++-
 .../ContinuousFileProcessingCheckpointITCase.java  |  23 +-
 .../test/checkpointing/RegionFailoverITCase.java   | 394 +
 .../StreamFaultToleranceTestBase.java  |  87 -
 11 files changed, 748 insertions(+), 66 deletions(-)

diff --git 
a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
 
b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
index 886055d..daa8884 100644
--- 
a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
+++ 
b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
@@ -32,9 +32,9 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 
@@ -77,8 +77,8 @@ public class RollingSinkFaultToleranceITCase extends 
StreamFaultToleranceTestBas
 
private static String outPath;
 
-   @BeforeClass
-   public static void createHDFS() throws IOException {
+   @Before
+   public void createHDFS() throws IOException {
Configuration conf = new Configuration();
 
File dataDir = tempFolder.newFolder();
@@ -94,8 +94,8 @@ public class RollingSinkFaultToleranceITCase extends 
StreamFaultToleranceTestBas
+ "/string-non-rolling-out";
}
 
-   @AfterClass
-   public static void destroyHDFS() {
+   @After
+   public void destroyHDFS() {
if (hdfsCluster != null) {
hdfsCluster.shutdown();
}
diff --git 
a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
 
b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
index dcb77bf..71b210d 100644
--- 
a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
+++ 
b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java
@@ -32,9 +32,9 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
+import org.junit.After;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 
@@ -77,8 +77,8 @@ public class BucketingSinkFaultToleranceITCase extends 
StreamFaultToleranceTestB
 
private static String outPath;
 
-   @BeforeClass
-   public static void createHDFS() throws IOException {
+   @Before
+   public void createHDFS() throws IOException {
Configuration conf = new Configuration();
 
File dataDir = tempFolder.newFolder();
@@ -94,8 +94,8 @@ public class BucketingSinkFaultToleranceITCase extends 
StreamFaultToleranceTestB
+ "/string-non-rolling-out";
}
 
-   @AfterClass
-   public static void destroyHDFS() {
+   @After
+   public void destroyHDFS() {
if (hdfsCluster != null) {

[flink] branch master updated: [hotfix] Delete empty TaskManager.scala file

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 851ac7d  [hotfix] Delete empty TaskManager.scala file
851ac7d is described below

commit 851ac7d1c0e9661e9b75faaef994bb270a9f112f
Author: Stefan Richter 
AuthorDate: Mon Apr 15 18:36:39 2019 +0200

[hotfix] Delete empty TaskManager.scala file
---
 .../src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala | 0
 1 file changed, 0 insertions(+), 0 deletions(-)

diff --git 
a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 
b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
deleted file mode 100644
index e69de29..000



[flink] branch master updated: [hotfix] Remove unused method ExecutionGraph#restoreLatestCheckpointedState

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 32f0ad7  [hotfix] Remove unused method 
ExecutionGraph#restoreLatestCheckpointedState
32f0ad7 is described below

commit 32f0ad79be4b18774e1506db56ae59658ec211e1
Author: Stefan Richter 
AuthorDate: Mon Apr 15 17:28:02 2019 +0200

[hotfix] Remove unused method ExecutionGraph#restoreLatestCheckpointedState
---
 .../flink/runtime/executiongraph/ExecutionGraph.java | 20 
 1 file changed, 20 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 5e44e94..56e31a3 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -1291,26 +1291,6 @@ public class ExecutionGraph implements 
AccessExecutionGraph {
}
 
/**
-* Restores the latest checkpointed state.
-*
-* The recovery of checkpoints might block. Make sure that calls to 
this method don't
-* block the job manager actor and run asynchronously.
-*
-* @param errorIfNoCheckpoint Fail if there is no checkpoint available
-* @param allowNonRestoredState Allow to skip checkpoint state that 
cannot be mapped
-* to the ExecutionGraph vertices (if the checkpoint contains state for 
a
-* job vertex that is not part of this ExecutionGraph).
-*/
-   public void restoreLatestCheckpointedState(boolean errorIfNoCheckpoint, 
boolean allowNonRestoredState) throws Exception {
-   assertRunningInJobMasterMainThread();
-   synchronized (progressLock) {
-   if (checkpointCoordinator != null) {
-   
checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), 
errorIfNoCheckpoint, allowNonRestoredState);
-   }
-   }
-   }
-
-   /**
 * Returns the serializable {@link ArchivedExecutionConfig}.
 *
 * @return ArchivedExecutionConfig which may be null in case of errors



[flink] 03/06: [FLINK-11952][2/4] Introduce basic plugin mechanism for Flink

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 0c4953c4c5ff583d88f1686cd96fd7e7be9d8f11
Author: Stefan Richter 
AuthorDate: Tue Apr 9 10:37:03 2019 +0200

[FLINK-11952][2/4] Introduce basic plugin mechanism for Flink

The mechanism uses child-first classloading and creates classloaders from 
jars that are discovered
from a directory hierarchy.
---
 .../apache/flink/core/fs/FileSystemFactory.java|  12 +-
 .../flink/core/fs/UnsupportedSchemeFactory.java|   6 -
 .../core/fs/local/LocalFileSystemFactory.java  |   6 -
 .../core/plugin/DirectoryBasedPluginFinder.java| 103 
 .../Plugin.java}   |  33 ++---
 .../apache/flink/core/plugin/PluginDescriptor.java |  67 ++
 .../org/apache/flink/core/plugin/PluginFinder.java |  37 ++
 .../org/apache/flink/core/plugin/PluginLoader.java |  94 ++
 .../apache/flink/core/plugin/PluginManager.java|  77 
 .../org/apache/flink/core/plugin/PluginUtils.java  |  54 
 .../TemporaryClassLoaderContext.java}  |  33 ++---
 .../plugin/DirectoryBasedPluginFinderTest.java | 137 +
 .../plugin/TemporaryClassLoaderContextTest.java|  46 +++
 .../testutils/EntropyInjectingTestFileSystem.java  |   5 -
 .../org/apache/flink/testutils/TestFileSystem.java |  10 +-
 .../flink/runtime/fs/maprfs/MapRFsFactory.java |   6 -
 flink-tests/pom.xml|  31 +
 .../src/test/assembly/test-plugin-a-assembly.xml   |  43 +++
 .../src/test/assembly/test-plugin-b-assembly.xml   |  43 +++
 .../org/apache/flink/test/plugin/OtherTestSpi.java |  28 +
 .../apache/flink/test/plugin/PluginLoaderTest.java |  71 +++
 .../flink/test/plugin/PluginManagerTest.java   | 105 
 .../apache/flink/test/plugin/PluginTestBase.java   |  54 
 .../java/org/apache/flink/test/plugin/TestSpi.java |  28 +
 .../test/plugin/jar/plugina/DynamicClassA.java |  31 ++---
 .../test/plugin/jar/plugina/TestServiceA.java  |  34 +++--
 .../test/plugin/jar/pluginb/OtherTestServiceB.java |  30 ++---
 .../test/plugin/jar/pluginb/TestServiceB.java  |  30 ++---
 .../plugin-a/org.apache.flink.test.plugin.TestSpi  |  16 +++
 .../org.apache.flink.test.plugin.OtherTestSpi  |  16 +++
 .../plugin-b/org.apache.flink.test.plugin.TestSpi  |  16 +++
 31 files changed, 1130 insertions(+), 172 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java 
b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java
index 8a35471..eecf6f1 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java
@@ -20,6 +20,7 @@ package org.apache.flink.core.fs;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.plugin.Plugin;
 
 import java.io.IOException;
 import java.net.URI;
@@ -31,7 +32,7 @@ import java.net.URI;
  * creating file systems via {@link #create(URI)}.
  */
 @PublicEvolving
-public interface FileSystemFactory {
+public interface FileSystemFactory extends Plugin {
 
/**
 * Gets the scheme of the file system created by this factory.
@@ -39,15 +40,6 @@ public interface FileSystemFactory {
String getScheme();
 
/**
-* Applies the given configuration to this factory. All future file 
system
-* instantiations via {@link #create(URI)} should take the 
configuration into
-* account.
-*
-* @param config The configuration to apply.
-*/
-   void configure(Configuration config);
-
-   /**
 * Creates a new file system for the given file system URI.
 * The URI describes the type of file system (via its scheme) and 
optionally the
 * authority (for example the host) of the file system.
diff --git 
a/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java
 
b/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java
index c2cb2d5..e873e63 100644
--- 
a/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java
+++ 
b/flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedSchemeFactory.java
@@ -19,7 +19,6 @@
 package org.apache.flink.core.fs;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.Configuration;
 
 import javax.annotation.Nullable;
 
@@ -54,11 +53,6 @@ class UnsupportedSchemeFactory implements FileSystemFactory {
}
 
@Override
-   public void configure(Configuration config) {
-   // nothing to do here
-   }
-
-   @Override
public FileSystem create(URI fsUri) throws

[flink] 01/06: [hotfix] Remove unused exception from FileSystem#initialize

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 5aee179cb6677aad1e1b6f1808873167c4a05789
Author: Stefan Richter 
AuthorDate: Fri Mar 15 15:02:51 2019 +0100

[hotfix] Remove unused exception from FileSystem#initialize
---
 .../src/main/java/org/apache/flink/client/cli/CliFrontend.java | 10 ++
 .../src/main/java/org/apache/flink/core/fs/FileSystem.java |  2 +-
 .../apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java |  7 +--
 .../apache/flink/runtime/webmonitor/history/HistoryServer.java |  6 +-
 .../org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java | 10 ++
 .../apache/flink/runtime/taskexecutor/TaskManagerRunner.java   |  8 +---
 .../apache/flink/table/client/gateway/local/LocalExecutor.java |  7 +--
 .../org/apache/flink/yarn/AbstractYarnClusterDescriptor.java   |  7 +--
 8 files changed, 10 insertions(+), 47 deletions(-)

diff --git 
a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java 
b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 6b20e78..c7cbe9b 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -67,7 +67,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.FileNotFoundException;
-import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.UndeclaredThrowableException;
@@ -124,16 +123,11 @@ public class CliFrontend {
 
public CliFrontend(
Configuration configuration,
-   List> customCommandLines) throws 
Exception {
+   List> customCommandLines) {
this.configuration = Preconditions.checkNotNull(configuration);
this.customCommandLines = 
Preconditions.checkNotNull(customCommandLines);
 
-   try {
-   FileSystem.initialize(this.configuration);
-   } catch (IOException e) {
-   throw new Exception("Error while setting the default " +
-   "filesystem scheme from configuration.", e);
-   }
+   FileSystem.initialize(this.configuration);
 
this.customCommandLineOptions = new Options();
 
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java 
b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
index d451109..e7a3765 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
@@ -251,7 +251,7 @@ public abstract class FileSystem {
 *
 * @param config the configuration from where to fetch the parameter.
 */
-   public static void initialize(Configuration config) throws IOException, 
IllegalConfigurationException {
+   public static void initialize(Configuration config) throws 
IllegalConfigurationException {
LOCK.lock();
try {
// make sure file systems are re-instantiated after 
re-configuration
diff --git 
a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java
 
b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java
index cc1289f..98d2157 100644
--- 
a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java
+++ 
b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java
@@ -40,7 +40,6 @@ import org.apache.commons.cli.PosixParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.util.Map;
 
@@ -86,11 +85,7 @@ public class MesosTaskExecutorRunner {
final Map envs = System.getenv();
 
// configure the filesystems
-   try {
-   FileSystem.initialize(configuration);
-   } catch (IOException e) {
-   throw new IOException("Error while configuring the 
filesystems.", e);
-   }
+   FileSystem.initialize(configuration);
 
// tell akka to die in case of an error
configuration.setBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR, 
true);
diff --git 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
index a93fe93..8d5183b 100644
--- 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java
+++ 
b/flink-

[flink] 04/06: [FLINK-11952][3/4] Integrate plugin mechanism with FileSystem

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit c7b141b4b8767c9b7c8c72ef21055fd65908e848
Author: Stefan Richter 
AuthorDate: Fri Mar 22 14:57:09 2019 +0100

[FLINK-11952][3/4] Integrate plugin mechanism with FileSystem
---
 .../java/org/apache/flink/core/fs/FileSystem.java  | 109 ++---
 1 file changed, 76 insertions(+), 33 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java 
b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
index e7a3765..d159e70 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
@@ -31,6 +31,7 @@ import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.core.fs.local.LocalFileSystem;
 import org.apache.flink.core.fs.local.LocalFileSystemFactory;
+import org.apache.flink.core.plugin.PluginManager;
 import org.apache.flink.util.ExceptionUtils;
 
 import org.slf4j.Logger;
@@ -44,12 +45,14 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.ServiceLoader;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Supplier;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -217,11 +220,8 @@ public abstract class FileSystem {
/** Cache for file systems, by scheme + authority. */
private static final HashMap CACHE = new HashMap<>();
 
-   /** All available file system factories. */
-   private static final List RAW_FACTORIES = 
loadFileSystems();
-
/** Mapping of file system schemes to the corresponding factories,
-* populated in {@link FileSystem#initialize(Configuration)}. */
+* populated in {@link FileSystem#initialize(Configuration, 
PluginManager)}. */
private static final HashMap FS_FACTORIES = 
new HashMap<>();
 
/** The default factory that is used when no scheme matches. */
@@ -249,17 +249,57 @@ public abstract class FileSystem {
 * A file path of {@code '/user/USERNAME/in.txt'} is interpreted as
 * {@code 'hdfs://localhost:9000/user/USERNAME/in.txt'}.
 *
+* @deprecated use {@link #initialize(Configuration, PluginManager)} 
instead.
+*
 * @param config the configuration from where to fetch the parameter.
 */
+   @Deprecated
public static void initialize(Configuration config) throws 
IllegalConfigurationException {
+   initializeWithoutPlugins(config);
+   }
+
+   private static void initializeWithoutPlugins(Configuration config) 
throws IllegalConfigurationException {
+   initialize(config, null);
+   }
+
+   /**
+* Initializes the shared file system settings.
+*
+* The given configuration is passed to each file system factory to 
initialize the respective
+* file systems. Because the configuration of file systems may be 
different subsequent to the call
+* of this method, this method clears the file system instance cache.
+*
+* This method also reads the default file system URI from the 
configuration key
+* {@link CoreOptions#DEFAULT_FILESYSTEM_SCHEME}. All calls to {@link 
FileSystem#get(URI)} where
+* the URI has no scheme will be interpreted as relative to that URI.
+* As an example, assume the default file system URI is set to {@code 
'hdfs://localhost:9000/'}.
+* A file path of {@code '/user/USERNAME/in.txt'} is interpreted as
+* {@code 'hdfs://localhost:9000/user/USERNAME/in.txt'}.
+*
+* @param config the configuration from where to fetch the parameter.
+* @param pluginManager optional plugin manager that is used to 
initialized filesystems provided as plugins.
+*/
+   public static void initialize(
+   Configuration config,
+   PluginManager pluginManager) throws 
IllegalConfigurationException {
+
LOCK.lock();
try {
// make sure file systems are re-instantiated after 
re-configuration
CACHE.clear();
FS_FACTORIES.clear();
 
+   Collection>> 
factorySuppliers = new ArrayList<>(2);
+   factorySuppliers.add(() -> 
ServiceLoader.load(FileSystemFactory.class).iterator());
+
+   if (pluginManager != null) {
+   factorySuppliers.add(() -> 
pluginManage

[flink] branch master updated (2c503c2 -> 703dc8d)

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from 2c503c2  [FLINK-11884][table] Ported sort & limit validation on top of 
Expressions
 new 5aee179  [hotfix] Remove unused exception from FileSystem#initialize
 new 1a10fbe  [FLINK-11952][1/3] Make ChildFirstClassLoader a top-level 
class in flink-core
 new 0c4953c  [FLINK-11952][2/4] Introduce basic plugin mechanism for Flink
 new c7b141b  [FLINK-11952][3/4] Integrate plugin mechanism with FileSystem
 new c6307b5  [FLINK-11952][4/4] Integrate plugin mechanism with FileSystem 
initialization in process entry points
 new 703dc8d  [hotfix] Use TemporaryClassLoaderContext in other appropriate 
places in the codebase

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:
 .../org/apache/flink/client/cli/CliFrontend.java   |  13 +-
 .../java/org/apache/flink/core/fs/FileSystem.java  | 111 -
 .../apache/flink/core/fs/FileSystemFactory.java|  12 +-
 .../flink/core/fs/UnsupportedSchemeFactory.java|   6 -
 .../core/fs/local/LocalFileSystemFactory.java  |   6 -
 .../core/plugin/DirectoryBasedPluginFinder.java| 103 
 .../Plugin.java}   |  33 ++---
 .../apache/flink/core/plugin/PluginDescriptor.java |  67 ++
 .../org/apache/flink/core/plugin/PluginFinder.java |  37 ++
 .../org/apache/flink/core/plugin/PluginLoader.java |  94 ++
 .../apache/flink/core/plugin/PluginManager.java|  77 
 .../org/apache/flink/core/plugin/PluginUtils.java  |  54 
 .../TemporaryClassLoaderContext.java}  |  33 ++---
 .../apache/flink/util/ChildFirstClassLoader.java   | 123 ++
 .../java/org/apache/flink/util/LambdaUtil.java |  19 +--
 .../plugin/DirectoryBasedPluginFinderTest.java | 137 +
 .../plugin/TemporaryClassLoaderContextTest.java|  46 +++
 .../testutils/EntropyInjectingTestFileSystem.java  |   5 -
 .../org/apache/flink/testutils/TestFileSystem.java |  10 +-
 .../flink/runtime/fs/maprfs/MapRFsFactory.java |   6 -
 .../mesos/entrypoint/MesosTaskExecutorRunner.java  |  10 +-
 .../runtime/webmonitor/history/HistoryServer.java  |   9 +-
 .../runtime/entrypoint/ClusterEntrypoint.java  |  14 +--
 .../librarycache/FlinkUserCodeClassLoaders.java| 103 +---
 .../runtime/taskexecutor/TaskManagerRunner.java|  11 +-
 .../flink/runtime/taskmanager/TaskManager.scala|   0
 .../client/gateway/local/ExecutionContext.java |   7 +-
 .../table/client/gateway/local/LocalExecutor.java  |  10 +-
 flink-tests/pom.xml|  31 +
 .../src/test/assembly/test-plugin-a-assembly.xml   |  43 +++
 .../src/test/assembly/test-plugin-b-assembly.xml   |  43 +++
 .../org/apache/flink/test/plugin/OtherTestSpi.java |  28 +
 .../apache/flink/test/plugin/PluginLoaderTest.java |  71 +++
 .../flink/test/plugin/PluginManagerTest.java   | 105 
 .../apache/flink/test/plugin/PluginTestBase.java   |  54 
 .../java/org/apache/flink/test/plugin/TestSpi.java |  28 +
 .../test/plugin/jar/plugina/DynamicClassA.java |  31 ++---
 .../test/plugin/jar/plugina/TestServiceA.java  |  34 +++--
 .../test/plugin/jar/pluginb/OtherTestServiceB.java |  30 ++---
 .../test/plugin/jar/pluginb/TestServiceB.java  |  30 ++---
 .../plugin-a/org.apache.flink.test.plugin.TestSpi  |  16 +++
 .../org.apache.flink.test.plugin.OtherTestSpi  |  16 +++
 .../plugin-b/org.apache.flink.test.plugin.TestSpi  |  16 +++
 .../flink/yarn/AbstractYarnClusterDescriptor.java  |  10 +-
 .../apache/flink/yarn/YarnTaskExecutorRunner.java  |   6 +-
 45 files changed, 1373 insertions(+), 375 deletions(-)
 create mode 100644 
flink-core/src/main/java/org/apache/flink/core/plugin/DirectoryBasedPluginFinder.java
 copy 
flink-core/src/main/java/org/apache/flink/core/{fs/local/LocalFileSystemFactory.java
 => plugin/Plugin.java} (57%)
 create mode 100644 
flink-core/src/main/java/org/apache/flink/core/plugin/PluginDescriptor.java
 create mode 100644 
flink-core/src/main/java/org/apache/flink/core/plugin/PluginFinder.java
 create mode 100644 
flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java
 create mode 100644 
flink-core/src/main/java/org/apache/flink/core/plugin/PluginManager.java
 create mode 100644 
flink-core/src/main/java/org/apache/flink/core/plugin/PluginUtils.java
 copy 
flink-core/src/main/java/org/apache/flink/core/{fs/local/LocalFileSystemFactory.java
 => plugin/TemporaryClassLoaderContext.java} (51%)
 create mode 100644 
flink-core/src/m

[flink] 06/06: [hotfix] Use TemporaryClassLoaderContext in other appropriate places in the codebase

2019-04-15 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 703dc8dab1ee35b2532b472eec40fdc7dd124cc3
Author: Stefan Richter 
AuthorDate: Mon Apr 8 17:38:26 2019 +0200

[hotfix] Use TemporaryClassLoaderContext in other appropriate places in the 
codebase
---
 .../main/java/org/apache/flink/util/LambdaUtil.java   | 19 +++
 .../table/client/gateway/local/ExecutionContext.java  |  7 ++-
 2 files changed, 5 insertions(+), 21 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java 
b/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
index fdb6943..163b583 100644
--- a/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.util;
 
+import org.apache.flink.core.plugin.TemporaryClassLoaderContext;
 import org.apache.flink.util.function.SupplierWithException;
 import org.apache.flink.util.function.ThrowingConsumer;
 import org.apache.flink.util.function.ThrowingRunnable;
@@ -76,16 +77,9 @@ public final class LambdaUtil {
final ClassLoader cl,
final ThrowingRunnable r) throws E {
 
-   final Thread currentThread = Thread.currentThread();
-   final ClassLoader oldClassLoader = 
currentThread.getContextClassLoader();
-
-   try {
-   currentThread.setContextClassLoader(cl);
+   try (TemporaryClassLoaderContext tmpCl = new 
TemporaryClassLoaderContext(cl)) {
r.run();
}
-   finally {
-   currentThread.setContextClassLoader(oldClassLoader);
-   }
}
 
/**
@@ -99,15 +93,8 @@ public final class LambdaUtil {
final ClassLoader cl,
final SupplierWithException s) throws E {
 
-   final Thread currentThread = Thread.currentThread();
-   final ClassLoader oldClassLoader = 
currentThread.getContextClassLoader();
-
-   try {
-   currentThread.setContextClassLoader(cl);
+   try (TemporaryClassLoaderContext tmpCl = new 
TemporaryClassLoaderContext(cl)) {
return s.get();
}
-   finally {
-   currentThread.setContextClassLoader(oldClassLoader);
-   }
}
 }
diff --git 
a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
 
b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
index dab6bbe..1649c23 100644
--- 
a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
+++ 
b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
@@ -29,6 +29,7 @@ import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.plugin.TemporaryClassLoaderContext;
 import org.apache.flink.optimizer.DataStatistics;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.costs.DefaultCostEstimator;
@@ -185,12 +186,8 @@ public class ExecutionContext {
 * Executes the given supplier using the execution context's 
classloader as thread classloader.
 */
public  R wrapClassLoader(Supplier supplier) {
-   final ClassLoader previousClassloader = 
Thread.currentThread().getContextClassLoader();
-   Thread.currentThread().setContextClassLoader(classLoader);
-   try {
+   try (TemporaryClassLoaderContext tmpCl = new 
TemporaryClassLoaderContext(classLoader)){
return supplier.get();
-   } finally {
-   
Thread.currentThread().setContextClassLoader(previousClassloader);
}
}
 



[flink] branch master updated: [FLINK-10205] Introduce fault tolerance for InputSplits in batch execution

2019-04-10 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 4f689b1  [FLINK-10205] Introduce fault tolerance for InputSplits in 
batch execution
4f689b1 is described below

commit 4f689b1994eb2af584b447dc2f518e5d37bc2b6f
Author: Ryantaocer <43952554+ryantao...@users.noreply.github.com>
AuthorDate: Wed Apr 10 17:45:14 2019 +0800

[FLINK-10205] Introduce fault tolerance for InputSplits in batch execution

This closes #8125.
---
 .../api/common/io/DefaultInputSplitAssigner.java   |   9 ++
 .../api/common/io/LocatableInputSplitAssigner.java |  13 ++
 .../common/io/ReplicatingInputSplitAssigner.java   |   8 ++
 .../apache/flink/core/io/InputSplitAssigner.java   |   9 ++
 .../java/distcp/FileCopyTaskInputFormat.java   |  10 ++
 .../flink/runtime/executiongraph/Execution.java|   7 ++
 .../runtime/executiongraph/ExecutionVertex.java|  44 +--
 .../apache/flink/runtime/jobmaster/JobMaster.java  |   9 +-
 .../flink/runtime/jobmaster/JobMasterTest.java | 136 -
 .../classloading/jar/CustomInputSplitProgram.java  |   9 ++
 .../jar/StreamingCustomInputSplitProgram.java  |   9 ++
 11 files changed, 243 insertions(+), 20 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
index 25acc42..f8fd187 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
@@ -72,4 +72,13 @@ public class DefaultInputSplitAssigner implements 
InputSplitAssigner {
}
return next;
}
+
+   @Override
+   public void returnInputSplit(List splits, int taskId) {
+   synchronized (this.splits) {
+   for (InputSplit split : splits) {
+   this.splits.add(split);
+   }
+   }
+   }
 }
diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java
index 517f40a..7cfcf9f 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java
@@ -21,11 +21,13 @@ package org.apache.flink.api.common.io;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Locale;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.flink.annotation.Public;
+import org.apache.flink.core.io.InputSplit;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.core.io.InputSplitAssigner;
@@ -202,6 +204,17 @@ public final class LocatableInputSplitAssigner implements 
InputSplitAssigner {
}
}
 
+   @Override
+   public void returnInputSplit(List splits, int taskId) {
+   synchronized (this.unassigned) {
+   for (InputSplit split : splits) {
+   LocatableInputSplitWithCount lisw = new 
LocatableInputSplitWithCount((LocatableInputSplit) split);
+   this.remoteSplitChooser.addInputSplit(lisw);
+   this.unassigned.add(lisw);
+   }
+   }
+   }
+
private static final boolean isLocal(String flinkHost, String[] hosts) {
if (flinkHost == null || hosts == null) {
return false;
diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputSplitAssigner.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputSplitAssigner.java
index e7dda94..ee2c721 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputSplitAssigner.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/io/ReplicatingInputSplitAssigner.java
@@ -21,9 +21,11 @@ package org.apache.flink.api.common.io;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.util.Preconditions;
 
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 
 /**
  * Assigns each InputSplit to each requesting parallel instance.
@@ -78,4 +80,10 @@ public class ReplicatingInputSplitAssigner implements 
InputSplitAssigner {
}
 
}
+
+   @Override
+   public void returnInputSplit(List splits, int 

[flink] 03/03: [hotfix] Remove unused method in AbstractKeyedStateBackend

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 2ab4ae7e862f98cffe9be187292cea7c6ed2e05c
Author: Stefan Richter 
AuthorDate: Wed Mar 20 12:35:04 2019 +0100

[hotfix] Remove unused method in AbstractKeyedStateBackend

(cherry picked from commit b5c5499789f4e2ab21c4a05bf3da8ce7408b9663)
---
 .../org/apache/flink/runtime/state/AbstractKeyedStateBackend.java | 8 
 1 file changed, 8 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index 062092e..ab4cbf7 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -23,8 +23,6 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
-import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
@@ -206,12 +204,6 @@ public abstract class AbstractKeyedStateBackend 
implements
return keySerializerProvider.currentSchemaSerializer();
}
 
-   public TypeSerializerSchemaCompatibility 
checkKeySerializerSchemaCompatibility(
-   TypeSerializerSnapshot 
previousKeySerializerSnapshot) {
-
-   return 
keySerializerProvider.setPreviousSerializerSnapshotForRestoredState(previousKeySerializerSnapshot);
-   }
-
/**
 * @see KeyedStateBackend
 */



[flink] branch release-1.8 updated (f421f32 -> 2ab4ae7)

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from f421f32  Update NOTICE-binary with latest changes
 new 0ced11c  [hotfix] Fix inactive branch in TtlStateTestBase
 new ae91fd3  [FLINK-11980] Improve efficiency of iterating 
KeySelectionListener on notification
 new 2ab4ae7  [hotfix] Remove unused method in AbstractKeyedStateBackend

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


Summary of changes:
 .../runtime/state/AbstractKeyedStateBackend.java | 20 +++-
 .../flink/runtime/state/ttl/TtlStateTestBase.java|  2 +-
 2 files changed, 8 insertions(+), 14 deletions(-)



[flink] 02/03: [FLINK-11980] Improve efficiency of iterating KeySelectionListener on notification

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit ae91fd3bb5d1d132885399e7cc259a59459b8c6e
Author: Stefan Richter 
AuthorDate: Tue Mar 19 15:01:06 2019 +0100

[FLINK-11980] Improve efficiency of iterating KeySelectionListener on 
notification

KeySelectionListener was introduced for incremental TTL state cleanup as
 a driver of the cleanup process. Listeners are notified whenever the
 current key in the backend is set (i.e. for every event). The current
implementation of the collection that holds the listener is a HashSet,
iterated via forEach on each key change. This method comes with the
overhead of creating temporaray objects, e.g. iterators, on every
invocation and even if there is no listener registered. We should rather
use an ArrayList with for-loop iteration in this hot code path to i)
minimize overhead and ii) minimize costs for the very likely case that
there is no listener at all.

This closes #8020.

(cherry picked from commit 284e4486e57270c63b4f08d56599b1b4f97006c7)
---
 .../flink/runtime/state/AbstractKeyedStateBackend.java   | 12 +++-
 1 file changed, 7 insertions(+), 5 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index e28aeef..062092e 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -36,9 +36,8 @@ import org.apache.flink.util.Preconditions;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
 import java.util.stream.Stream;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -62,7 +61,7 @@ public abstract class AbstractKeyedStateBackend implements
private K currentKey;
 
/** Listeners to changes of keyed context ({@link #currentKey}). */
-   private final Set> keySelectionListeners;
+   private final ArrayList> keySelectionListeners;
 
/** The key group of the currently active key. */
private int currentKeyGroup;
@@ -142,7 +141,7 @@ public abstract class AbstractKeyedStateBackend 
implements
this.executionConfig = executionConfig;
this.keyGroupCompressionDecorator = 
keyGroupCompressionDecorator;
this.ttlTimeProvider = 
Preconditions.checkNotNull(ttlTimeProvider);
-   this.keySelectionListeners = new HashSet<>();
+   this.keySelectionListeners = new ArrayList<>(1);
}
 
private static StreamCompressionDecorator 
determineStreamCompression(ExecutionConfig executionConfig) {
@@ -183,7 +182,10 @@ public abstract class AbstractKeyedStateBackend 
implements
}
 
private void notifyKeySelected(K newKey) {
-   keySelectionListeners.forEach(listener -> 
listener.keySelected(newKey));
+   // we prefer a for-loop over other iteration schemes for 
performance reasons here.
+   for (int i = 0; i < keySelectionListeners.size(); ++i) {
+   keySelectionListeners.get(i).keySelected(newKey);
+   }
}
 
@Override



[flink] 01/03: [hotfix] Fix inactive branch in TtlStateTestBase

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 0ced11c8be7ccaf5112f3e050406cf75d5e55637
Author: Stefan Richter 
AuthorDate: Tue Mar 19 15:00:50 2019 +0100

[hotfix] Fix inactive branch in TtlStateTestBase

(cherry picked from commit aee1f182e8fbd311713d1661ab61755b069b1a6c)
---
 .../test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
index e6d5ba3..5c92e67 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
@@ -515,7 +515,7 @@ public abstract class TtlStateTestBase {
// trigger more cleanup by doing something out side of 
INC_CLEANUP_ALL_KEYS
for (int i = INC_CLEANUP_ALL_KEYS; i < INC_CLEANUP_ALL_KEYS * 
2; i++) {
sbetc.setCurrentKey(Integer.toString(i));
-   if (i / 2 == 0) {
+   if (i % 2 == 0) {
ctx().get();
} else {
ctx().update(ctx().updateEmpty);



[flink] 02/03: [FLINK-11980] Improve efficiency of iterating KeySelectionListener on notification

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 284e4486e57270c63b4f08d56599b1b4f97006c7
Author: Stefan Richter 
AuthorDate: Tue Mar 19 15:01:06 2019 +0100

[FLINK-11980] Improve efficiency of iterating KeySelectionListener on 
notification

KeySelectionListener was introduced for incremental TTL state cleanup as a 
driver of the cleanup process. Listeners are notified whenever the current key 
in the backend is set (i.e. for every event). The current implementation of the 
collection that holds the listener is a HashSet, iterated via forEach on each 
key change. This method comes with the overhead of creating temporaray objects, 
e.g. iterators, on every invocation and even if there is no listener 
registered. We should rather [...]

This closes #8020.
---
 .../flink/runtime/state/AbstractKeyedStateBackend.java   | 12 +++-
 1 file changed, 7 insertions(+), 5 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index e28aeef..062092e 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -36,9 +36,8 @@ import org.apache.flink.util.Preconditions;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
 import java.util.stream.Stream;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -62,7 +61,7 @@ public abstract class AbstractKeyedStateBackend implements
private K currentKey;
 
/** Listeners to changes of keyed context ({@link #currentKey}). */
-   private final Set> keySelectionListeners;
+   private final ArrayList> keySelectionListeners;
 
/** The key group of the currently active key. */
private int currentKeyGroup;
@@ -142,7 +141,7 @@ public abstract class AbstractKeyedStateBackend 
implements
this.executionConfig = executionConfig;
this.keyGroupCompressionDecorator = 
keyGroupCompressionDecorator;
this.ttlTimeProvider = 
Preconditions.checkNotNull(ttlTimeProvider);
-   this.keySelectionListeners = new HashSet<>();
+   this.keySelectionListeners = new ArrayList<>(1);
}
 
private static StreamCompressionDecorator 
determineStreamCompression(ExecutionConfig executionConfig) {
@@ -183,7 +182,10 @@ public abstract class AbstractKeyedStateBackend 
implements
}
 
private void notifyKeySelected(K newKey) {
-   keySelectionListeners.forEach(listener -> 
listener.keySelected(newKey));
+   // we prefer a for-loop over other iteration schemes for 
performance reasons here.
+   for (int i = 0; i < keySelectionListeners.size(); ++i) {
+   keySelectionListeners.get(i).keySelected(newKey);
+   }
}
 
@Override



[flink] 01/03: [hotfix] Fix inactive branch in TtlStateTestBase

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit aee1f182e8fbd311713d1661ab61755b069b1a6c
Author: Stefan Richter 
AuthorDate: Tue Mar 19 15:00:50 2019 +0100

[hotfix] Fix inactive branch in TtlStateTestBase
---
 .../test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
index e6d5ba3..5c92e67 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/TtlStateTestBase.java
@@ -515,7 +515,7 @@ public abstract class TtlStateTestBase {
// trigger more cleanup by doing something out side of 
INC_CLEANUP_ALL_KEYS
for (int i = INC_CLEANUP_ALL_KEYS; i < INC_CLEANUP_ALL_KEYS * 
2; i++) {
sbetc.setCurrentKey(Integer.toString(i));
-   if (i / 2 == 0) {
+   if (i % 2 == 0) {
ctx().get();
} else {
ctx().update(ctx().updateEmpty);



[flink] branch master updated (21cff1d -> b5c5499)

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from 21cff1d  [hotfix][runtime] Delete unused interface 
ExecutionStatusListener
 new aee1f18  [hotfix] Fix inactive branch in TtlStateTestBase
 new 284e448  [FLINK-11980] Improve efficiency of iterating 
KeySelectionListener on notification
 new b5c5499  [hotfix] Remove unused method in AbstractKeyedStateBackend

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


Summary of changes:
 .../runtime/state/AbstractKeyedStateBackend.java | 20 +++-
 .../flink/runtime/state/ttl/TtlStateTestBase.java|  2 +-
 2 files changed, 8 insertions(+), 14 deletions(-)



[flink] 03/03: [hotfix] Remove unused method in AbstractKeyedStateBackend

2019-03-20 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit b5c5499789f4e2ab21c4a05bf3da8ce7408b9663
Author: Stefan Richter 
AuthorDate: Wed Mar 20 12:35:04 2019 +0100

[hotfix] Remove unused method in AbstractKeyedStateBackend
---
 .../org/apache/flink/runtime/state/AbstractKeyedStateBackend.java | 8 
 1 file changed, 8 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index 062092e..ab4cbf7 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -23,8 +23,6 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
-import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
@@ -206,12 +204,6 @@ public abstract class AbstractKeyedStateBackend 
implements
return keySerializerProvider.currentSchemaSerializer();
}
 
-   public TypeSerializerSchemaCompatibility 
checkKeySerializerSchemaCompatibility(
-   TypeSerializerSnapshot 
previousKeySerializerSnapshot) {
-
-   return 
keySerializerProvider.setPreviousSerializerSnapshotForRestoredState(previousKeySerializerSnapshot);
-   }
-
/**
 * @see KeyedStateBackend
 */



[flink] branch master updated: [FLINK-11903][tests] Improve FileStateBackendTest by using JUnit's parameterized

2019-03-14 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 76b9f8b  [FLINK-11903][tests] Improve FileStateBackendTest by using 
JUnit's parameterized
76b9f8b is described below

commit 76b9f8b5a4a2e0384850ec1cfb3dfa7607189933
Author: Congxian Qiu 
AuthorDate: Thu Mar 14 19:12:04 2019 +0800

[FLINK-11903][tests] Improve FileStateBackendTest by using JUnit's 
parameterized

This closes #7973.
---
 .../runtime/state/AsyncFileStateBackendTest.java   | 27 --
 .../flink/runtime/state/FileStateBackendTest.java  | 19 +++
 2 files changed, 14 insertions(+), 32 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/AsyncFileStateBackendTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/AsyncFileStateBackendTest.java
deleted file mode 100644
index dd73e42..000
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/AsyncFileStateBackendTest.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.state;
-
-public class AsyncFileStateBackendTest extends FileStateBackendTest {
-
-   @Override
-   protected boolean useAsyncMode() {
-   return true;
-   }
-}
\ No newline at end of file
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
index beea0c2..8b03745 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendTest.java
@@ -24,26 +24,35 @@ import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.File;
+import java.util.Arrays;
+import java.util.List;
 
 /**
  * Tests for the keyed state backend and operator state backend, as created by 
the
  * {@link FsStateBackend}.
  */
+@RunWith(Parameterized.class)
 public class FileStateBackendTest extends StateBackendTestBase 
{
 
+   @Parameterized.Parameters
+   public static List modes() {
+   return Arrays.asList(true, false);
+   }
+
+   @Parameterized.Parameter
+   public boolean useAsyncMode;
+
@Rule
public final TemporaryFolder tempFolder = new TemporaryFolder();
 
@Override
protected FsStateBackend getStateBackend() throws Exception {
File checkpointPath = tempFolder.newFolder();
-   return new FsStateBackend(checkpointPath.toURI(), 
useAsyncMode());
-   }
-
-   protected boolean useAsyncMode() {
-   return false;
+   return new FsStateBackend(checkpointPath.toURI(), useAsyncMode);
}
 
@Override



[flink] branch master updated: [FLINK-11904][tests]Improve MemoryStateBackendTest by using JUnit's Parameterized

2019-03-14 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 6c7ddf3  [FLINK-11904][tests]Improve MemoryStateBackendTest by using 
JUnit's Parameterized
6c7ddf3 is described below

commit 6c7ddf39af5bd3bd3e9131332c104be7f0f894fb
Author: Congxian Qiu 
AuthorDate: Thu Mar 14 19:08:47 2019 +0800

[FLINK-11904][tests]Improve MemoryStateBackendTest by using JUnit's 
Parameterized

This closes #7984.
---
 .../runtime/state/AsyncMemoryStateBackendTest.java | 27 --
 .../runtime/state/MemoryStateBackendTest.java  | 20 
 2 files changed, 15 insertions(+), 32 deletions(-)

diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/AsyncMemoryStateBackendTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/AsyncMemoryStateBackendTest.java
deleted file mode 100644
index ba4a89d..000
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/AsyncMemoryStateBackendTest.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.state;
-
-public class AsyncMemoryStateBackendTest extends MemoryStateBackendTest {
-
-   @Override
-   protected boolean useAsyncMode() {
-   return true;
-   }
-}
\ No newline at end of file
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
index 2ac1ea1..9dd60a1 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java
@@ -22,19 +22,29 @@ import 
org.apache.flink.runtime.state.memory.MemoryStateBackend;
 
 import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.List;
 
 /**
  * Tests for the {@link 
org.apache.flink.runtime.state.memory.MemoryStateBackend}.
  */
+@RunWith(Parameterized.class)
 public class MemoryStateBackendTest extends 
StateBackendTestBase {
 
-   @Override
-   protected MemoryStateBackend getStateBackend() throws Exception {
-   return new MemoryStateBackend(useAsyncMode());
+   @Parameterized.Parameters(name = "useAsyncmode")
+   public static List modes() {
+   return Arrays.asList(true, false);
}
 
-   protected boolean useAsyncMode() {
-   return false;
+   @Parameterized.Parameter
+   public boolean useAsyncmode;
+
+   @Override
+   protected MemoryStateBackend getStateBackend() {
+   return new MemoryStateBackend(useAsyncmode);
}
 
@Override



[flink] branch master updated: [FLINK-11874][checkpoint] Split CheckpointStorage interface to distinguish JM and TM side

2019-03-14 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 b6b4baf  [FLINK-11874][checkpoint] Split CheckpointStorage interface 
to distinguish JM and TM side
b6b4baf is described below

commit b6b4baf588fae0494e32492ac16df8977d8cfc1b
Author: Yun Tang 
AuthorDate: Thu Mar 14 18:48:39 2019 +0800

[FLINK-11874][checkpoint] Split CheckpointStorage interface to distinguish 
JM and TM side

This closes #7970.
---
 .../runtime/checkpoint/CheckpointCoordinator.java  |  12 ++-
 .../flink/runtime/state/CheckpointStorage.java | 117 +
 java => CheckpointStorageCoordinatorView.java} |  57 ++
 .../runtime/state/CheckpointStorageWorkerView.java |  71 +
 .../state/filesystem/FsCheckpointStorage.java  |   4 +-
 .../memory/MemoryBackendCheckpointStorage.java |   6 +-
 .../flink/runtime/dispatcher/DispatcherTest.java   |   4 +-
 .../flink/runtime/state/StateBackendTestBase.java  |  10 +-
 .../state/testutils/BackendForTestStream.java  |  12 +--
 .../runtime/state/ttl/StateBackendTestContext.java |  14 +--
 .../runtime/operators/GenericWriteAheadSink.java   |   4 +-
 .../flink/streaming/runtime/tasks/StreamTask.java  |   6 +-
 .../flink/streaming/util/MockStreamTask.java   |   8 +-
 .../streaming/util/MockStreamTaskBuilder.java  |   3 +-
 14 files changed, 129 insertions(+), 199 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
index 9935455..ac4c1e2 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
@@ -35,13 +35,14 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
-import org.apache.flink.runtime.state.CheckpointStorage;
+import org.apache.flink.runtime.state.CheckpointStorageCoordinatorView;
 import org.apache.flink.runtime.state.CheckpointStorageLocation;
 import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.SharedStateRegistryFactory;
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
+import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StringUtils;
 
@@ -50,6 +51,7 @@ import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 
+import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -119,7 +121,7 @@ public class CheckpointCoordinator {
 
/** The root checkpoint state backend, which is responsible for 
initializing the
 * checkpoint, storing the metadata, and cleaning up the checkpoint. */
-   private final CheckpointStorage checkpointStorage;
+   private final CheckpointStorageCoordinatorView checkpointStorage;
 
/** A list of recent checkpoint IDs, to identify late messages (vs 
invalid ones). */
private final ArrayDeque recentPendingCheckpoints;
@@ -247,7 +249,11 @@ public class CheckpointCoordinator {
 
try {
this.checkpointStorage = 
checkpointStateBackend.createCheckpointStorage(job);
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Failed to create 
checkpoint storage at checkpoint coordinator side.", e);
+   }
 
+   try {
// Make sure the checkpoint ID enumerator is running. 
Possibly
// issues a blocking call to ZooKeeper.
checkpointIDCounter.start();
@@ -1140,7 +1146,7 @@ public class CheckpointCoordinator {
}
}
 
-   public CheckpointStorage getCheckpointStorage() {
+   public CheckpointStorageCoordinatorView getCheckpointStorage() {
return checkpointStorage;
}
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
index 0f8aa69..af02da5 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStorage.java
@@ -18,119 +18,10 @@
 
 package org.apache.flink.runtime.sta

[flink] branch master updated: [FLINK-11861][tests] Fix JobMasterTriggerSavepointIT not executed

2019-03-11 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

srichter 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 a709ed2  [FLINK-11861][tests] Fix JobMasterTriggerSavepointIT not 
executed
a709ed2 is described below

commit a709ed24ba1e4197551d3b00e55089e0722b1827
Author: ZiLi Chen 
AuthorDate: Tue Mar 12 00:53:35 2019 +0800

[FLINK-11861][tests] Fix JobMasterTriggerSavepointIT not executed

This closes #7943.
---
 ...erTriggerSavepointIT.java => JobMasterTriggerSavepointITCase.java} | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java
 
b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java
similarity index 98%
rename from 
flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java
rename to 
flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java
index 5cbde5d..73d19d8 100644
--- 
a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java
+++ 
b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java
@@ -65,7 +65,7 @@ import static org.hamcrest.Matchers.isOneOf;
  *
  * @see org.apache.flink.runtime.jobmaster.JobMaster
  */
-public class JobMasterTriggerSavepointIT extends AbstractTestBase {
+public class JobMasterTriggerSavepointITCase extends AbstractTestBase {
 
private static CountDownLatch invokeLatch;
 
@@ -206,7 +206,7 @@ public class JobMasterTriggerSavepointIT extends 
AbstractTestBase {
 
/**
 * Invokable which calls {@link CountDownLatch#countDown()} on
-* {@link JobMasterTriggerSavepointIT#invokeLatch}, and then blocks 
afterwards.
+* {@link JobMasterTriggerSavepointITCase#invokeLatch}, and then blocks 
afterwards.
 */
public static class NoOpBlockingInvokable extends AbstractInvokable {
 



[flink] branch release-1.8 updated: [hotfix][statebackend] Reduce and simplify code for column creation in RocksDB backend

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

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


The following commit(s) were added to refs/heads/release-1.8 by this push:
 new 9d87fe1  [hotfix][statebackend] Reduce and simplify code for column 
creation in RocksDB backend
9d87fe1 is described below

commit 9d87fe16c4b34e58d604f425f8e4b37de3c13ad0
Author: azagrebin 
AuthorDate: Wed Mar 6 16:45:33 2019 +0100

[hotfix][statebackend] Reduce and simplify code for column creation in 
RocksDB backend

This closes #7830.

(cherry picked from commit 953a5ffcbdae4115f7d525f310723cf8770779df)
---
 .../streaming/state/RocksDBKeyedStateBackend.java  | 16 +
 .../state/RocksDBKeyedStateBackendBuilder.java | 13 ++--
 .../streaming/state/RocksDBOperationUtils.java | 82 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  5 +-
 .../restore/AbstractRocksDBRestoreOperation.java   | 19 ++---
 .../state/restore/RocksDBFullRestoreOperation.java |  9 +--
 .../RocksDBIncrementalRestoreOperation.java| 36 +++---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +-
 .../state/ttl/RocksDbTtlCompactFiltersManager.java | 31 ++--
 9 files changed, 78 insertions(+), 140 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index c6d3863..4a6bd3c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -126,9 +126,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
RocksDBKeyedStateBackend backend) throws Exception;
}
 
-   /** String that identifies the operator that owns this backend. */
-   private final String operatorIdentifier;
-
/** Factory function to create column family options from state name. */
private final Function 
columnFamilyOptionsFactory;
 
@@ -158,9 +155,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
/** Number of bytes required to prefix the key groups. */
private final int keyGroupPrefixBytes;
 
-   /** Thread number used to transfer state files while 
restoring/snapshotting. */
-   private final int numberOfTransferingThreads;
-
/**
 * We are not using the default column family for Flink state ops, but 
we still need to remember this handle so that
 * we can close it properly when the backend is closed. Note that the 
one returned by {@link RocksDB#open(String)}
@@ -201,7 +195,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
private final RocksDbTtlCompactFiltersManager ttlCompactFiltersManager;
 
public RocksDBKeyedStateBackend(
-   String operatorIdentifier,
ClassLoader userCodeClassLoader,
File instanceBasePath,
DBOptions dbOptions,
@@ -211,7 +204,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
-   int numberOfTransferingThreads,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
LinkedHashMap kvStateInformation,
@@ -233,10 +225,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
 
this.ttlCompactFiltersManager = ttlCompactFiltersManager;
 
-   this.operatorIdentifier = 
Preconditions.checkNotNull(operatorIdentifier);
-
-   this.numberOfTransferingThreads = numberOfTransferingThreads;
-
// ensure that we use the right merge operator, because other 
code relies on this
this.columnFamilyOptionsFactory = 
Preconditions.checkNotNull(columnFamilyOptionsFactory);
 
@@ -500,8 +488,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
stateSerializer,
StateSnapshotTransformFactory.noTransform());
 
-   newRocksStateInfo = 
RocksDBOperationUtils.createStateInfo(newMetaInfo, ttlCompactFiltersManager,
-   ttlTimeProvider, db, 
columnFamilyOptionsFactory);
+   newRocksStateInfo = 
RocksDBOperationUtils.createStateInfo(
+   newMetaInfo, db, columnFamilyOptionsFactory, 
ttlCompactFiltersManager

[flink] branch master updated: [hotfix][statebackend] Reduce and simplify code for column creation in RocksDB backend

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

srichter 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 953a5ff  [hotfix][statebackend] Reduce and simplify code for column 
creation in RocksDB backend
953a5ff is described below

commit 953a5ffcbdae4115f7d525f310723cf8770779df
Author: azagrebin 
AuthorDate: Wed Mar 6 16:45:33 2019 +0100

[hotfix][statebackend] Reduce and simplify code for column creation in 
RocksDB backend

This closes #7830.
---
 .../streaming/state/RocksDBKeyedStateBackend.java  | 16 +
 .../state/RocksDBKeyedStateBackendBuilder.java | 13 ++--
 .../streaming/state/RocksDBOperationUtils.java | 82 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  5 +-
 .../restore/AbstractRocksDBRestoreOperation.java   | 19 ++---
 .../state/restore/RocksDBFullRestoreOperation.java |  9 +--
 .../RocksDBIncrementalRestoreOperation.java| 36 +++---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +-
 .../state/ttl/RocksDbTtlCompactFiltersManager.java | 31 ++--
 9 files changed, 78 insertions(+), 140 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index c6d3863..4a6bd3c 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -126,9 +126,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
RocksDBKeyedStateBackend backend) throws Exception;
}
 
-   /** String that identifies the operator that owns this backend. */
-   private final String operatorIdentifier;
-
/** Factory function to create column family options from state name. */
private final Function 
columnFamilyOptionsFactory;
 
@@ -158,9 +155,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
/** Number of bytes required to prefix the key groups. */
private final int keyGroupPrefixBytes;
 
-   /** Thread number used to transfer state files while 
restoring/snapshotting. */
-   private final int numberOfTransferingThreads;
-
/**
 * We are not using the default column family for Flink state ops, but 
we still need to remember this handle so that
 * we can close it properly when the backend is closed. Note that the 
one returned by {@link RocksDB#open(String)}
@@ -201,7 +195,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
private final RocksDbTtlCompactFiltersManager ttlCompactFiltersManager;
 
public RocksDBKeyedStateBackend(
-   String operatorIdentifier,
ClassLoader userCodeClassLoader,
File instanceBasePath,
DBOptions dbOptions,
@@ -211,7 +204,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
-   int numberOfTransferingThreads,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
LinkedHashMap kvStateInformation,
@@ -233,10 +225,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
 
this.ttlCompactFiltersManager = ttlCompactFiltersManager;
 
-   this.operatorIdentifier = 
Preconditions.checkNotNull(operatorIdentifier);
-
-   this.numberOfTransferingThreads = numberOfTransferingThreads;
-
// ensure that we use the right merge operator, because other 
code relies on this
this.columnFamilyOptionsFactory = 
Preconditions.checkNotNull(columnFamilyOptionsFactory);
 
@@ -500,8 +488,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
stateSerializer,
StateSnapshotTransformFactory.noTransform());
 
-   newRocksStateInfo = 
RocksDBOperationUtils.createStateInfo(newMetaInfo, ttlCompactFiltersManager,
-   ttlTimeProvider, db, 
columnFamilyOptionsFactory);
+   newRocksStateInfo = 
RocksDBOperationUtils.createStateInfo(
+   newMetaInfo, db, columnFamilyOptionsFactory, 
ttlCompactFiltersManager);

RocksDBOperationUtils.registerKvStateInformation(this.kvStateInformation, 
this.nativeMetricMonitor

[flink] branch master updated: [FLINK-11829][checkpoint] Avoid FsCheckpointStateOutputStream to store state in files when size below fileStateThreshold

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

srichter 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 753bc10  [FLINK-11829][checkpoint] Avoid FsCheckpointStateOutputStream 
to store state in files when size below fileStateThreshold
753bc10 is described below

commit 753bc10b6fddd83c5545ad45b944f7cd7f666c3f
Author: Yun Tang 
AuthorDate: Wed Mar 6 23:34:14 2019 +0800

[FLINK-11829][checkpoint] Avoid FsCheckpointStateOutputStream to store 
state in files when size below fileStateThreshold

This closes #7907.
---
 .../flink/runtime/state/filesystem/FsCheckpointStreamFactory.java| 2 +-
 .../runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java  | 5 +++--
 2 files changed, 4 insertions(+), 3 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
index 8253606..fcb7708 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
@@ -187,7 +187,7 @@ public class FsCheckpointStreamFactory implements 
CheckpointStreamFactory {
 
@Override
public void write(byte[] b, int off, int len) throws 
IOException {
-   if (len < writeBuffer.length / 2) {
+   if (len < writeBuffer.length) {
// copy it into our write buffer first
final int remaining = writeBuffer.length - pos;
if (len > remaining) {
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
index 8bafdf7..c06cdfa 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
@@ -86,7 +86,7 @@ public class FsCheckpointStateOutputStreamTest {
 
@Test
public void testStateBelowMemThreshold() throws Exception {
-   runTest(222, 999, 512, false);
+   runTest(999, 1024, 1000, false);
}
 
@Test
@@ -216,7 +216,8 @@ public class FsCheckpointStateOutputStreamTest {
stream.write(bytes[pos++]);
}
else {
-   int num = rnd.nextInt(Math.min(10, bytes.length 
- pos));
+   int num = rnd.nextBoolean() ?
+   (bytes.length - pos) : 
rnd.nextInt(bytes.length - pos);
stream.write(bytes, pos, num);
pos += num;
}



[flink] branch master updated: [hotfix][flink-core] fix errors in MemorySegment description

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

srichter 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 4399181  [hotfix][flink-core] fix errors in MemorySegment description
4399181 is described below

commit 4399181844a1657b78d32e1e543f63a621f07441
Author: leesf <490081...@qq.com>
AuthorDate: Wed Mar 6 23:24:14 2019 +0800

[hotfix][flink-core] fix errors in MemorySegment description

This closes #7916.
---
 .../src/main/java/org/apache/flink/core/memory/MemorySegment.java | 8 
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java 
b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
index 39b6d9c..e747324 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
@@ -991,7 +991,7 @@ public abstract class MemorySegment {
 * @return The long value at the given position.
 *
 * @throws IndexOutOfBoundsException Thrown, if the index is negative, 
or larger than the segment
-*   size minus 8.
+*   size minus 4.
 */
public final float getFloatLittleEndian(int index) {
return Float.intBitsToFloat(getIntLittleEndian(index));
@@ -1009,7 +1009,7 @@ public abstract class MemorySegment {
 * @return The long value at the given position.
 *
 * @throws IndexOutOfBoundsException Thrown, if the index is negative, 
or larger than the segment
-*   size minus 8.
+*   size minus 4.
 */
public final float getFloatBigEndian(int index) {
return Float.intBitsToFloat(getIntBigEndian(index));
@@ -1045,7 +1045,7 @@ public abstract class MemorySegment {
 * @param value The long value to be written.
 *
 * @throws IndexOutOfBoundsException Thrown, if the index is negative, 
or larger than the segment
-*   size minus 8.
+*   size minus 4.
 */
public final void putFloatLittleEndian(int index, float value) {
putIntLittleEndian(index, Float.floatToRawIntBits(value));
@@ -1063,7 +1063,7 @@ public abstract class MemorySegment {
 * @param value The long value to be written.
 *
 * @throws IndexOutOfBoundsException Thrown, if the index is negative, 
or larger than the segment
-*   size minus 8.
+*   size minus 4.
 */
public final void putFloatBigEndian(int index, float value) {
putIntBigEndian(index, Float.floatToRawIntBits(value));



[flink] branch release-1.8 updated: [FLINK-11833] [State Backends] Cleanup unnecessary createKeyedStateBackend methods in StateBackend

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

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


The following commit(s) were added to refs/heads/release-1.8 by this push:
 new b2265cd  [FLINK-11833] [State Backends] Cleanup unnecessary 
createKeyedStateBackend methods in StateBackend
b2265cd is described below

commit b2265cd7e37c130a5b31e1f276d15cca4a5735ae
Author: Yu Li 
AuthorDate: Wed Mar 6 10:26:58 2019 +0100

[FLINK-11833] [State Backends] Cleanup unnecessary createKeyedStateBackend 
methods in StateBackend

This closes #7909.

(cherry picked from commit f10a7d8b5da5d9836a8fb3b2f38e099152f1d75f)
---
 .../flink/queryablestate/network/ClientTest.java   |  21 ++--
 .../queryablestate/network/KvStateServerTest.java  |   9 +-
 .../apache/flink/runtime/state/StateBackend.java   | 119 +++--
 .../HeapKeyedStateBackendAsyncByDefaultTest.java   |   9 +-
 .../state/StateBackendMigrationTestBase.java   |  15 ++-
 .../flink/runtime/state/StateBackendTestBase.java  |  25 +++--
 .../runtime/state/ttl/StateBackendTestContext.java |  15 ++-
 .../streaming/state/RocksDBAsyncSnapshotTest.java  |  10 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |  10 +-
 .../state/RocksDBStateBackendConfigTest.java   |  69 +++-
 .../api/operators/StreamingRuntimeContextTest.java |  39 ---
 .../operators/windowing/TriggerTestHarness.java|  23 ++--
 12 files changed, 185 insertions(+), 179 deletions(-)

diff --git 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
index bceb361..85b29fa 100644
--- 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
+++ 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
@@ -22,6 +22,8 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.queryablestate.client.VoidNamespace;
 import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
@@ -40,6 +42,7 @@ import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.NetUtils;
 
@@ -630,13 +633,17 @@ public class ClientTest {
dummyEnv.setKvStateRegistry(dummyRegistry);
 
AbstractKeyedStateBackend backend = 
abstractBackend.createKeyedStateBackend(
-   dummyEnv,
-   new JobID(),
-   "test_op",
-   IntSerializer.INSTANCE,
-   numKeyGroups,
-   new KeyGroupRange(0, 0),
-   dummyRegistry.createTaskRegistry(new JobID(), 
new JobVertexID()));
+   dummyEnv,
+   new JobID(),
+   "test_op",
+   IntSerializer.INSTANCE,
+   numKeyGroups,
+   new KeyGroupRange(0, 0),
+   dummyRegistry.createTaskRegistry(new JobID(), new 
JobVertexID()),
+   TtlTimeProvider.DEFAULT,
+   new UnregisteredMetricsGroup(),
+   Collections.emptyList(),
+   new CloseableRegistry());
 
final FiniteDuration timeout = new FiniteDuration(10, 
TimeUnit.SECONDS);
 
diff --git 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
index 79c23ad..6aace48 100644
--- 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
+++ 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
@@ -22,6 +22,8 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.state

[flink] branch master updated: [FLINK-11833] [State Backends] Cleanup unnecessary createKeyedStateBackend methods in StateBackend

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

srichter 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 f10a7d8  [FLINK-11833] [State Backends] Cleanup unnecessary 
createKeyedStateBackend methods in StateBackend
f10a7d8 is described below

commit f10a7d8b5da5d9836a8fb3b2f38e099152f1d75f
Author: Yu Li 
AuthorDate: Wed Mar 6 17:26:58 2019 +0800

[FLINK-11833] [State Backends] Cleanup unnecessary createKeyedStateBackend 
methods in StateBackend

This closes #7909.
---
 .../flink/queryablestate/network/ClientTest.java   |  21 ++--
 .../queryablestate/network/KvStateServerTest.java  |   9 +-
 .../apache/flink/runtime/state/StateBackend.java   | 119 +++--
 .../HeapKeyedStateBackendAsyncByDefaultTest.java   |   9 +-
 .../state/StateBackendMigrationTestBase.java   |  15 ++-
 .../flink/runtime/state/StateBackendTestBase.java  |  25 +++--
 .../runtime/state/ttl/StateBackendTestContext.java |  15 ++-
 .../streaming/state/RocksDBAsyncSnapshotTest.java  |  10 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |  10 +-
 .../state/RocksDBStateBackendConfigTest.java   |  69 +++-
 .../api/operators/StreamingRuntimeContextTest.java |  39 ---
 .../operators/windowing/TriggerTestHarness.java|  23 ++--
 12 files changed, 185 insertions(+), 179 deletions(-)

diff --git 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
index bceb361..85b29fa 100644
--- 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
+++ 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java
@@ -22,6 +22,8 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.queryablestate.KvStateID;
 import org.apache.flink.queryablestate.client.VoidNamespace;
 import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
@@ -40,6 +42,7 @@ import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.NetUtils;
 
@@ -630,13 +633,17 @@ public class ClientTest {
dummyEnv.setKvStateRegistry(dummyRegistry);
 
AbstractKeyedStateBackend backend = 
abstractBackend.createKeyedStateBackend(
-   dummyEnv,
-   new JobID(),
-   "test_op",
-   IntSerializer.INSTANCE,
-   numKeyGroups,
-   new KeyGroupRange(0, 0),
-   dummyRegistry.createTaskRegistry(new JobID(), 
new JobVertexID()));
+   dummyEnv,
+   new JobID(),
+   "test_op",
+   IntSerializer.INSTANCE,
+   numKeyGroups,
+   new KeyGroupRange(0, 0),
+   dummyRegistry.createTaskRegistry(new JobID(), new 
JobVertexID()),
+   TtlTimeProvider.DEFAULT,
+   new UnregisteredMetricsGroup(),
+   Collections.emptyList(),
+   new CloseableRegistry());
 
final FiniteDuration timeout = new FiniteDuration(10, 
TimeUnit.SECONDS);
 
diff --git 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
index 79c23ad..6aace48 100644
--- 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
+++ 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java
@@ -22,6 +22,8 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStat

[flink] 01/02: [FLINK-11731] [State Backends] Make DefaultOperatorStateBackend follow the builder pattern

2019-03-05 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit 2f62b890624116cbe205cdcc9762e41f6a80c84f
Author: Yu Li 
AuthorDate: Sat Mar 2 11:19:21 2019 +0100

[FLINK-11731] [State Backends] Make DefaultOperatorStateBackend follow the 
builder pattern

This closes #7899.

(cherry picked from commit 94f84a5c7b4876dee5b34e6a3725da7ee06a607f)
---
 .../flink/streaming/tests/StubStateBackend.java|   9 +-
 .../flink/runtime/state/AbstractStateBackend.java  |   6 +-
 .../runtime/state/DefaultOperatorStateBackend.java | 486 +
 .../state/DefaultOperatorStateBackendBuilder.java  |  98 +
 ...efaultOperatorStateBackendSnapshotStrategy.java | 213 +
 .../state/OperatorStateRestoreOperation.java   | 219 ++
 .../runtime/state/PartitionableListState.java  | 135 ++
 .../apache/flink/runtime/state/StateBackend.java   |   8 +-
 .../runtime/state/filesystem/FsStateBackend.java   |  13 +-
 .../state/heap/HeapKeyedStateBackendBuilder.java   |   1 +
 .../runtime/state/memory/MemoryStateBackend.java   |  19 +-
 .../CheckpointSettingsSerializableTest.java|   6 +-
 .../runtime/state/OperatorStateBackendTest.java| 114 +++--
 .../state/StateBackendMigrationTestBase.java   |  13 +-
 .../runtime/state/ttl/mock/MockStateBackend.java   |   7 +-
 .../streaming/state/RocksDBStateBackend.java   |  19 +-
 .../operators/StreamTaskStateInitializerImpl.java  |  21 +-
 .../operators/BackendRestorerProcedureTest.java|  10 +-
 .../StreamTaskStateInitializerImplTest.java|   5 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   6 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |  97 ++--
 .../runtime/tasks/TestSpyWrapperStateBackend.java  |   8 +-
 .../test/streaming/runtime/StateBackendITCase.java |   5 +-
 23 files changed, 962 insertions(+), 556 deletions(-)

diff --git 
a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java
 
b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java
index dec4f2d..0d0c895 100644
--- 
a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java
+++ 
b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java
@@ -30,6 +30,7 @@ import 
org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.runtime.state.ttl.TtlTimeProvider;
 
@@ -96,7 +97,11 @@ final class StubStateBackend implements StateBackend {
}
 
@Override
-   public OperatorStateBackend createOperatorStateBackend(Environment env, 
String operatorIdentifier) throws Exception {
-   return backend.createOperatorStateBackend(env, 
operatorIdentifier);
+   public OperatorStateBackend createOperatorStateBackend(
+   Environment env,
+   String operatorIdentifier,
+   @Nonnull Collection stateHandles,
+   CloseableRegistry cancelStreamRegistry) throws Exception {
+   return backend.createOperatorStateBackend(env, 
operatorIdentifier, stateHandles, cancelStreamRegistry);
}
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
index 2343d83..8d94898 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractStateBackend.java
@@ -71,6 +71,8 @@ public abstract class AbstractStateBackend implements 
StateBackend, java.io.Seri
 
@Override
public abstract OperatorStateBackend createOperatorStateBackend(
-   Environment env,
-   String operatorIdentifier) throws Exception;
+   Environment env,
+   String operatorIdentifier,
+   @Nonnull Collection stateHandles,
+   CloseableRegistry cancelStreamRegistry) throws Exception;
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
index a3541b6..f6a0dba 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state

[flink] branch release-1.8 updated (b027fa7 -> ce2d65d)

2019-03-05 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from b027fa7  Fix version change expressions in releasing scripts
 new 2f62b89  [FLINK-11731] [State Backends] Make 
DefaultOperatorStateBackend follow the builder pattern
 new ce2d65d  [FLINK-11796] [State Backends] Remove Snapshotable interface

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


Summary of changes:
 .../flink/streaming/tests/StubStateBackend.java|   9 +-
 .../runtime/state/AbstractKeyedStateBackend.java   |   3 +-
 .../flink/runtime/state/AbstractStateBackend.java  |   6 +-
 .../runtime/state/DefaultOperatorStateBackend.java | 492 +
 .../state/DefaultOperatorStateBackendBuilder.java  |  98 
 ...efaultOperatorStateBackendSnapshotStrategy.java | 213 +
 .../flink/runtime/state/OperatorStateBackend.java  |   5 +-
 .../state/OperatorStateRestoreOperation.java   | 219 +
 .../runtime/state/PartitionableListState.java  | 135 ++
 .../apache/flink/runtime/state/Snapshotable.java   |  41 --
 .../apache/flink/runtime/state/StateBackend.java   |   8 +-
 .../runtime/state/filesystem/FsStateBackend.java   |  13 +-
 .../runtime/state/heap/HeapKeyedStateBackend.java  |   6 -
 .../state/heap/HeapKeyedStateBackendBuilder.java   |   1 +
 .../runtime/state/memory/MemoryStateBackend.java   |  19 +-
 .../CheckpointSettingsSerializableTest.java|   6 +-
 .../runtime/state/OperatorStateBackendTest.java| 114 +++--
 .../state/StateBackendMigrationTestBase.java   |  13 +-
 .../state/ttl/mock/MockKeyedStateBackend.java  |   7 -
 .../runtime/state/ttl/mock/MockStateBackend.java   |   7 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |   6 -
 .../streaming/state/RocksDBStateBackend.java   |  19 +-
 .../api/operators/BackendRestorerProcedure.java|  23 +-
 .../operators/StreamTaskStateInitializerImpl.java  |  21 +-
 .../operators/BackendRestorerProcedureTest.java|  10 +-
 .../StreamTaskStateInitializerImplTest.java|  11 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   6 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |  97 ++--
 .../runtime/tasks/TestSpyWrapperStateBackend.java  |   8 +-
 .../test/streaming/runtime/StateBackendITCase.java |   5 +-
 30 files changed, 968 insertions(+), 653 deletions(-)
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackendBuilder.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackendSnapshotStrategy.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateRestoreOperation.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java
 delete mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java



[flink] 02/02: [FLINK-11796] [State Backends] Remove Snapshotable interface

2019-03-05 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit ce2d65d88d28bcd5cf182395d511a239dacfb89d
Author: Yu Li 
AuthorDate: Sat Mar 2 20:00:43 2019 +0100

[FLINK-11796] [State Backends] Remove Snapshotable interface

(cherry picked from commit e8daa49a593edc401cd44761b25b1324b11be4a6)
---
 .../runtime/state/AbstractKeyedStateBackend.java   |  3 +-
 .../runtime/state/DefaultOperatorStateBackend.java |  8 +
 .../flink/runtime/state/OperatorStateBackend.java  |  5 ++-
 .../apache/flink/runtime/state/Snapshotable.java   | 41 --
 .../runtime/state/heap/HeapKeyedStateBackend.java  |  6 
 .../state/ttl/mock/MockKeyedStateBackend.java  |  7 
 .../streaming/state/RocksDBKeyedStateBackend.java  |  6 
 .../api/operators/BackendRestorerProcedure.java| 23 ++--
 .../StreamTaskStateInitializerImplTest.java|  6 
 9 files changed, 7 insertions(+), 98 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index 98e21ff..e28aeef 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -36,7 +36,6 @@ import org.apache.flink.util.Preconditions;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
@@ -52,7 +51,7 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
  */
 public abstract class AbstractKeyedStateBackend implements
KeyedStateBackend,
-   Snapshotable, 
Collection>,
+   SnapshotStrategy>,
Closeable,
CheckpointListener {
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
index f6a0dba..48c8eb8 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
@@ -39,7 +39,6 @@ import javax.annotation.Nonnull;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.RunnableFuture;
@@ -246,13 +245,8 @@ public class DefaultOperatorStateBackend implements 
OperatorStateBackend {
}
 
// 
---
-   //  Snapshot and restore
+   //  Snapshot
// 
---
-
-   public void restore(Collection restoreSnapshots) 
throws Exception {
-   // all restore work done in builder and nothing to do here
-   }
-
@Nonnull
@Override
public RunnableFuture> snapshot(
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
index 3cbb351..4fb8024 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
@@ -22,16 +22,15 @@ import org.apache.flink.api.common.state.OperatorStateStore;
 import org.apache.flink.util.Disposable;
 
 import java.io.Closeable;
-import java.util.Collection;
 
 /**
  * Interface that combines both, the user facing {@link OperatorStateStore} 
interface and the system interface
- * {@link Snapshotable}
+ * {@link SnapshotStrategy}
  *
  */
 public interface OperatorStateBackend extends
OperatorStateStore,
-   Snapshotable, 
Collection>,
+   SnapshotStrategy>,
Closeable,
Disposable {
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java
deleted file mode 100644
index 1677855..000
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obt

[flink] branch master updated (fb256d4 -> e8daa49)

2019-03-05 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from fb256d4  [FLINK-11803][table-planner-blink] Improve FlinkTypeFactory 
for Blink (#7898)
 new 94f84a5  [FLINK-11731] [State Backends] Make 
DefaultOperatorStateBackend follow the builder pattern
 new e8daa49  [FLINK-11796] [State Backends] Remove Snapshotable interface

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


Summary of changes:
 .../flink/streaming/tests/StubStateBackend.java|   9 +-
 .../runtime/state/AbstractKeyedStateBackend.java   |   3 +-
 .../flink/runtime/state/AbstractStateBackend.java  |   6 +-
 .../runtime/state/DefaultOperatorStateBackend.java | 492 +
 .../state/DefaultOperatorStateBackendBuilder.java  |  98 
 ...efaultOperatorStateBackendSnapshotStrategy.java | 213 +
 .../flink/runtime/state/OperatorStateBackend.java  |   5 +-
 .../state/OperatorStateRestoreOperation.java   | 219 +
 .../runtime/state/PartitionableListState.java  | 135 ++
 .../apache/flink/runtime/state/Snapshotable.java   |  41 --
 .../apache/flink/runtime/state/StateBackend.java   |   8 +-
 .../runtime/state/filesystem/FsStateBackend.java   |  13 +-
 .../runtime/state/heap/HeapKeyedStateBackend.java  |   6 -
 .../state/heap/HeapKeyedStateBackendBuilder.java   |   1 +
 .../runtime/state/memory/MemoryStateBackend.java   |  19 +-
 .../CheckpointSettingsSerializableTest.java|   6 +-
 .../runtime/state/OperatorStateBackendTest.java| 114 +++--
 .../state/StateBackendMigrationTestBase.java   |  13 +-
 .../state/ttl/mock/MockKeyedStateBackend.java  |   7 -
 .../runtime/state/ttl/mock/MockStateBackend.java   |   7 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |   6 -
 .../streaming/state/RocksDBStateBackend.java   |  19 +-
 .../api/operators/BackendRestorerProcedure.java|  23 +-
 .../operators/StreamTaskStateInitializerImpl.java  |  21 +-
 .../operators/BackendRestorerProcedureTest.java|  10 +-
 .../StreamTaskStateInitializerImplTest.java|  11 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   6 +-
 .../tasks/TaskCheckpointingBehaviourTest.java  |  97 ++--
 .../runtime/tasks/TestSpyWrapperStateBackend.java  |   8 +-
 .../test/streaming/runtime/StateBackendITCase.java |   5 +-
 30 files changed, 968 insertions(+), 653 deletions(-)
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackendBuilder.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackendSnapshotStrategy.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateRestoreOperation.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionableListState.java
 delete mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java



[flink] 02/02: [FLINK-11796] [State Backends] Remove Snapshotable interface

2019-03-05 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit e8daa49a593edc401cd44761b25b1324b11be4a6
Author: Yu Li 
AuthorDate: Sun Mar 3 03:00:43 2019 +0800

[FLINK-11796] [State Backends] Remove Snapshotable interface
---
 .../runtime/state/AbstractKeyedStateBackend.java   |  3 +-
 .../runtime/state/DefaultOperatorStateBackend.java |  8 +
 .../flink/runtime/state/OperatorStateBackend.java  |  5 ++-
 .../apache/flink/runtime/state/Snapshotable.java   | 41 --
 .../runtime/state/heap/HeapKeyedStateBackend.java  |  6 
 .../state/ttl/mock/MockKeyedStateBackend.java  |  7 
 .../streaming/state/RocksDBKeyedStateBackend.java  |  6 
 .../api/operators/BackendRestorerProcedure.java| 23 ++--
 .../StreamTaskStateInitializerImplTest.java|  6 
 9 files changed, 7 insertions(+), 98 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
index 98e21ff..e28aeef 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java
@@ -36,7 +36,6 @@ import org.apache.flink.util.Preconditions;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
@@ -52,7 +51,7 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
  */
 public abstract class AbstractKeyedStateBackend implements
KeyedStateBackend,
-   Snapshotable, 
Collection>,
+   SnapshotStrategy>,
Closeable,
CheckpointListener {
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
index f6a0dba..48c8eb8 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
@@ -39,7 +39,6 @@ import javax.annotation.Nonnull;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.RunnableFuture;
@@ -246,13 +245,8 @@ public class DefaultOperatorStateBackend implements 
OperatorStateBackend {
}
 
// 
---
-   //  Snapshot and restore
+   //  Snapshot
// 
---
-
-   public void restore(Collection restoreSnapshots) 
throws Exception {
-   // all restore work done in builder and nothing to do here
-   }
-
@Nonnull
@Override
public RunnableFuture> snapshot(
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
index 3cbb351..4fb8024 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java
@@ -22,16 +22,15 @@ import org.apache.flink.api.common.state.OperatorStateStore;
 import org.apache.flink.util.Disposable;
 
 import java.io.Closeable;
-import java.util.Collection;
 
 /**
  * Interface that combines both, the user facing {@link OperatorStateStore} 
interface and the system interface
- * {@link Snapshotable}
+ * {@link SnapshotStrategy}
  *
  */
 public interface OperatorStateBackend extends
OperatorStateStore,
-   Snapshotable, 
Collection>,
+   SnapshotStrategy>,
Closeable,
Disposable {
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java
deleted file mode 100644
index 1677855..000
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unle

[flink] 02/02: [FLINK-11730] [State Backends] Make HeapKeyedStateBackend follow the builder pattern

2019-03-04 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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

commit d5364dff200728d8611e9cdf0e8f798de145adcf
Author: Yu Li 
AuthorDate: Thu Feb 28 20:02:35 2019 +0100

[FLINK-11730] [State Backends] Make HeapKeyedStateBackend follow the 
builder pattern

This closes #7866.

(cherry picked from commit 237d07c76b51c171f0f41f9f82d777df26da1dd4)
---
 .../KVStateRequestSerializerRocksDBTest.java   |   5 +-
 .../network/KvStateRequestSerializerTest.java  |  57 +--
 .../flink/runtime/state/AbstractStateBackend.java  |   9 +
 .../flink/runtime/state/RestoreOperation.java  |  10 +-
 .../runtime/state/filesystem/FsStateBackend.java   |  34 +-
 ...AsyncSnapshotStrategySynchronicityBehavior.java |  26 +-
 .../runtime/state/heap/HeapKeyedStateBackend.java  | 546 ++---
 .../state/heap/HeapKeyedStateBackendBuilder.java   | 152 ++
 .../runtime/state/heap/HeapRestoreOperation.java   | 293 +++
 .../runtime/state/heap/HeapSnapshotStrategy.java   | 271 ++
 .../SnapshotStrategySynchronicityBehavior.java |  24 +-
 .../apache/flink/runtime/state/heap/StateUID.java  |  73 +++
 .../SyncSnapshotStrategySynchronicityBehavior.java |  32 +-
 .../runtime/state/memory/MemoryStateBackend.java   |  34 +-
 .../state/StateBackendMigrationTestBase.java   |   2 -
 .../flink/runtime/state/StateBackendTestBase.java  |  14 +-
 .../state/StateSnapshotCompressionTest.java|  89 ++--
 ...HeapKeyedStateBackendSnapshotMigrationTest.java |  25 +-
 .../state/heap/HeapStateBackendTestBase.java   |  23 +-
 .../runtime/state/ttl/StateBackendTestContext.java |  13 -
 .../flink/runtime/state/ttl/TtlStateTestBase.java  |   3 -
 .../state/ttl/mock/MockKeyedStateBackend.java  |  24 +-
 .../ttl/mock/MockKeyedStateBackendBuilder.java |  85 
 .../state/ttl/mock/MockRestoreOperation.java   |  53 ++
 .../runtime/state/ttl/mock/MockStateBackend.java   |   8 +-
 .../streaming/state/RocksDBStateBackend.java   |  10 -
 .../state/restore/RocksDBRestoreOperation.java |   4 +-
 .../streaming/state/RocksDBStateBackendTest.java   |   5 +-
 28 files changed, 1188 insertions(+), 736 deletions(-)

diff --git 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java
 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java
index 3431199..a5df958 100644
--- 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java
+++ 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java
@@ -32,6 +32,7 @@ import 
org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
 import org.apache.flink.queryablestate.client.VoidNamespace;
 import org.apache.flink.queryablestate.client.VoidNamespaceSerializer;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
 import org.apache.flink.runtime.state.internal.InternalListState;
@@ -88,7 +89,7 @@ public final class KVStateRequestSerializerRocksDBTest {
TtlTimeProvider.DEFAULT,
new UnregisteredMetricsGroup(),
Collections.emptyList(),
-   
RocksDBStateBackend.getCompressionDecorator(executionConfig),
+   
AbstractStateBackend.getCompressionDecorator(executionConfig),
new CloseableRegistry()
).build();
longHeapKeyedStateBackend.setCurrentKey(key);
@@ -132,7 +133,7 @@ public final class KVStateRequestSerializerRocksDBTest {
TtlTimeProvider.DEFAULT,
new UnregisteredMetricsGroup(),
Collections.emptyList(),
-   
RocksDBStateBackend.getCompressionDecorator(executionConfig),
+   
AbstractStateBackend.getCompressionDecorator(executionConfig),
new CloseableRegistry()
).build();
longHeapKeyedStateBackend.setCurrentKey(key);
diff --git 
a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
 
b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java
index aac3394..2ad202f 100644

[flink] branch release-1.8 updated (8a5c972 -> d5364df)

2019-03-04 Thread srichter
This is an automated email from the ASF dual-hosted git repository.

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


from 8a5c972  [hotfix][tests] Added matcher that performs deep comparison 
with taking Tuples into account
 new e177ba9  [FLINK-11804] [State Backends] Make sure the 
CloseableRegistry used in backend builder is registered with task
 new d5364df  [FLINK-11730] [State Backends] Make HeapKeyedStateBackend 
follow the builder pattern

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


Summary of changes:
 .../flink/streaming/tests/StubStateBackend.java|   9 +-
 .../KVStateRequestSerializerRocksDBTest.java   |   8 +-
 .../network/KvStateRequestSerializerTest.java  |  58 ++-
 .../network/KvStateServerHandlerTest.java  |   4 +-
 .../state/AbstractKeyedStateBackendBuilder.java|   6 +-
 .../flink/runtime/state/AbstractStateBackend.java  |  13 +-
 .../flink/runtime/state/RestoreOperation.java  |  10 +-
 .../apache/flink/runtime/state/StateBackend.java   |  10 +-
 .../runtime/state/filesystem/FsStateBackend.java   |  35 +-
 ...syncSnapshotStrategySynchronicityBehavior.java} |  24 +-
 .../runtime/state/heap/HeapKeyedStateBackend.java  | 559 ++---
 .../state/heap/HeapKeyedStateBackendBuilder.java   | 152 ++
 .../runtime/state/heap/HeapRestoreOperation.java   | 293 +++
 .../runtime/state/heap/HeapSnapshotStrategy.java   | 271 ++
 .../SnapshotStrategySynchronicityBehavior.java}|  28 +-
 .../apache/flink/runtime/state/heap/StateUID.java  |  73 +++
 ...SyncSnapshotStrategySynchronicityBehavior.java} |  30 +-
 .../runtime/state/memory/MemoryStateBackend.java   |  36 +-
 .../CheckpointSettingsSerializableTest.java|   5 +-
 .../state/StateBackendMigrationTestBase.java   |   2 -
 .../flink/runtime/state/StateBackendTestBase.java  |  14 +-
 .../state/StateSnapshotCompressionTest.java|  86 ++--
 ...HeapKeyedStateBackendSnapshotMigrationTest.java |  25 +-
 .../state/heap/HeapStateBackendTestBase.java   |  23 +-
 .../runtime/state/ttl/StateBackendTestContext.java |  13 -
 .../flink/runtime/state/ttl/TtlStateTestBase.java  |   3 -
 .../state/ttl/mock/MockKeyedStateBackend.java  |  27 +-
 .../ttl/mock/MockKeyedStateBackendBuilder.java |  85 
 .../state/ttl/mock/MockRestoreOperation.java   |  53 ++
 .../runtime/state/ttl/mock/MockStateBackend.java   |  12 +-
 .../state/RocksDBKeyedStateBackendBuilder.java |  21 +-
 .../streaming/state/RocksDBStateBackend.java   |  17 +-
 .../state/restore/RocksDBRestoreOperation.java |   4 +-
 .../streaming/state/RocksDBStateBackendTest.java   |  14 +-
 .../operators/StreamTaskStateInitializerImpl.java  |  18 +-
 .../StreamTaskStateInitializerImplTest.java|   6 +-
 .../runtime/tasks/StreamTaskTerminationTest.java   |   6 +-
 .../runtime/tasks/TestSpyWrapperStateBackend.java  |  10 +-
 .../test/streaming/runtime/StateBackendITCase.java |   6 +-
 39 files changed, 1286 insertions(+), 783 deletions(-)
 copy 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/restore/RocksDBRestoreOperation.java
 => 
flink-runtime/src/main/java/org/apache/flink/runtime/state/RestoreOperation.java
 (82%)
 copy 
flink-runtime/src/{test/java/org/apache/flink/runtime/state/heap/HeapPriorityQueueSetTest.java
 => 
main/java/org/apache/flink/runtime/state/heap/AsyncSnapshotStrategySynchronicityBehavior.java}
 (62%)
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendBuilder.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapRestoreOperation.java
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapSnapshotStrategy.java
 copy 
flink-runtime/src/{test/java/org/apache/flink/runtime/state/heap/HeapPriorityQueueSetTest.java
 => 
main/java/org/apache/flink/runtime/state/heap/SnapshotStrategySynchronicityBehavior.java}
 (63%)
 create mode 100644 
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateUID.java
 copy 
flink-runtime/src/{test/java/org/apache/flink/runtime/state/heap/HeapPriorityQueueSetTest.java
 => 
main/java/org/apache/flink/runtime/state/heap/SyncSnapshotStrategySynchronicityBehavior.java}
 (54%)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockKeyedStateBackendBuilder.java
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockRestoreOperation.java



  1   2   3   4   5   6   7   >