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

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


The following commit(s) were added to refs/heads/release-1.11 by this push:
     new 54ecdc2  [FLINK-15620][state][TTL] Remove deprecated enable default 
background cleanup
54ecdc2 is described below

commit 54ecdc260ba1028b52ff052c6de957c25939245b
Author: Jiayi Liao <[email protected]>
AuthorDate: Sun May 24 23:40:45 2020 +0800

    [FLINK-15620][state][TTL] Remove deprecated enable default background 
cleanup
    
    This closes #12304.
---
 .../apache/flink/api/common/state/StateTtlConfig.java  | 18 +-----------------
 .../flink/api/common/state/StateTtlConfigTest.java     |  1 -
 .../streaming/tests/DataStreamStateTTLTestProgram.java |  1 -
 .../streaming/state/ttl/RocksDBTtlStateTestBase.java   |  1 -
 4 files changed, 1 insertion(+), 20 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/state/StateTtlConfig.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/state/StateTtlConfig.java
index 1108bac..9ac0c39 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/state/StateTtlConfig.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/state/StateTtlConfig.java
@@ -282,7 +282,7 @@ public class StateTtlConfig implements Serializable {
                 * 
https://github.com/facebook/rocksdb/blob/master/include/rocksdb/compaction_filter.h#L69
                 * It means that the TTL filter should be tested for List state 
taking into account this caveat.
                 *
-                * @deprecated Use more general configuration method {@link 
#cleanupInBackground()} instead
+                * @deprecated Use more general configuration method {@link 
#cleanupInRocksdbCompactFilter(long)} instead
                 */
                @Nonnull
                @Deprecated
@@ -309,22 +309,6 @@ public class StateTtlConfig implements Serializable {
                }
 
                /**
-                * Enable default cleanup of expired state in background 
(enabled by default).
-                *
-                * <p>Depending on actually used backend, the corresponding 
default cleanup will kick in if supported.
-                * If some specific cleanup is also configured, e.g. {@link 
#cleanupIncrementally(int, boolean)} or
-                * {@link #cleanupInRocksdbCompactFilter()}, then the specific 
one will kick in instead of default.
-                *
-                * @deprecated enabled by default, no need to enable it manually
-                */
-               @Nonnull
-               @Deprecated
-               public Builder cleanupInBackground() {
-                       isCleanupInBackground = true;
-                       return this;
-               }
-
-               /**
                 * Disable default cleanup of expired state in background 
(enabled by default).
                 *
                 * <p>If some specific cleanup is configured, e.g. {@link 
#cleanupIncrementally(int, boolean)} or
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/state/StateTtlConfigTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/state/StateTtlConfigTest.java
index 8b604d3..5b9f061 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/common/state/StateTtlConfigTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/state/StateTtlConfigTest.java
@@ -59,7 +59,6 @@ public class StateTtlConfigTest {
        public void testStateTtlConfigBuildWithCleanupInBackground() {
                StateTtlConfig ttlConfig = StateTtlConfig
                        .newBuilder(Time.seconds(1))
-                       .cleanupInBackground()
                        .build();
 
                assertThat(ttlConfig.getCleanupStrategies(), notNullValue());
diff --git 
a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
 
b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
index 5232433..21baed5 100644
--- 
a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
+++ 
b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
@@ -58,7 +58,6 @@ public class DataStreamStateTTLTestProgram {
                TtlTestConfig config = TtlTestConfig.fromArgs(pt);
                StateTtlConfig ttlConfig = StateTtlConfig.newBuilder(config.ttl)
                        .cleanupFullSnapshot()
-                       .cleanupInBackground()
                        .build();
 
                env
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/ttl/RocksDBTtlStateTestBase.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/ttl/RocksDBTtlStateTestBase.java
index 82e5289..4129c00 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/ttl/RocksDBTtlStateTestBase.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/ttl/RocksDBTtlStateTestBase.java
@@ -99,7 +99,6 @@ public abstract class RocksDBTtlStateTestBase extends 
TtlStateTestBase {
                }
 
                StateDescriptor<?, ?> stateDesc = initTest(getConfBuilder(TTL)
-                       .cleanupInBackground()
                        
.setStateVisibility(StateTtlConfig.StateVisibility.ReturnExpiredIfNotCleanedUp)
                        .build());
 

Reply via email to