[flink] branch master updated: [FLINK-21413][state] Clean TtlMapState and TtlListState after all elements are expired

2021-03-31 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu 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 1e17621  [FLINK-21413][state] Clean TtlMapState and TtlListState after 
all elements are expired
1e17621 is described below

commit 1e17621e65bfa8f4f3aff379118654ed77a82bd2
Author: liaojiayi 
AuthorDate: Thu Mar 18 18:55:43 2021 +0800

[FLINK-21413][state] Clean TtlMapState and TtlListState after all elements 
are expired

This closes #15016.
---
 .../flink/runtime/state/ttl/TtlListState.java  |  8 -
 .../flink/runtime/state/ttl/TtlMapState.java   | 11 ++-
 .../flink/runtime/state/ttl/TtlStateTestBase.java  | 34 +-
 .../runtime/state/ttl/TtlStateTestContextBase.java |  5 
 4 files changed, 55 insertions(+), 3 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlListState.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlListState.java
index 5ab9813..7971275 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlListState.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlListState.java
@@ -103,6 +103,11 @@ class TtlListState
 @Nullable
 @Override
 public List> getUnexpiredOrNull(@Nonnull List> 
ttlValues) {
+// the update operation will clear the whole state if the list becomes 
empty after init
+if (ttlValues.isEmpty()) {
+return ttlValues;
+}
+
 long currentTimestamp = timeProvider.currentTimestamp();
 List> unexpired = new ArrayList<>(ttlValues.size());
 TypeSerializer> elementSerializer =
@@ -117,7 +122,8 @@ class TtlListState
 if (!unexpired.isEmpty()) {
 return unexpired;
 } else {
-return ttlValues.size() == unexpired.size() ? ttlValues : 
unexpired;
+// list is not empty and all expired
+return null;
 }
 }
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java
index d4fb7a8..bb1e20e 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java
@@ -134,6 +134,10 @@ class TtlMapState
 @Nullable
 @Override
 public Map> getUnexpiredOrNull(@Nonnull Map> ttlValue) {
+// the remove operation will clear the whole state if the map becomes 
empty after init
+if (ttlValue.isEmpty()) {
+return ttlValue;
+}
 Map> unexpired = new HashMap<>();
 TypeSerializer> valueSerializer =
 ((MapSerializer>) 
original.getValueSerializer())
@@ -144,7 +148,12 @@ class TtlMapState
 unexpired.put(e.getKey(), valueSerializer.copy(e.getValue()));
 }
 }
-return ttlValue.size() == unexpired.size() ? ttlValue : unexpired;
+if (!unexpired.isEmpty()) {
+return unexpired;
+} else {
+// map is not empty but all values expired
+return null;
+}
 }
 
 @Override
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 ef8fe2e..d6225ab 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
@@ -25,8 +25,10 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.heap.AbstractHeapState;
 import org.apache.flink.runtime.state.heap.CopyOnWriteStateMap;
 import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StateMigrationException;
 
 import org.junit.After;
@@ -151,9 +153,11 @@ public abstract class TtlStateTestBase {
 private  StateDescriptor createState() throws 
Exception {
 StateDescriptor stateDescriptor = 
ctx().createStateDescriptor();
 stateDescriptor.enableTimeToLive(ttlConfig);
+String defaultNamespace = "defaultNamespace";
 ctx().ttlState =
 (InternalKvState)
-sbetc.createState(stateDescriptor, "defaultNamespace");
+sbetc.createState(stateDescriptor, defaultNamespace);
+ctx().setCurrentNamespace(defaultNamespace);
 return stateDesc

[flink] branch master updated (0ca3615 -> 0297fb9)

2021-03-18 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 0ca3615  [FLINK-21328] Optimize the initialization of 
DefaultExecutionTopology
 add 0297fb9  [hotfix][docs] Fix invalid link in windows.md

No new revisions were added by this update.

Summary of changes:
 docs/content.zh/docs/dev/datastream/operators/windows.md | 2 +-
 docs/content/docs/dev/datastream/operators/windows.md| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)



[flink] branch master updated (0adc5c2 -> f743974)

2021-03-03 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 0adc5c2  [FLINK-21021][python] Bump Beam to 2.27.0 (#14741)
 new be78727  [FLINK-20496][state backends] Introduce RocksDB metadata 
block size setting.
 new f743974  [FLINK-20496][state backends] RocksDB partitioned 
index/filters option.

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 +++
 .../generated/rocksdb_configuration.html   |  6 +++
 .../generated/state_backend_rocksdb_section.html   |  6 +++
 .../state/DefaultConfigurableOptionsFactory.java   | 24 +
 .../state/RocksDBConfigurableOptions.java  |  9 
 .../state/RocksDBMemoryConfiguration.java  | 19 +++
 .../state/RocksDBMemoryControllerUtils.java|  8 ++-
 .../streaming/state/RocksDBOperationUtils.java |  6 ++-
 .../contrib/streaming/state/RocksDBOptions.java| 14 +
 .../streaming/state/RocksDBResourceContainer.java  | 49 +
 .../streaming/state/RocksDBSharedResources.java| 12 -
 .../state/RocksDBMemoryControllerUtilsTest.java|  5 +-
 .../state/RocksDBResourceContainerTest.java| 62 +-
 .../state/RocksDBStateBackendConfigTest.java   |  5 ++
 14 files changed, 223 insertions(+), 8 deletions(-)



[flink] 02/02: [FLINK-20496][state backends] RocksDB partitioned index/filters option.

2021-03-03 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit f7439740e8e023d458d2ac0cb2a58682eb9b6beb
Author: liuyufei 
AuthorDate: Wed Dec 9 23:39:32 2020 +0800

[FLINK-20496][state backends] RocksDB partitioned index/filters option.

Configure partitioned index and filters options according to 
'https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters'.

This closes #14341.
---
 .../generated/rocksdb_configuration.html   |  6 +++
 .../generated/state_backend_rocksdb_section.html   |  6 +++
 .../state/RocksDBMemoryConfiguration.java  | 19 +++
 .../state/RocksDBMemoryControllerUtils.java|  8 ++-
 .../streaming/state/RocksDBOperationUtils.java |  6 ++-
 .../contrib/streaming/state/RocksDBOptions.java| 14 +
 .../streaming/state/RocksDBResourceContainer.java  | 49 +
 .../streaming/state/RocksDBSharedResources.java| 12 -
 .../state/RocksDBMemoryControllerUtilsTest.java|  5 +-
 .../state/RocksDBResourceContainerTest.java| 62 +-
 10 files changed, 179 insertions(+), 8 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/rocksdb_configuration.html 
b/docs/layouts/shortcodes/generated/rocksdb_configuration.html
index 79b769d..74b1236 100644
--- a/docs/layouts/shortcodes/generated/rocksdb_configuration.html
+++ b/docs/layouts/shortcodes/generated/rocksdb_configuration.html
@@ -39,6 +39,12 @@
 If set, the RocksDB state backend will automatically configure 
itself to use the managed memory budget of the task slot, and divide the memory 
over write buffers, indexes, block caches, etc. That way, the three major uses 
of memory of RocksDB will be capped.
 
 
+
state.backend.rocksdb.memory.partitioned-index-filters
+false
+Boolean
+With partitioning, the index/filter block of an SST file is 
partitioned into smaller blocks with an additional top-level index on them. 
When reading an index/filter, only top-level index is loaded into memory. The 
partitioned index/filter then uses the top-level index to load on demand into 
the block cache the partitions that are required to perform the index/filter 
query. This option only has an effect when 
'state.backend.rocksdb.memory.managed' or 'state.backend.rocksdb [...]
+
+
 state.backend.rocksdb.memory.write-buffer-ratio
 0.5
 Double
diff --git 
a/docs/layouts/shortcodes/generated/state_backend_rocksdb_section.html 
b/docs/layouts/shortcodes/generated/state_backend_rocksdb_section.html
index 8a74eee..a8ed2c7 100644
--- a/docs/layouts/shortcodes/generated/state_backend_rocksdb_section.html
+++ b/docs/layouts/shortcodes/generated/state_backend_rocksdb_section.html
@@ -27,6 +27,12 @@
 If set, the RocksDB state backend will automatically configure 
itself to use the managed memory budget of the task slot, and divide the memory 
over write buffers, indexes, block caches, etc. That way, the three major uses 
of memory of RocksDB will be capped.
 
 
+
state.backend.rocksdb.memory.partitioned-index-filters
+false
+Boolean
+With partitioning, the index/filter block of an SST file is 
partitioned into smaller blocks with an additional top-level index on them. 
When reading an index/filter, only top-level index is loaded into memory. The 
partitioned index/filter then uses the top-level index to load on demand into 
the block cache the partitions that are required to perform the index/filter 
query. This option only has an effect when 
'state.backend.rocksdb.memory.managed' or 'state.backend.rocksdb [...]
+
+
 state.backend.rocksdb.memory.write-buffer-ratio
 0.5
 Double
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java
index dda5409..3bfcc03 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java
@@ -51,6 +51,9 @@ public final class RocksDBMemoryConfiguration implements 
Serializable {
  */
 @Nullable private Double highPriorityPoolRatio;
 
+/** Flag whether to use partition index/filters. Null if not set. */
+@Nullable private Boolean usePartitionedIndexFilters;
+
 // 
 
 /**
@@ -166,6 +169,17 @@ public final class

[flink] 01/02: [FLINK-20496][state backends] Introduce RocksDB metadata block size setting.

2021-03-03 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit be78727ae125c036cbb297d020e8a7ad23aae083
Author: liuyufei 
AuthorDate: Wed Dec 9 23:38:49 2020 +0800

[FLINK-20496][state backends] Introduce RocksDB metadata block size setting.
---
 .../rocksdb_configurable_configuration.html|  6 ++
 .../state/DefaultConfigurableOptionsFactory.java   | 24 ++
 .../state/RocksDBConfigurableOptions.java  |  9 
 .../state/RocksDBStateBackendConfigTest.java   |  5 +
 4 files changed, 44 insertions(+)

diff --git 
a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html 
b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
index 29142bb..a55c319 100644
--- a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
+++ b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
@@ -21,6 +21,12 @@
 The amount of the cache for data blocks in RocksDB. RocksDB 
has default block-cache size as '8MB'.
 
 
+state.backend.rocksdb.block.metadata-blocksize
+(none)
+MemorySize
+Approximate size of partitioned metadata packed per block. 
Currently applied to indexes block when partitioned index/filters option is 
enabled. RocksDB has default metadata blocksize as '4KB'.
+
+
 
state.backend.rocksdb.compaction.level.max-size-level-base
 (none)
 MemorySize
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java
index 94ae969..f5dd310 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java
@@ -47,6 +47,7 @@ import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOption
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.MAX_OPEN_FILES;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.MAX_SIZE_LEVEL_BASE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.MAX_WRITE_BUFFER_NUMBER;
+import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.METADATA_BLOCK_SIZE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.MIN_WRITE_BUFFER_NUMBER_TO_MERGE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.TARGET_FILE_SIZE_BASE;
 import static 
org.apache.flink.contrib.streaming.state.RocksDBConfigurableOptions.USE_DYNAMIC_LEVEL_SIZE;
@@ -131,6 +132,10 @@ public class DefaultConfigurableOptionsFactory implements 
ConfigurableRocksDBOpt
 blockBasedTableConfig.setBlockSize(getBlockSize());
 }
 
+if (isOptionConfigured(METADATA_BLOCK_SIZE)) {
+blockBasedTableConfig.setMetadataBlockSize(getMetadataBlockSize());
+}
+
 if (isOptionConfigured(BLOCK_CACHE_SIZE)) {
 blockBasedTableConfig.setBlockCacheSize(getBlockCacheSize());
 }
@@ -316,6 +321,23 @@ public class DefaultConfigurableOptionsFactory implements 
ConfigurableRocksDBOpt
 }
 
 // 
--
+// Approximate size of partitioned metadata packed per block.
+// Currently applied to indexes block when partitioned index/filters 
option is enabled.
+// 
--
+
+private long getMetadataBlockSize() {
+return MemorySize.parseBytes(getInternal(METADATA_BLOCK_SIZE.key()));
+}
+
+public DefaultConfigurableOptionsFactory setMetadataBlockSize(String 
metadataBlockSize) {
+Preconditions.checkArgument(
+MemorySize.parseBytes(metadataBlockSize) > 0,
+"Invalid configuration " + metadataBlockSize + " for metadata 
block size.");
+setInternal(METADATA_BLOCK_SIZE.key(), metadataBlockSize);
+return this;
+}
+
+// 
--
 // The amount of the cache for data blocks in RocksDB
 // 
--
 
@@ -348,6 +370,7 @@ public class DefaultConfigurableOptionsFactory implements 
ConfigurableRocksDBOpt
 MAX_WRITE_BUFFER_NUMBER,
 MIN_WRITE_BUFFER_NUMBER_TO_MERGE,

[flink] branch release-1.12 updated: [FLINK-20287][docs] Add documentation of how to switch memory allocator in Flink docker image

2020-11-30 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/release-1.12 by this push:
 new e1c0220  [FLINK-20287][docs] Add documentation of how to switch memory 
allocator in Flink docker image
e1c0220 is described below

commit e1c0220984a91091618ccb653cee7b007077a2eb
Author: Yun Tang 
AuthorDate: Fri Nov 27 18:16:32 2020 +0800

[FLINK-20287][docs] Add documentation of how to switch memory allocator in 
Flink docker image

This closes #14248.
---
 docs/deployment/resource-providers/standalone/docker.md| 11 +++
 docs/deployment/resource-providers/standalone/docker.zh.md | 11 +++
 2 files changed, 22 insertions(+)

diff --git a/docs/deployment/resource-providers/standalone/docker.md 
b/docs/deployment/resource-providers/standalone/docker.md
index 8ee9d4d..30e9512 100644
--- a/docs/deployment/resource-providers/standalone/docker.md
+++ b/docs/deployment/resource-providers/standalone/docker.md
@@ -268,6 +268,17 @@ The `ENABLE_BUILT_IN_PLUGINS` should contain a list of 
plugin jar file names sep
 
 There are also more [advanced ways](#advanced-customization) for customizing 
the Flink image.
 
+### Switch memory allocator
+
+Flink introduced `jemalloc` as default memory allocator to resolve memory 
fragmentation problem (please refer to 
[FLINK-19125](https://issues.apache.org/jira/browse/FLINK-19125)).
+
+You could switch back to use `glibc` as memory allocator to restore the old 
behavior or if any unexpected memory consumption or problem observed
+(and please report the issue via JIRA or mailing list if you found any), by 
passing `disable-jemalloc` parameter:
+
+```sh
+docker run  disable-jemalloc
+```
+
 ### Advanced customization
 
 There are several ways in which you can further customize the Flink image:
diff --git a/docs/deployment/resource-providers/standalone/docker.zh.md 
b/docs/deployment/resource-providers/standalone/docker.zh.md
index 91eab55..c1e2b71 100644
--- a/docs/deployment/resource-providers/standalone/docker.zh.md
+++ b/docs/deployment/resource-providers/standalone/docker.zh.md
@@ -268,6 +268,17 @@ The `ENABLE_BUILT_IN_PLUGINS` should contain a list of 
plugin jar file names sep
 
 There are also more [advanced ways](#advanced-customization) for customizing 
the Flink image.
 
+### Switch memory allocator
+
+Flink introduced `jemalloc` as default memory allocator to resolve memory 
fragmentation problem (please refer to 
[FLINK-19125](https://issues.apache.org/jira/browse/FLINK-19125)).
+
+You could switch back to use `glibc` as memory allocator to restore the old 
behavior or if any unexpected memory consumption or problem observed
+(and please report the issue via JIRA or mailing list if you found any), by 
passing `disable-jemalloc` parameter:
+
+```sh
+docker run  disable-jemalloc
+```
+
 ### Advanced customization
 
 There are several ways in which you can further customize the Flink image:



[flink] branch master updated (6f46458 -> b457225)

2020-11-30 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 6f46458  [FLINK-20410] Retry querying for schema in the schema 
registry e2e test.
 add b457225  [FLINK-20287][docs] Add documentation of how to switch memory 
allocator in Flink docker image

No new revisions were added by this update.

Summary of changes:
 docs/deployment/resource-providers/standalone/docker.md| 11 +++
 docs/deployment/resource-providers/standalone/docker.zh.md | 11 +++
 2 files changed, 22 insertions(+)



[flink] branch master updated (2cce1ac -> 93c6256)

2020-11-08 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 2cce1ac  [FLINK-19717][connectors/common] Fix spurious 
InputStatus.END_OF_INPUT from SourceReaderBase.pollNext caused by split reader 
exception (#13776)
 add 93c6256  [FLINK-19238][state-backend-rocksdb] Sanity check for RocksDB 
arena block size

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/RocksDBKeyedStateBackend.java  |  3 +-
 .../state/RocksDBKeyedStateBackendBuilder.java | 16 ---
 .../state/RocksDBMemoryControllerUtils.java| 51 -
 .../streaming/state/RocksDBOperationUtils.java | 52 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  7 ++-
 .../streaming/state/RocksDBResourceContainer.java  | 13 ++
 .../streaming/state/RocksDBSharedResources.java|  8 +++-
 .../streaming/state/RocksDBStateBackend.java   |  3 +-
 .../restore/AbstractRocksDBRestoreOperation.java   |  7 ++-
 .../state/restore/RocksDBFullRestoreOperation.java |  6 ++-
 .../RocksDBIncrementalRestoreOperation.java| 17 ---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +--
 .../state/RocksDBMemoryControllerUtilsTest.java| 37 ++-
 .../state/RocksDBOperationsUtilsTest.java  | 20 +
 .../state/RocksDBResourceContainerTest.java|  4 +-
 15 files changed, 216 insertions(+), 35 deletions(-)



[flink] branch master updated (2cce1ac -> 93c6256)

2020-11-08 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 2cce1ac  [FLINK-19717][connectors/common] Fix spurious 
InputStatus.END_OF_INPUT from SourceReaderBase.pollNext caused by split reader 
exception (#13776)
 add 93c6256  [FLINK-19238][state-backend-rocksdb] Sanity check for RocksDB 
arena block size

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/RocksDBKeyedStateBackend.java  |  3 +-
 .../state/RocksDBKeyedStateBackendBuilder.java | 16 ---
 .../state/RocksDBMemoryControllerUtils.java| 51 -
 .../streaming/state/RocksDBOperationUtils.java | 52 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  7 ++-
 .../streaming/state/RocksDBResourceContainer.java  | 13 ++
 .../streaming/state/RocksDBSharedResources.java|  8 +++-
 .../streaming/state/RocksDBStateBackend.java   |  3 +-
 .../restore/AbstractRocksDBRestoreOperation.java   |  7 ++-
 .../state/restore/RocksDBFullRestoreOperation.java |  6 ++-
 .../RocksDBIncrementalRestoreOperation.java| 17 ---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +--
 .../state/RocksDBMemoryControllerUtilsTest.java| 37 ++-
 .../state/RocksDBOperationsUtilsTest.java  | 20 +
 .../state/RocksDBResourceContainerTest.java|  4 +-
 15 files changed, 216 insertions(+), 35 deletions(-)



[flink] branch master updated (2cce1ac -> 93c6256)

2020-11-08 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 2cce1ac  [FLINK-19717][connectors/common] Fix spurious 
InputStatus.END_OF_INPUT from SourceReaderBase.pollNext caused by split reader 
exception (#13776)
 add 93c6256  [FLINK-19238][state-backend-rocksdb] Sanity check for RocksDB 
arena block size

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/RocksDBKeyedStateBackend.java  |  3 +-
 .../state/RocksDBKeyedStateBackendBuilder.java | 16 ---
 .../state/RocksDBMemoryControllerUtils.java| 51 -
 .../streaming/state/RocksDBOperationUtils.java | 52 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  7 ++-
 .../streaming/state/RocksDBResourceContainer.java  | 13 ++
 .../streaming/state/RocksDBSharedResources.java|  8 +++-
 .../streaming/state/RocksDBStateBackend.java   |  3 +-
 .../restore/AbstractRocksDBRestoreOperation.java   |  7 ++-
 .../state/restore/RocksDBFullRestoreOperation.java |  6 ++-
 .../RocksDBIncrementalRestoreOperation.java| 17 ---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +--
 .../state/RocksDBMemoryControllerUtilsTest.java| 37 ++-
 .../state/RocksDBOperationsUtilsTest.java  | 20 +
 .../state/RocksDBResourceContainerTest.java|  4 +-
 15 files changed, 216 insertions(+), 35 deletions(-)



[flink] branch master updated (2cce1ac -> 93c6256)

2020-11-08 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 2cce1ac  [FLINK-19717][connectors/common] Fix spurious 
InputStatus.END_OF_INPUT from SourceReaderBase.pollNext caused by split reader 
exception (#13776)
 add 93c6256  [FLINK-19238][state-backend-rocksdb] Sanity check for RocksDB 
arena block size

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/RocksDBKeyedStateBackend.java  |  3 +-
 .../state/RocksDBKeyedStateBackendBuilder.java | 16 ---
 .../state/RocksDBMemoryControllerUtils.java| 51 -
 .../streaming/state/RocksDBOperationUtils.java | 52 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  7 ++-
 .../streaming/state/RocksDBResourceContainer.java  | 13 ++
 .../streaming/state/RocksDBSharedResources.java|  8 +++-
 .../streaming/state/RocksDBStateBackend.java   |  3 +-
 .../restore/AbstractRocksDBRestoreOperation.java   |  7 ++-
 .../state/restore/RocksDBFullRestoreOperation.java |  6 ++-
 .../RocksDBIncrementalRestoreOperation.java| 17 ---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +--
 .../state/RocksDBMemoryControllerUtilsTest.java| 37 ++-
 .../state/RocksDBOperationsUtilsTest.java  | 20 +
 .../state/RocksDBResourceContainerTest.java|  4 +-
 15 files changed, 216 insertions(+), 35 deletions(-)



[flink] branch master updated (2cce1ac -> 93c6256)

2020-11-08 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 2cce1ac  [FLINK-19717][connectors/common] Fix spurious 
InputStatus.END_OF_INPUT from SourceReaderBase.pollNext caused by split reader 
exception (#13776)
 add 93c6256  [FLINK-19238][state-backend-rocksdb] Sanity check for RocksDB 
arena block size

No new revisions were added by this update.

Summary of changes:
 .../streaming/state/RocksDBKeyedStateBackend.java  |  3 +-
 .../state/RocksDBKeyedStateBackendBuilder.java | 16 ---
 .../state/RocksDBMemoryControllerUtils.java| 51 -
 .../streaming/state/RocksDBOperationUtils.java | 52 --
 .../state/RocksDBPriorityQueueSetFactory.java  |  7 ++-
 .../streaming/state/RocksDBResourceContainer.java  | 13 ++
 .../streaming/state/RocksDBSharedResources.java|  8 +++-
 .../streaming/state/RocksDBStateBackend.java   |  3 +-
 .../restore/AbstractRocksDBRestoreOperation.java   |  7 ++-
 .../state/restore/RocksDBFullRestoreOperation.java |  6 ++-
 .../RocksDBIncrementalRestoreOperation.java| 17 ---
 .../state/restore/RocksDBNoneRestoreOperation.java |  7 +--
 .../state/RocksDBMemoryControllerUtilsTest.java| 37 ++-
 .../state/RocksDBOperationsUtilsTest.java  | 20 +
 .../state/RocksDBResourceContainerTest.java|  4 +-
 15 files changed, 216 insertions(+), 35 deletions(-)



[flink-web] branch asf-site updated: Update status for Yu to PMC in community web page

2020-10-26 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 0f16e95  Update status for Yu to PMC in community web page
0f16e95 is described below

commit 0f16e95069373a04d8f3b62bf28fe5267467e03b
Author: Yu Li 
AuthorDate: Tue Oct 27 11:20:43 2020 +0800

Update status for Yu to PMC in community web page
---
 community.md  | 2 +-
 community.zh.md   | 2 +-
 content/community.html| 2 +-
 content/zh/community.html | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/community.md b/community.md
index ab6a439..230f480 100644
--- a/community.md
+++ b/community.md
@@ -510,7 +510,7 @@ Flink Forward is a conference happening yearly in different 
locations around the
   
 https://avatars0.githubusercontent.com/u/6239804?s=50; 
class="committer-avatar">
 Yu Li
-Committer
+PMC, Committer
 liyu
   
   
diff --git a/community.zh.md b/community.zh.md
index fd903d7..c3be762 100644
--- a/community.zh.md
+++ b/community.zh.md
@@ -499,7 +499,7 @@ Flink Forward 大会每年都会在世界的不同地方举办。关于大会最
   
 https://avatars0.githubusercontent.com/u/6239804?s=50; 
class="committer-avatar">
 Yu Li
-Committer
+    PMC, Committer
 liyu
   
   
diff --git a/content/community.html b/content/community.html
index 6bd7d8e..659f2fd 100644
--- a/content/community.html
+++ b/content/community.html
@@ -736,7 +736,7 @@
   
 https://avatars0.githubusercontent.com/u/6239804?s=50; 
class="committer-avatar" />
 Yu Li
-Committer
+PMC, Committer
 liyu
   
   
diff --git a/content/zh/community.html b/content/zh/community.html
index d8327eb..6c67b1d 100644
--- a/content/zh/community.html
+++ b/content/zh/community.html
@@ -727,7 +727,7 @@
   
 https://avatars0.githubusercontent.com/u/6239804?s=50; 
class="committer-avatar" />
     Yu Li
-Committer
+PMC, Committer
 liyu
   
   



svn commit: r40622 - /release/flink/flink-1.11.0/

2020-07-21 Thread liyu
Author: liyu
Date: Wed Jul 22 03:35:51 2020
New Revision: 40622

Log:
Remove Flink 1.11.0

Removed:
release/flink/flink-1.11.0/



svn commit: r40597 - /dev/flink/flink-1.11.1-rc1/ /release/flink/flink-1.11.1/

2020-07-20 Thread liyu
Author: liyu
Date: Tue Jul 21 05:50:27 2020
New Revision: 40597

Log:
Release Flink 1.11.1

Added:
release/flink/flink-1.11.1/
  - copied from r40596, dev/flink/flink-1.11.1-rc1/
Removed:
dev/flink/flink-1.11.1-rc1/



svn commit: r40537 - /dev/flink/flink-1.11.1-rc1/flink-1.11.1-src.tgz.asc

2020-07-17 Thread liyu
Author: liyu
Date: Fri Jul 17 13:43:17 2020
New Revision: 40537

Log:
Update flink-1.11.1-src.tgz.asc

Modified:
dev/flink/flink-1.11.1-rc1/flink-1.11.1-src.tgz.asc

Modified: dev/flink/flink-1.11.1-rc1/flink-1.11.1-src.tgz.asc
==
Binary files - no diff available.




svn commit: r40484 - in /dev/flink/flink-1.11.1-rc1: ./ python/

2020-07-15 Thread liyu
Author: liyu
Date: Wed Jul 15 09:52:22 2020
New Revision: 40484

Log:
Add flink-1.11.1-rc1

Added:
dev/flink/flink-1.11.1-rc1/
dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.11.1-rc1/flink-1.11.1-src.tgz   (with props)
dev/flink/flink-1.11.1-rc1/flink-1.11.1-src.tgz.asc   (with props)
dev/flink/flink-1.11.1-rc1/flink-1.11.1-src.tgz.sha512
dev/flink/flink-1.11.1-rc1/python/
dev/flink/flink-1.11.1-rc1/python/apache-flink-1.11.1.tar.gz   (with props)
dev/flink/flink-1.11.1-rc1/python/apache-flink-1.11.1.tar.gz.asc   (with 
props)
dev/flink/flink-1.11.1-rc1/python/apache-flink-1.11.1.tar.gz.sha512

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp35-cp35m-macosx_10_6_x86_64.whl
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp35-cp35m-macosx_10_6_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp35-cp35m-macosx_10_6_x86_64.whl.sha512

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp35-cp35m-manylinux1_x86_64.whl
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp35-cp35m-manylinux1_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp35-cp35m-manylinux1_x86_64.whl.sha512

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp36-cp36m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp36-cp36m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp36-cp36m-macosx_10_9_x86_64.whl.sha512

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp36-cp36m-manylinux1_x86_64.whl
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp36-cp36m-manylinux1_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp36-cp36m-manylinux1_x86_64.whl.sha512

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp37-cp37m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp37-cp37m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp37-cp37m-macosx_10_9_x86_64.whl.sha512

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp37-cp37m-manylinux1_x86_64.whl
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp37-cp37m-manylinux1_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.1-rc1/python/apache_flink-1.11.1-cp37-cp37m-manylinux1_x86_64.whl.sha512

Added: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.11.tgz.sha512 Wed Jul 
15 09:52:22 2020
@@ -0,0 +1 @@
+638cfa0da3168175169256fcc11a739826dad2019ffa318ad3a3a8d81c2bae525fa2b62439108de54e7b013a549aa149d328653947e6b174141489de1eac7be0
  flink-1.11.1-bin-scala_2.11.tgz

Added: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.1-rc1/flink-1.11.1-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp

svn commit: r40331 - /release/flink/flink-1.9.3/

2020-07-06 Thread liyu
Author: liyu
Date: Tue Jul  7 05:41:32 2020
New Revision: 40331

Log:
remove outdated version flink-1.9.3

Removed:
release/flink/flink-1.9.3/



svn commit: r40319 - in /dev/flink: flink-1.11.0-rc1/ flink-1.11.0-rc2/ flink-1.11.0-rc3/

2020-07-06 Thread liyu
Author: liyu
Date: Mon Jul  6 10:33:39 2020
New Revision: 40319

Log:
Remove old release candidates for Apache Flink 1.11.0

Removed:
dev/flink/flink-1.11.0-rc1/
dev/flink/flink-1.11.0-rc2/
dev/flink/flink-1.11.0-rc3/



svn commit: r40318 - /dev/flink/flink-1.11.0-rc4/ /release/flink/flink-1.11.0/

2020-07-06 Thread liyu
Author: liyu
Date: Mon Jul  6 09:46:57 2020
New Revision: 40318

Log:
Release Flink 1.11.0

Added:
release/flink/flink-1.11.0/
  - copied from r40317, dev/flink/flink-1.11.0-rc4/
Removed:
dev/flink/flink-1.11.0-rc4/



svn commit: r40232 - in /dev/flink/flink-1.11.0-rc4: ./ python/

2020-06-30 Thread liyu
Author: liyu
Date: Tue Jun 30 10:09:29 2020
New Revision: 40232

Log:
release-1.11.0-rc4

Added:
dev/flink/flink-1.11.0-rc4/
dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.11.0-rc4/flink-1.11.0-src.tgz   (with props)
dev/flink/flink-1.11.0-rc4/flink-1.11.0-src.tgz.asc   (with props)
dev/flink/flink-1.11.0-rc4/flink-1.11.0-src.tgz.sha512
dev/flink/flink-1.11.0-rc4/python/
dev/flink/flink-1.11.0-rc4/python/apache-flink-1.11.0.tar.gz   (with props)
dev/flink/flink-1.11.0-rc4/python/apache-flink-1.11.0.tar.gz.asc   (with 
props)
dev/flink/flink-1.11.0-rc4/python/apache-flink-1.11.0.tar.gz.sha512

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc4/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.sha512

Added: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.11.tgz.sha512 Tue Jun 
30 10:09:29 2020
@@ -0,0 +1 @@
+21ef9c6f5b4fcdc38a8913fc22dc67b7d64c2fdfd37cd3f51ae471a427ae04845b0c44bb47ba0a983032d1a9713020ffaea4db07580c60f00fe6ef3aba78a13b
  flink-1.11.0-bin-scala_2.11.tgz

Added: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc4/flink-1.11.0-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc4/flink

[flink] 03/03: [FLINK-17800][rocksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

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

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

commit 9b45486ccce31ebef3f91dd4e6102efe3c6d51a3
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][rocksdb] Support customized RocksDB write/read options and 
use RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 505bd2a..591e8b7 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 08b4864..94e0d5b 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -249,9 +247,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -289,9 +284,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] 02/03: [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at RocksDBStateMisuseOptionTest

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

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

commit 3f8649e5c7bf731fac3cc5bfd3c5ed466f1dc561
Author: Yun Tang 
AuthorDate: Mon Jun 22 16:27:02 2020 +0800

[hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at 
RocksDBStateMisuseOptionTest

This closes #12736.
---
 .../state/RocksDBStateMisuseOptionTest.java| 86 --
 1 file changed, 48 insertions(+), 38 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
index 59a4822..20e2906 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
@@ -71,26 +71,31 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithMapState() throws 
Exception {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
-   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
-
-   keyedStateBackend.setCurrentKey(1);
-   Map expectedResult = new HashMap<>();
-   for (int i = 0; i < 100; i++) {
-   long uv = ThreadLocalRandom.current().nextLong();
-   mapState.put(i, uv);
-   expectedResult.put(i, uv);
-   }
+   RocksDBKeyedStateBackend keyedStateBackend =
+   createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
+   try {
+   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
+   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
+
+   keyedStateBackend.setCurrentKey(1);
+   Map expectedResult = new HashMap<>();
+   for (int i = 0; i < 100; i++) {
+   long uv = 
ThreadLocalRandom.current().nextLong();
+   mapState.put(i, uv);
+   expectedResult.put(i, uv);
+   }
 
-   Iterator> iterator = 
mapState.entries().iterator();
-   while (iterator.hasNext()) {
-   Map.Entry entry = iterator.next();
-   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
-   iterator.remove();
+   Iterator> iterator = 
mapState.entries().iterator();
+   while (iterator.hasNext()) {
+   Map.Entry entry = 
iterator.next();
+   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
+   iterator.remove();
+   }
+   assertTrue(expectedResult.isEmpty());
+   assertTrue(mapState.isEmpty());
+   } finally {
+   keyedStateBackend.dispose();
}
-   assertTrue(expectedResult.isEmpty());
-   assertTrue(mapState.isEmpty());
}
 
/**
@@ -101,27 +106,32 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithPriorityQueue() throws 
IOException {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   KeyGroupedInternalPriorityQueue> priorityQueue =
-   keyedStateBackend.create("timer", new 
TimerSerializer<>(keyedStateBackend.getKeySerializer(), 
VoidNamespaceSerializer.INSTANCE));
-
-   PriorityQueue> 
expectedPriorityQueue

[flink] branch release-1.10 updated (23ad7e3 -> 9b45486)

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

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


from 23ad7e3  [FLINK-18168][table-runtime-blink] Fix array reuse for 
BinaryArrayData in converters
 new de6f3aa  [FLINK-17800][rocksdb] Ensure total order seek to avoid user 
misuse
 new 3f8649e  [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed 
at RocksDBStateMisuseOptionTest
 new 9b45486  [FLINK-17800][rocksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 157 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 20 files changed, 338 insertions(+), 54 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 01/03: [FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse

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

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

commit de6f3aa7e5b2e4fcfbed4adeab12d4d519f1e6fb
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 17 files changed, 256 insertions(+), 47 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class

[flink] 02/03: [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at RocksDBStateMisuseOptionTest

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

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

commit 33caa00e8df88565f022d4258148d09c90d9452b
Author: Yun Tang 
AuthorDate: Mon Jun 22 16:27:02 2020 +0800

[hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at 
RocksDBStateMisuseOptionTest

This closes #12736.
---
 .../state/RocksDBStateMisuseOptionTest.java| 86 --
 1 file changed, 48 insertions(+), 38 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
index 59a4822..20e2906 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
@@ -71,26 +71,31 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithMapState() throws 
Exception {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
-   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
-
-   keyedStateBackend.setCurrentKey(1);
-   Map expectedResult = new HashMap<>();
-   for (int i = 0; i < 100; i++) {
-   long uv = ThreadLocalRandom.current().nextLong();
-   mapState.put(i, uv);
-   expectedResult.put(i, uv);
-   }
+   RocksDBKeyedStateBackend keyedStateBackend =
+   createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
+   try {
+   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
+   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
+
+   keyedStateBackend.setCurrentKey(1);
+   Map expectedResult = new HashMap<>();
+   for (int i = 0; i < 100; i++) {
+   long uv = 
ThreadLocalRandom.current().nextLong();
+   mapState.put(i, uv);
+   expectedResult.put(i, uv);
+   }
 
-   Iterator> iterator = 
mapState.entries().iterator();
-   while (iterator.hasNext()) {
-   Map.Entry entry = iterator.next();
-   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
-   iterator.remove();
+   Iterator> iterator = 
mapState.entries().iterator();
+   while (iterator.hasNext()) {
+   Map.Entry entry = 
iterator.next();
+   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
+   iterator.remove();
+   }
+   assertTrue(expectedResult.isEmpty());
+   assertTrue(mapState.isEmpty());
+   } finally {
+   keyedStateBackend.dispose();
}
-   assertTrue(expectedResult.isEmpty());
-   assertTrue(mapState.isEmpty());
}
 
/**
@@ -101,27 +106,32 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithPriorityQueue() throws 
IOException {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   KeyGroupedInternalPriorityQueue> priorityQueue =
-   keyedStateBackend.create("timer", new 
TimerSerializer<>(keyedStateBackend.getKeySerializer(), 
VoidNamespaceSerializer.INSTANCE));
-
-   PriorityQueue> 
expectedPriorityQueue

[flink] branch release-1.11 updated (6ecf2d3 -> 7e1c83d)

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

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


from 6ecf2d3  [FLINK-18349][docs] Add release notes for Flink 1.11
 new b2e344a  [FLINK-17800][rocksdb] Ensure total order seek to avoid user 
misuse
 new 33caa00  [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed 
at RocksDBStateMisuseOptionTest
 new 7e1c83d  [FLINK-17800][rocksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 157 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 337 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 03/03: [FLINK-17800][rocksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

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

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

commit 7e1c83ddcf0e5e4417ccf25fd1d0facce9f30e0e
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][rocksdb] Support customized RocksDB write/read options and 
use RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 f0cce0b..24b897a 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 b9bdbb8..1846bde 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -243,9 +241,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -283,9 +278,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] branch release-1.11 updated (6ecf2d3 -> 7e1c83d)

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

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


from 6ecf2d3  [FLINK-18349][docs] Add release notes for Flink 1.11
 new b2e344a  [FLINK-17800][rocksdb] Ensure total order seek to avoid user 
misuse
 new 33caa00  [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed 
at RocksDBStateMisuseOptionTest
 new 7e1c83d  [FLINK-17800][rocksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 157 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 337 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 03/03: [FLINK-17800][rocksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

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

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

commit 7e1c83ddcf0e5e4417ccf25fd1d0facce9f30e0e
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][rocksdb] Support customized RocksDB write/read options and 
use RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 f0cce0b..24b897a 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 b9bdbb8..1846bde 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -243,9 +241,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -283,9 +278,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] 01/03: [FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse

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

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

commit b2e344a46c5d30ad46231d5c6a42bf09d9e8e559
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 16 files changed, 255 insertions(+), 46 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class RocksDBIncrementalCheckpointUtils {
@Nonnegative long

[flink] 02/03: [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at RocksDBStateMisuseOptionTest

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

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

commit 33caa00e8df88565f022d4258148d09c90d9452b
Author: Yun Tang 
AuthorDate: Mon Jun 22 16:27:02 2020 +0800

[hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at 
RocksDBStateMisuseOptionTest

This closes #12736.
---
 .../state/RocksDBStateMisuseOptionTest.java| 86 --
 1 file changed, 48 insertions(+), 38 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
index 59a4822..20e2906 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
@@ -71,26 +71,31 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithMapState() throws 
Exception {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
-   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
-
-   keyedStateBackend.setCurrentKey(1);
-   Map expectedResult = new HashMap<>();
-   for (int i = 0; i < 100; i++) {
-   long uv = ThreadLocalRandom.current().nextLong();
-   mapState.put(i, uv);
-   expectedResult.put(i, uv);
-   }
+   RocksDBKeyedStateBackend keyedStateBackend =
+   createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
+   try {
+   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
+   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
+
+   keyedStateBackend.setCurrentKey(1);
+   Map expectedResult = new HashMap<>();
+   for (int i = 0; i < 100; i++) {
+   long uv = 
ThreadLocalRandom.current().nextLong();
+   mapState.put(i, uv);
+   expectedResult.put(i, uv);
+   }
 
-   Iterator> iterator = 
mapState.entries().iterator();
-   while (iterator.hasNext()) {
-   Map.Entry entry = iterator.next();
-   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
-   iterator.remove();
+   Iterator> iterator = 
mapState.entries().iterator();
+   while (iterator.hasNext()) {
+   Map.Entry entry = 
iterator.next();
+   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
+   iterator.remove();
+   }
+   assertTrue(expectedResult.isEmpty());
+   assertTrue(mapState.isEmpty());
+   } finally {
+   keyedStateBackend.dispose();
}
-   assertTrue(expectedResult.isEmpty());
-   assertTrue(mapState.isEmpty());
}
 
/**
@@ -101,27 +106,32 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithPriorityQueue() throws 
IOException {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   KeyGroupedInternalPriorityQueue> priorityQueue =
-   keyedStateBackend.create("timer", new 
TimerSerializer<>(keyedStateBackend.getKeySerializer(), 
VoidNamespaceSerializer.INSTANCE));
-
-   PriorityQueue> 
expectedPriorityQueue

[flink] 01/03: [FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse

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

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

commit b2e344a46c5d30ad46231d5c6a42bf09d9e8e559
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 16 files changed, 255 insertions(+), 46 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class RocksDBIncrementalCheckpointUtils {
@Nonnegative long

[flink] branch master updated (6227fff -> 1718f50)

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

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


from 6227fff  fixup! [FLINK-18349][docs] Add release notes for Flink 1.11
 add 3516e37  [FLINK-17800][rocksdb] Ensure total order seek to avoid user 
misuse
 add 11d4513  [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed 
at RocksDBStateMisuseOptionTest
 add 1718f50  [FLINK-17800][rocksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

No new revisions were added by this update.

Summary of changes:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 157 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 337 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 03/03: [FLINK-17800][rocksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

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

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

commit 1718f50645ddc01d5e2e13cc5627bafe98191fa2
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][rocksdb] Support customized RocksDB write/read options and 
use RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 f0cce0b..24b897a 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 b9bdbb8..1846bde 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -243,9 +241,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -283,9 +278,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writeBatchWrapper = new Rocks

[flink] 01/03: [FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse

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

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

commit 3516e37ae0aa4ee040b6844f336541315a455ce9
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][rocksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 16 files changed, 255 insertions(+), 46 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class RocksDBIncrementalCheckpointUtils {
@Nonnegative long

[flink] branch master updated (6227fff -> 1718f50)

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

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


from 6227fff  fixup! [FLINK-18349][docs] Add release notes for Flink 1.11
 new 3516e37  [FLINK-17800][rocksdb] Ensure total order seek to avoid user 
misuse
 new 11d4513  [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed 
at RocksDBStateMisuseOptionTest
 new 1718f50  [FLINK-17800][rocksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 157 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 337 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 02/03: [hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at RocksDBStateMisuseOptionTest

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

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

commit 11d45135d85937edd16fb4f8f94ba71f5f794626
Author: Yun Tang 
AuthorDate: Mon Jun 22 16:27:02 2020 +0800

[hot-fix][rocksdb] Ensure RocksDBKeyedStateBackend disposed at 
RocksDBStateMisuseOptionTest

This closes #12736.
---
 .../state/RocksDBStateMisuseOptionTest.java| 86 --
 1 file changed, 48 insertions(+), 38 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
index 59a4822..20e2906 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java
@@ -71,26 +71,31 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithMapState() throws 
Exception {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
-   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
-
-   keyedStateBackend.setCurrentKey(1);
-   Map expectedResult = new HashMap<>();
-   for (int i = 0; i < 100; i++) {
-   long uv = ThreadLocalRandom.current().nextLong();
-   mapState.put(i, uv);
-   expectedResult.put(i, uv);
-   }
+   RocksDBKeyedStateBackend keyedStateBackend =
+   createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
+   try {
+   MapStateDescriptor stateDescriptor = new 
MapStateDescriptor<>("map", IntSerializer.INSTANCE, LongSerializer.INSTANCE);
+   MapState mapState = 
keyedStateBackend.getPartitionedState(VoidNamespace.INSTANCE, 
VoidNamespaceSerializer.INSTANCE, stateDescriptor);
+
+   keyedStateBackend.setCurrentKey(1);
+   Map expectedResult = new HashMap<>();
+   for (int i = 0; i < 100; i++) {
+   long uv = 
ThreadLocalRandom.current().nextLong();
+   mapState.put(i, uv);
+   expectedResult.put(i, uv);
+   }
 
-   Iterator> iterator = 
mapState.entries().iterator();
-   while (iterator.hasNext()) {
-   Map.Entry entry = iterator.next();
-   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
-   iterator.remove();
+   Iterator> iterator = 
mapState.entries().iterator();
+   while (iterator.hasNext()) {
+   Map.Entry entry = 
iterator.next();
+   assertEquals(entry.getValue(), 
expectedResult.remove(entry.getKey()));
+   iterator.remove();
+   }
+   assertTrue(expectedResult.isEmpty());
+   assertTrue(mapState.isEmpty());
+   } finally {
+   keyedStateBackend.dispose();
}
-   assertTrue(expectedResult.isEmpty());
-   assertTrue(mapState.isEmpty());
}
 
/**
@@ -101,27 +106,32 @@ public class RocksDBStateMisuseOptionTest {
@Test
public void testMisuseOptimizePointLookupWithPriorityQueue() throws 
IOException {
RocksDBStateBackend rocksDBStateBackend = 
createStateBackendWithOptimizePointLookup();
-   RocksDBKeyedStateBackend keyedStateBackend = 
createKeyedStateBackend(rocksDBStateBackend, new 
MockEnvironmentBuilder().build(), IntSerializer.INSTANCE);
-   KeyGroupedInternalPriorityQueue> priorityQueue =
-   keyedStateBackend.create("timer", new 
TimerSerializer<>(keyedStateBackend.getKeySerializer(), 
VoidNamespaceSerializer.INSTANCE));
-
-   PriorityQueue> 
expectedPriorityQueue

[flink-web] branch asf-site updated (dc268f2 -> 355c2ab)

2020-06-25 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git.


from dc268f2  regenerate website
 new a5fea70  [blog] flink on zeppelin - part2
 new 355c2ab  Rebuild website

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


Summary of changes:
 _posts/2020-06-23-flink-on-zeppelin-part2.md   | 109 +++
 content/blog/feed.xml  | 350 +++--
 content/blog/index.html|  39 ++-
 content/blog/page10/index.html |  40 ++-
 content/blog/page11/index.html |  40 ++-
 content/blog/page12/index.html |  25 ++
 content/blog/page2/index.html  |  36 ++-
 content/blog/page3/index.html  |  36 ++-
 content/blog/page4/index.html  |  36 ++-
 content/blog/page5/index.html  |  38 ++-
 content/blog/page6/index.html  |  38 ++-
 content/blog/page7/index.html  |  38 ++-
 content/blog/page8/index.html  |  40 ++-
 content/blog/page9/index.html  |  40 ++-
 .../2020/06/23/flink-on-zeppelin-part2.html}   | 112 ---
 .../flink_append_mode.gif  | Bin 0 -> 294307 bytes
 .../flink_python_udf.png   | Bin 0 -> 83093 bytes
 .../flink_scala_udf.png| Bin 0 -> 84516 bytes
 .../flink_single_mode.gif  | Bin 0 -> 58198 bytes
 .../flink_update_mode.gif  | Bin 0 -> 131055 bytes
 content/index.html |   9 +-
 content/zh/index.html  |   9 +-
 .../flink_append_mode.gif  | Bin 0 -> 294307 bytes
 .../flink_python_udf.png   | Bin 0 -> 83093 bytes
 .../flink_scala_udf.png| Bin 0 -> 84516 bytes
 .../flink_single_mode.gif  | Bin 0 -> 58198 bytes
 .../flink_update_mode.gif  | Bin 0 -> 131055 bytes
 27 files changed, 586 insertions(+), 449 deletions(-)
 create mode 100644 _posts/2020-06-23-flink-on-zeppelin-part2.md
 copy content/{news/2020/06/15/flink-on-zeppelin-part1.html => 
ecosystem/2020/06/23/flink-on-zeppelin-part2.html} (69%)
 create mode 100644 
content/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_append_mode.gif
 create mode 100644 
content/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_python_udf.png
 create mode 100644 
content/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_scala_udf.png
 create mode 100644 
content/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_single_mode.gif
 create mode 100644 
content/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_update_mode.gif
 create mode 100644 
img/blog/2020-06-23-flink-on-zeppelin-part2/flink_append_mode.gif
 create mode 100644 
img/blog/2020-06-23-flink-on-zeppelin-part2/flink_python_udf.png
 create mode 100644 
img/blog/2020-06-23-flink-on-zeppelin-part2/flink_scala_udf.png
 create mode 100644 
img/blog/2020-06-23-flink-on-zeppelin-part2/flink_single_mode.gif
 create mode 100644 
img/blog/2020-06-23-flink-on-zeppelin-part2/flink_update_mode.gif



[flink-web] 02/02: Rebuild website

2020-06-25 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git

commit 355c2ab49228658ad80cf5f1bc3e97cbac7cc479
Author: Yu Li 
AuthorDate: Thu Jun 25 14:15:43 2020 +0800

Rebuild website
---
 content/blog/feed.xml  | 350 +++-
 content/blog/index.html|  39 ++-
 content/blog/page10/index.html |  40 ++-
 content/blog/page11/index.html |  40 ++-
 content/blog/page12/index.html |  25 ++
 content/blog/page2/index.html  |  36 +-
 content/blog/page3/index.html  |  36 +-
 content/blog/page4/index.html  |  36 +-
 content/blog/page5/index.html  |  38 ++-
 content/blog/page6/index.html  |  38 ++-
 content/blog/page7/index.html  |  38 ++-
 content/blog/page8/index.html  |  40 ++-
 content/blog/page9/index.html  |  40 ++-
 .../2020/06/23/flink-on-zeppelin-part2.html| 365 +
 .../flink_append_mode.gif  | Bin 0 -> 294307 bytes
 .../flink_python_udf.png   | Bin 0 -> 83093 bytes
 .../flink_scala_udf.png| Bin 0 -> 84516 bytes
 .../flink_single_mode.gif  | Bin 0 -> 58198 bytes
 .../flink_update_mode.gif  | Bin 0 -> 131055 bytes
 content/index.html |   9 +-
 content/zh/index.html  |   9 +-
 21 files changed, 779 insertions(+), 400 deletions(-)

diff --git a/content/blog/feed.xml b/content/blog/feed.xml
index eabf57c..e191286 100644
--- a/content/blog/feed.xml
+++ b/content/blog/feed.xml
@@ -7,6 +7,117 @@
 https://flink.apache.org/blog/feed.xml; rel="self" 
type="application/rss+xml" />
 
 
+Flink on Zeppelin Notebooks for Interactive Data Analysis - Part 
2
+pIn a previous post, we introduced the basics of Flink on 
Zeppelin and how to do Streaming ETL. In this second part of the “Flink on 
Zeppelin” series of posts, I will share how to 
+perform streaming data visualization via Flink on Zeppelin and how to use 
Apache Flink UDFs in Zeppelin./p
+
+h1 id=streaming-data-visualizationStreaming Data 
Visualization/h1
+
+pWith a 
href=https://zeppelin.apache.org/Zeppelin/a;, you can 
build a real time streaming dashboard without writing any line of 
javascript/html/css code./p
+
+pOverall, Zeppelin supports 3 kinds of streaming data 
analytics:/p
+
+ul
+  liSingle Mode/li
+  liUpdate Mode/li
+  liAppend Mode/li
+/ul
+
+h3 id=single-modeSingle Mode/h3
+pSingle mode is used for cases when the result of a SQL statement is 
always one row, such as the following example. 
+The output format is translated in HTML, and you can specify a paragraph local 
property template for the final output content template. 
+And you can use code{i}/code as placeholder for the {i}th 
column of the result./p
+
+center
+img 
src=/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_single_mode.gif
 width=80% alt=Single Mode /
+/center
+
+h3 id=update-modeUpdate Mode/h3
+pUpdate mode is suitable for the cases when the output format is more 
than one row, 
+and will always be continuously updated. Here’s one example where we use 
codeGROUP BY/code./p
+
+center
+img 
src=/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_update_mode.gif
 width=80% alt=Update Mode /
+/center
+
+h3 id=append-modeAppend Mode/h3
+pAppend mode is suitable for the cases when the output data is always 
appended. 
+For instance, the example below uses a tumble window./p
+
+center
+img 
src=/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_append_mode.gif
 width=80% alt=Append Mode /
+/center
+
+h1 id=udfUDF/h1
+
+pSQL is a very powerful language, especially in expressing data flow. 
But most of the time, you need to handle complicated business logic that cannot 
be expressed by SQL.
+In these cases UDFs (user-defined functions) come particularly handy. In 
Zeppelin, you can write Scala or Python UDFs, while you can also import Scala, 
Python and Java UDFs.
+Here are 2 examples of Scala and Python UDFs:/p
+
+ul
+  liScala UDF/li
+/ul
+
+div class=highlightprecode 
class=language-scalaspan 
class=o%/spanspan 
class=nflink/span
+
+span class=kclass/span span 
class=ncScalaUpper/span span 
class=kextends/span span 
class=ncScalarFunction/span span 
class=o{/span
+span class=kdef/span span 
class=neval/spanspan 
class=o(/spanspan 
class=nstr/spanspan 
class=k:/span span 
class=ktString/spanspan 
class=o)/span span 
class=k=/span span 
class=nstr/spanspan 
class=o./spanspan clas [...]
+span class=o}/span
+span class=nbtenv/spanspan 
class=o./spanspan 
class=nregisterFunction/spanspan 
class=o(/spanspan 
class=squot;scala_upperquot;/spanspan 
cl

[flink-web] 01/02: [blog] flink on zeppelin - part2

2020-06-25 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git

commit a5fea70a283d19e9c6573797ad2beab161da9f78
Author: Jeff Zhang 
AuthorDate: Tue Jun 2 12:50:20 2020 +0800

[blog] flink on zeppelin - part2

Co-authored-by: morsapaes 

This closes #344.
---
 _posts/2020-06-23-flink-on-zeppelin-part2.md   | 109 +
 .../flink_append_mode.gif  | Bin 0 -> 294307 bytes
 .../flink_python_udf.png   | Bin 0 -> 83093 bytes
 .../flink_scala_udf.png| Bin 0 -> 84516 bytes
 .../flink_single_mode.gif  | Bin 0 -> 58198 bytes
 .../flink_update_mode.gif  | Bin 0 -> 131055 bytes
 6 files changed, 109 insertions(+)

diff --git a/_posts/2020-06-23-flink-on-zeppelin-part2.md 
b/_posts/2020-06-23-flink-on-zeppelin-part2.md
new file mode 100644
index 000..782e74c
--- /dev/null
+++ b/_posts/2020-06-23-flink-on-zeppelin-part2.md
@@ -0,0 +1,109 @@
+---
+layout: post
+title:  "Flink on Zeppelin Notebooks for Interactive Data Analysis - Part 2"
+date:   2020-06-23T08:00:00.000Z
+categories: ecosystem
+authors:
+- zjffdu:
+  name: "Jeff Zhang"
+  twitter: "zjffdu"
+---
+
+In a previous post, we introduced the basics of Flink on Zeppelin and how to 
do Streaming ETL. In this second part of the "Flink on Zeppelin" series of 
posts, I will share how to 
+perform streaming data visualization via Flink on Zeppelin and how to use 
Apache Flink UDFs in Zeppelin. 
+
+# Streaming Data Visualization
+
+With [Zeppelin](https://zeppelin.apache.org/), you can build a real time 
streaming dashboard without writing any line of javascript/html/css code.
+
+Overall, Zeppelin supports 3 kinds of streaming data analytics:
+
+* Single Mode
+* Update Mode
+* Append Mode
+
+### Single Mode
+Single mode is used for cases when the result of a SQL statement is always one 
row, such as the following example. 
+The output format is translated in HTML, and you can specify a paragraph local 
property template for the final output content template. 
+And you can use `{i}` as placeholder for the {i}th column of the result.
+
+
+
+
+
+### Update Mode
+Update mode is suitable for the cases when the output format is more than one 
row, 
+and will always be continuously updated. Here’s one example where we use 
``GROUP BY``.
+
+
+
+
+
+### Append Mode
+Append mode is suitable for the cases when the output data is always appended. 
+For instance, the example below uses a tumble window.
+
+
+
+
+
+# UDF
+
+SQL is a very powerful language, especially in expressing data flow. But most 
of the time, you need to handle complicated business logic that cannot be 
expressed by SQL.
+In these cases UDFs (user-defined functions) come particularly handy. In 
Zeppelin, you can write Scala or Python UDFs, while you can also import Scala, 
Python and Java UDFs.
+Here are 2 examples of Scala and Python UDFs:
+
+* Scala UDF
+
+```scala
+%flink
+
+class ScalaUpper extends ScalarFunction {
+def eval(str: String) = str.toUpperCase
+}
+btenv.registerFunction("scala_upper", new ScalaUpper())
+
+```
+ 
+* Python UDF
+
+```python
+
+%flink.pyflink
+
+class PythonUpper(ScalarFunction):
+def eval(self, s):
+ return s.upper()
+
+bt_env.register_function("python_upper", udf(PythonUpper(), 
DataTypes.STRING(), DataTypes.STRING()))
+
+```
+
+After you define the UDFs, you can use them directly in SQL:
+
+* Use Scala UDF in SQL
+
+
+
+
+
+* Use Python UDF in SQL
+
+
+
+
+
+# Summary
+
+In this post, we explained how to perform streaming data visualization via 
Flink on Zeppelin and how to use UDFs. 
+Besides that, you can do more in Zeppelin with Flink, such as batch 
processing, Hive integration and more.
+You can check the following articles for more details and here's a list of 
[Flink on Zeppelin tutorial 
videos](https://www.youtube.com/watch?v=YxPo0Fosjjg=PL4oy12nnS7FFtg3KV1iS5vDb0pTz12VcX)
 for your reference.
+
+# References
+
+* [Apache Zeppelin official website](http://zeppelin.apache.org)
+* Flink on Zeppelin tutorials - [Part 
1](https://medium.com/@zjffdu/flink-on-zeppelin-part-1-get-started-2591aaa6aa47)
+* Flink on Zeppelin tutorials - [Part 
2](https://medium.com/@zjffdu/flink-on-zeppelin-part-2-batch-711731df5ad9)
+* Flink on Zeppelin tutorials - [Part 
3](https://medium.com/@zjffdu/flink-on-zeppelin-part-3-streaming-5fca1e16754)
+* Flink on Zeppelin tutorials - [Part 
4](https://medium.com/@zjffdu/flink-on-zeppelin-part-4-advanced-usage-998b74908cd9)
+* [Flink on Zeppelin tutorial 
videos](https://www.youtube.com/watch?v=YxPo0Fosjjg=PL4oy12nnS7FFtg3KV1iS5vDb0pTz12VcX)
 
diff --git a/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_append_mode.gif 
b/img/blog/2020-06-23-flink-on-zeppelin-part2/flink_append_mode.gif
new file mode 1

svn commit: r40165 - in /dev/flink/flink-1.11.0-rc3: ./ python/

2020-06-24 Thread liyu
Author: liyu
Date: Wed Jun 24 10:45:34 2020
New Revision: 40165

Log:
flink-1.11.0-rc3

Added:
dev/flink/flink-1.11.0-rc3/
dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.11.0-rc3/flink-1.11.0-src.tgz   (with props)
dev/flink/flink-1.11.0-rc3/flink-1.11.0-src.tgz.asc   (with props)
dev/flink/flink-1.11.0-rc3/flink-1.11.0-src.tgz.sha512
dev/flink/flink-1.11.0-rc3/python/
dev/flink/flink-1.11.0-rc3/python/apache-flink-1.11.0.tar.gz   (with props)
dev/flink/flink-1.11.0-rc3/python/apache-flink-1.11.0.tar.gz.asc   (with 
props)
dev/flink/flink-1.11.0-rc3/python/apache-flink-1.11.0.tar.gz.sha512

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc3/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.sha512

Added: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.11.tgz.sha512 Wed Jun 
24 10:45:34 2020
@@ -0,0 +1 @@
+b9b5f4c4fbb555d204539824636fc5b3c9307f33ab9d0c68f54ca3757fcaea41581d7102a0fe691e91e96b3389720768839b3f0778ca4092d3db86abc7ca6a4f
  flink-1.11.0-bin-scala_2.11.tgz

Added: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc3/flink-1.11.0-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc3/flink-1.11.0

[flink] 02/03: Revert "[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse"

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 831daaae2c0cdef6871001e856dfd6a54da2a943
Author: Yu Li 
AuthorDate: Sat Jun 20 16:59:06 2020 +0800

Revert "[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse"

This reverts commit b8ddbef9a5cc5dc769ba61bd5019dd96843c932f.
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +---
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +--
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 -
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +++-
 .../state/RocksDBStateMisuseOptionTest.java| 147 -
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 ---
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 17 files changed, 47 insertions(+), 256 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index fb9a833..364185a 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,7 +29,6 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -64,9 +63,6 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
-   @Nonnull
-   private final ReadOptions readOptions;
-
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -116,7 +112,6 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
-   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -124,7 +119,6 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
-   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -310,7 +304,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle, 
readOptions)));
+   db.newIterator(columnFamilyHandle)));
}
 
/**
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 5bce695..1f43dd0 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
@@ -21,7 +21,6 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksD

[flink] 01/03: Revert "[FLINK-17800][roksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them"

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit f87937be3a4309f9d532254983e33923ea85bfd9
Author: Yu Li 
AuthorDate: Sat Jun 20 16:58:59 2020 +0800

Revert "[FLINK-17800][roksdb] Support customized RocksDB write/read options 
and use RocksDBResourceContainer to get them"

This reverts commit 88c22864504d772764c5838afe0b944f1da50a3a.
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 +++--
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 +--
 .../streaming/state/RocksDBOptionsFactory.java | 34 --
 .../streaming/state/RocksDBResourceContainer.java  | 34 --
 .../state/RocksDBResourceContainerTest.java| 14 -
 5 files changed, 21 insertions(+), 86 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 591e8b7..505bd2a 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
@@ -219,6 +219,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
+   WriteOptions writeOptions,
+   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -257,8 +259,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = optionsContainer.getWriteOptions();
-   this.readOptions = optionsContainer.getReadOptions();
+   this.writeOptions = writeOptions;
+   this.readOptions = readOptions;
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -368,6 +370,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
+   IOUtils.closeQuietly(readOptions);
+   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 94e0d5b..08b4864 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
@@ -54,7 +54,9 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -247,6 +249,9 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
+   // The write options to use in the states.
+   WriteOptions writeOptions = null;
+   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -284,7 +289,9 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeBatchWrapper = new RocksDBWriteBatchWrapper(db, 
optionsContainer.getW

[flink] 03/03: [FLINK-18242][state-backend-rocksdb] Separate RocksDBOptionsFactory from OptionsFactory

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit fd91affd8040123e2f757b24859b7dad33e09532
Author: Yu Li 
AuthorDate: Thu Jun 11 19:32:39 2020 +0800

[FLINK-18242][state-backend-rocksdb] Separate RocksDBOptionsFactory from 
OptionsFactory

This closes #12673.
---
 flink-python/pyflink/datastream/state_backend.py   |  8 ++--
 .../streaming/state/OptionsFactoryAdapter.java | 55 ++
 .../streaming/state/RocksDBOptionsFactory.java | 28 +--
 .../state/RocksDBOptionsFactoryAdapter.java|  4 +-
 .../streaming/state/RocksDBStateBackend.java   | 10 ++--
 .../state/RocksDBStateBackendConfigTest.java   | 11 -
 6 files changed, 78 insertions(+), 38 deletions(-)

diff --git a/flink-python/pyflink/datastream/state_backend.py 
b/flink-python/pyflink/datastream/state_backend.py
index d894137..46ccf66 100644
--- a/flink-python/pyflink/datastream/state_backend.py
+++ b/flink-python/pyflink/datastream/state_backend.py
@@ -682,11 +682,11 @@ class RocksDBStateBackend(StateBackend):
The options factory must have a 
default constructor.
 """
 gateway = get_gateway()
-JOptionsFactory = 
gateway.jvm.org.apache.flink.contrib.streaming.state.OptionsFactory
+JOptionsFactory = 
gateway.jvm.org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory
 j_options_factory_clz = load_java_class(options_factory_class_name)
 if not 
get_java_class(JOptionsFactory).isAssignableFrom(j_options_factory_clz):
-raise ValueError("The input class not implements OptionsFactory.")
-
self._j_rocks_db_state_backend.setOptions(j_options_factory_clz.newInstance())
+raise ValueError("The input class not implements 
RocksDBOptionsFactory.")
+
self._j_rocks_db_state_backend.setRocksDBOptions(j_options_factory_clz.newInstance())
 
 def get_options(self):
 """
@@ -695,7 +695,7 @@ class RocksDBStateBackend(StateBackend):
 
 :return: The fully-qualified class name of the options factory in Java.
 """
-j_options_factory = self._j_rocks_db_state_backend.getOptions()
+j_options_factory = self._j_rocks_db_state_backend.getRocksDBOptions()
 if j_options_factory is not None:
 return j_options_factory.getClass().getName()
 else:
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
new file mode 100644
index 000..666bc4b
--- /dev/null
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
@@ -0,0 +1,55 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+
+import java.util.ArrayList;
+
+/**
+ * A conversion from {@link RocksDBOptionsFactory} to {@link OptionsFactory}.
+ */
+public class OptionsFactoryAdapter implements OptionsFactory {
+
+   private static final long serialVersionUID = 1L;
+
+   private final RocksDBOptionsFactory rocksDBOptionsFactory;
+
+   OptionsFactoryAdapter(RocksDBOptionsFactory rocksDBOptionsFactory) {
+   this.rocksDBOptionsFactory = rocksDBOptionsFactory;
+   }
+
+   @Override
+   public DBOptions createDBOptions(DBOptions currentOptions) {
+   return rocksDBOptionsFactory.createDBOptions(currentOptions, 
new ArrayList<>());
+   }
+
+   @Override
+   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
+   return 
rocksDBOptionsFactory.createColumnOptions(currentOptions, new ArrayList<>(

[flink] branch release-1.10 updated (b1fdf6d -> fd91aff)

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from b1fdf6d  [FLINK-18329][legal] Fix typo
 new f87937b  Revert "[FLINK-17800][roksdb] Support customized RocksDB 
write/read options and use RocksDBResourceContainer to get them"
 new 831daaa  Revert "[FLINK-17800][roksdb] Ensure total order seek to 
avoid user misuse"
 new fd91aff  [FLINK-18242][state-backend-rocksdb] Separate 
RocksDBOptionsFactory from OptionsFactory

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


Summary of changes:
 flink-python/pyflink/datastream/state_backend.py   |   8 +-
 .../streaming/state/OptionsFactoryAdapter.java |  47 +++
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 +--
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +--
 .../streaming/state/RocksDBOptionsFactory.java |  62 +
 .../state/RocksDBOptionsFactoryAdapter.java|   4 +-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 -
 .../streaming/state/RocksDBStateBackend.java   |  10 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 -
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 --
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  39 --
 .../state/RocksDBStateMisuseOptionTest.java| 147 -
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 ---
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 24 files changed, 94 insertions(+), 396 deletions(-)
 copy 
flink-core/src/main/java/org/apache/flink/api/common/accumulators/ListAccumulator.java
 => 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
 (50%)
 delete mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 02/03: Revert "[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse"

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 831daaae2c0cdef6871001e856dfd6a54da2a943
Author: Yu Li 
AuthorDate: Sat Jun 20 16:59:06 2020 +0800

Revert "[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse"

This reverts commit b8ddbef9a5cc5dc769ba61bd5019dd96843c932f.
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +---
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +--
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 -
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +++-
 .../state/RocksDBStateMisuseOptionTest.java| 147 -
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 ---
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 17 files changed, 47 insertions(+), 256 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index fb9a833..364185a 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,7 +29,6 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -64,9 +63,6 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
-   @Nonnull
-   private final ReadOptions readOptions;
-
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -116,7 +112,6 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
-   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -124,7 +119,6 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
-   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -310,7 +304,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle, 
readOptions)));
+   db.newIterator(columnFamilyHandle)));
}
 
/**
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 5bce695..1f43dd0 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
@@ -21,7 +21,6 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksD

[flink] 03/03: [FLINK-18242][state-backend-rocksdb] Separate RocksDBOptionsFactory from OptionsFactory

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit fd91affd8040123e2f757b24859b7dad33e09532
Author: Yu Li 
AuthorDate: Thu Jun 11 19:32:39 2020 +0800

[FLINK-18242][state-backend-rocksdb] Separate RocksDBOptionsFactory from 
OptionsFactory

This closes #12673.
---
 flink-python/pyflink/datastream/state_backend.py   |  8 ++--
 .../streaming/state/OptionsFactoryAdapter.java | 55 ++
 .../streaming/state/RocksDBOptionsFactory.java | 28 +--
 .../state/RocksDBOptionsFactoryAdapter.java|  4 +-
 .../streaming/state/RocksDBStateBackend.java   | 10 ++--
 .../state/RocksDBStateBackendConfigTest.java   | 11 -
 6 files changed, 78 insertions(+), 38 deletions(-)

diff --git a/flink-python/pyflink/datastream/state_backend.py 
b/flink-python/pyflink/datastream/state_backend.py
index d894137..46ccf66 100644
--- a/flink-python/pyflink/datastream/state_backend.py
+++ b/flink-python/pyflink/datastream/state_backend.py
@@ -682,11 +682,11 @@ class RocksDBStateBackend(StateBackend):
The options factory must have a 
default constructor.
 """
 gateway = get_gateway()
-JOptionsFactory = 
gateway.jvm.org.apache.flink.contrib.streaming.state.OptionsFactory
+JOptionsFactory = 
gateway.jvm.org.apache.flink.contrib.streaming.state.RocksDBOptionsFactory
 j_options_factory_clz = load_java_class(options_factory_class_name)
 if not 
get_java_class(JOptionsFactory).isAssignableFrom(j_options_factory_clz):
-raise ValueError("The input class not implements OptionsFactory.")
-
self._j_rocks_db_state_backend.setOptions(j_options_factory_clz.newInstance())
+raise ValueError("The input class not implements 
RocksDBOptionsFactory.")
+
self._j_rocks_db_state_backend.setRocksDBOptions(j_options_factory_clz.newInstance())
 
 def get_options(self):
 """
@@ -695,7 +695,7 @@ class RocksDBStateBackend(StateBackend):
 
 :return: The fully-qualified class name of the options factory in Java.
 """
-j_options_factory = self._j_rocks_db_state_backend.getOptions()
+j_options_factory = self._j_rocks_db_state_backend.getRocksDBOptions()
 if j_options_factory is not None:
 return j_options_factory.getClass().getName()
 else:
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
new file mode 100644
index 000..666bc4b
--- /dev/null
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
@@ -0,0 +1,55 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.annotation.VisibleForTesting;
+
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+
+import java.util.ArrayList;
+
+/**
+ * A conversion from {@link RocksDBOptionsFactory} to {@link OptionsFactory}.
+ */
+public class OptionsFactoryAdapter implements OptionsFactory {
+
+   private static final long serialVersionUID = 1L;
+
+   private final RocksDBOptionsFactory rocksDBOptionsFactory;
+
+   OptionsFactoryAdapter(RocksDBOptionsFactory rocksDBOptionsFactory) {
+   this.rocksDBOptionsFactory = rocksDBOptionsFactory;
+   }
+
+   @Override
+   public DBOptions createDBOptions(DBOptions currentOptions) {
+   return rocksDBOptionsFactory.createDBOptions(currentOptions, 
new ArrayList<>());
+   }
+
+   @Override
+   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
+   return 
rocksDBOptionsFactory.createColumnOptions(currentOptions, new ArrayList<>(

[flink] 01/03: Revert "[FLINK-17800][roksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them"

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit f87937be3a4309f9d532254983e33923ea85bfd9
Author: Yu Li 
AuthorDate: Sat Jun 20 16:58:59 2020 +0800

Revert "[FLINK-17800][roksdb] Support customized RocksDB write/read options 
and use RocksDBResourceContainer to get them"

This reverts commit 88c22864504d772764c5838afe0b944f1da50a3a.
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 +++--
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 +--
 .../streaming/state/RocksDBOptionsFactory.java | 34 --
 .../streaming/state/RocksDBResourceContainer.java  | 34 --
 .../state/RocksDBResourceContainerTest.java| 14 -
 5 files changed, 21 insertions(+), 86 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 591e8b7..505bd2a 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
@@ -219,6 +219,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
+   WriteOptions writeOptions,
+   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -257,8 +259,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = optionsContainer.getWriteOptions();
-   this.readOptions = optionsContainer.getReadOptions();
+   this.writeOptions = writeOptions;
+   this.readOptions = readOptions;
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -368,6 +370,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
+   IOUtils.closeQuietly(readOptions);
+   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 94e0d5b..08b4864 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
@@ -54,7 +54,9 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
+import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -247,6 +249,9 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
+   // The write options to use in the states.
+   WriteOptions writeOptions = null;
+   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -284,7 +289,9 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeBatchWrapper = new RocksDBWriteBatchWrapper(db, 
optionsContainer.getW

[flink] branch release-1.10 updated (b1fdf6d -> fd91aff)

2020-06-20 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from b1fdf6d  [FLINK-18329][legal] Fix typo
 new f87937b  Revert "[FLINK-17800][roksdb] Support customized RocksDB 
write/read options and use RocksDBResourceContainer to get them"
 new 831daaa  Revert "[FLINK-17800][roksdb] Ensure total order seek to 
avoid user misuse"
 new fd91aff  [FLINK-18242][state-backend-rocksdb] Separate 
RocksDBOptionsFactory from OptionsFactory

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


Summary of changes:
 flink-python/pyflink/datastream/state_backend.py   |   8 +-
 .../streaming/state/OptionsFactoryAdapter.java |  47 +++
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 +--
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 +--
 .../streaming/state/RocksDBOptionsFactory.java |  62 +
 .../state/RocksDBOptionsFactoryAdapter.java|   4 +-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 -
 .../streaming/state/RocksDBStateBackend.java   |  10 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 -
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 --
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  39 --
 .../state/RocksDBStateMisuseOptionTest.java| 147 -
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 ---
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 24 files changed, 94 insertions(+), 396 deletions(-)
 copy 
flink-core/src/main/java/org/apache/flink/api/common/accumulators/ListAccumulator.java
 => 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactoryAdapter.java
 (50%)
 delete mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



svn commit: r40081 - in /dev/flink/flink-1.11.0-rc2: ./ python/

2020-06-18 Thread liyu
Author: liyu
Date: Thu Jun 18 11:47:31 2020
New Revision: 40081

Log: (empty)

Added:
dev/flink/flink-1.11.0-rc2/
dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.11.0-rc2/flink-1.11.0-src.tgz   (with props)
dev/flink/flink-1.11.0-rc2/flink-1.11.0-src.tgz.asc   (with props)
dev/flink/flink-1.11.0-rc2/flink-1.11.0-src.tgz.sha512
dev/flink/flink-1.11.0-rc2/python/
dev/flink/flink-1.11.0-rc2/python/apache-flink-1.11.0.tar.gz   (with props)
dev/flink/flink-1.11.0-rc2/python/apache-flink-1.11.0.tar.gz.asc   (with 
props)
dev/flink/flink-1.11.0-rc2/python/apache-flink-1.11.0.tar.gz.sha512

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc2/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.sha512

Added: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.11.tgz.sha512 Thu Jun 
18 11:47:31 2020
@@ -0,0 +1 @@
+cc0184b2c1bc2db244e6a2b56be94be45b8d18eb8028f30ddec2488370939351605020316d2e11ca59448e329a8b91679b13580846cb159c569089c6f9786626
  flink-1.11.0-bin-scala_2.11.tgz

Added: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin

[flink] branch release-1.11 updated: [FLINK-18242][state-backend-rocksdb] Remove the deprecated OptionsFactory and related classes

2020-06-17 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu 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 e13146f  [FLINK-18242][state-backend-rocksdb] Remove the deprecated 
OptionsFactory and related classes
e13146f is described below

commit e13146f80114266aa34c9fe9f3dc27e87f7a7649
Author: Yu Li 
AuthorDate: Thu Jun 11 19:32:39 2020 +0800

[FLINK-18242][state-backend-rocksdb] Remove the deprecated OptionsFactory 
and related classes

This closes #12683.
---
 .../generated/expert_rocksdb_section.html  |  2 +-
 .../generated/rocks_db_configuration.html  |  2 +-
 docs/ops/state/state_backends.md   |  8 +-
 docs/ops/state/state_backends.zh.md|  4 +-
 flink-python/pyflink/datastream/state_backend.py   |  8 +-
 .../state/ConfigurableOptionsFactory.java  | 39 --
 .../state/DefaultConfigurableOptionsFactory.java   |  6 +-
 .../contrib/streaming/state/OptionsFactory.java| 69 -
 .../contrib/streaming/state/RocksDBOptions.java|  2 +-
 .../streaming/state/RocksDBOptionsFactory.java | 30 +---
 .../state/RocksDBOptionsFactoryAdapter.java| 75 --
 .../streaming/state/RocksDBStateBackend.java   | 24 --
 .../RocksDBOptionsFactoryCompatibilityTest.java| 89 --
 .../state/RocksDBStateBackendConfigTest.java   |  3 +-
 14 files changed, 20 insertions(+), 341 deletions(-)

diff --git a/docs/_includes/generated/expert_rocksdb_section.html 
b/docs/_includes/generated/expert_rocksdb_section.html
index d3cbfe1..b24058d 100644
--- a/docs/_includes/generated/expert_rocksdb_section.html
+++ b/docs/_includes/generated/expert_rocksdb_section.html
@@ -30,7 +30,7 @@
 state.backend.rocksdb.predefined-options
 "DEFAULT"
 String
-The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the OptionsFactory are applied on top of 
these predefined ones.
+The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the RocksDBOptionsFactory are applied on 
top of these predefined ones.
 
 
 
diff --git a/docs/_includes/generated/rocks_db_configuration.html 
b/docs/_includes/generated/rocks_db_configuration.html
index ee45dc2..44ccc96 100644
--- a/docs/_includes/generated/rocks_db_configuration.html
+++ b/docs/_includes/generated/rocks_db_configuration.html
@@ -54,7 +54,7 @@
 state.backend.rocksdb.predefined-options
 "DEFAULT"
 String
-The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the OptionsFactory are applied on top of 
these predefined ones.
+The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the RocksDBOptionsFactory are applied on 
top of these predefined ones.
 
 
 state.backend.rocksdb.timer-service.factory
diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md
index c13c3d6..d989d29 100644
--- a/docs/ops/state/state_backends.md
+++ b/docs/ops/state/state_backends.md
@@ -243,7 +243,7 @@ For advanced tuning, Flink also provides two parameters to 
control the division
   Moreover, the L0 level filter and index are pinned into the cache by default 
to mitigate performance problems,
   more details please refer to the 
[RocksDB-documentation](https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-filter-and-compression-dictionary-blocks).
 
-Note When the above described mechanism 
(`cache` and `write buffer manager`) is enabled, it will override any 
customized settings for block caches and write buffers done via 
[`PredefinedOptions`](#predefined-per-columnfamily-options) and 
[`OptionsFactory`](#passing-options-factory-to-rocksdb).
+Note When the above described mechanism 
(`cache` and `write buffer manager`) is enabled, it will override any 
customized settings for block caches and wr

[flink] branch release-1.11 updated: [FLINK-18242][state-backend-rocksdb] Remove the deprecated OptionsFactory and related classes

2020-06-17 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu 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 e13146f  [FLINK-18242][state-backend-rocksdb] Remove the deprecated 
OptionsFactory and related classes
e13146f is described below

commit e13146f80114266aa34c9fe9f3dc27e87f7a7649
Author: Yu Li 
AuthorDate: Thu Jun 11 19:32:39 2020 +0800

[FLINK-18242][state-backend-rocksdb] Remove the deprecated OptionsFactory 
and related classes

This closes #12683.
---
 .../generated/expert_rocksdb_section.html  |  2 +-
 .../generated/rocks_db_configuration.html  |  2 +-
 docs/ops/state/state_backends.md   |  8 +-
 docs/ops/state/state_backends.zh.md|  4 +-
 flink-python/pyflink/datastream/state_backend.py   |  8 +-
 .../state/ConfigurableOptionsFactory.java  | 39 --
 .../state/DefaultConfigurableOptionsFactory.java   |  6 +-
 .../contrib/streaming/state/OptionsFactory.java| 69 -
 .../contrib/streaming/state/RocksDBOptions.java|  2 +-
 .../streaming/state/RocksDBOptionsFactory.java | 30 +---
 .../state/RocksDBOptionsFactoryAdapter.java| 75 --
 .../streaming/state/RocksDBStateBackend.java   | 24 --
 .../RocksDBOptionsFactoryCompatibilityTest.java| 89 --
 .../state/RocksDBStateBackendConfigTest.java   |  3 +-
 14 files changed, 20 insertions(+), 341 deletions(-)

diff --git a/docs/_includes/generated/expert_rocksdb_section.html 
b/docs/_includes/generated/expert_rocksdb_section.html
index d3cbfe1..b24058d 100644
--- a/docs/_includes/generated/expert_rocksdb_section.html
+++ b/docs/_includes/generated/expert_rocksdb_section.html
@@ -30,7 +30,7 @@
 state.backend.rocksdb.predefined-options
 "DEFAULT"
 String
-The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the OptionsFactory are applied on top of 
these predefined ones.
+The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the RocksDBOptionsFactory are applied on 
top of these predefined ones.
 
 
 
diff --git a/docs/_includes/generated/rocks_db_configuration.html 
b/docs/_includes/generated/rocks_db_configuration.html
index ee45dc2..44ccc96 100644
--- a/docs/_includes/generated/rocks_db_configuration.html
+++ b/docs/_includes/generated/rocks_db_configuration.html
@@ -54,7 +54,7 @@
 state.backend.rocksdb.predefined-options
 "DEFAULT"
 String
-The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the OptionsFactory are applied on top of 
these predefined ones.
+The predefined settings for RocksDB DBOptions and 
ColumnFamilyOptions by Flink community. Current supported candidate 
predefined-options are DEFAULT, SPINNING_DISK_OPTIMIZED, 
SPINNING_DISK_OPTIMIZED_HIGH_MEM or FLASH_SSD_OPTIMIZED. Note that user 
customized options and options from the RocksDBOptionsFactory are applied on 
top of these predefined ones.
 
 
 state.backend.rocksdb.timer-service.factory
diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md
index c13c3d6..d989d29 100644
--- a/docs/ops/state/state_backends.md
+++ b/docs/ops/state/state_backends.md
@@ -243,7 +243,7 @@ For advanced tuning, Flink also provides two parameters to 
control the division
   Moreover, the L0 level filter and index are pinned into the cache by default 
to mitigate performance problems,
   more details please refer to the 
[RocksDB-documentation](https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-filter-and-compression-dictionary-blocks).
 
-Note When the above described mechanism 
(`cache` and `write buffer manager`) is enabled, it will override any 
customized settings for block caches and write buffers done via 
[`PredefinedOptions`](#predefined-per-columnfamily-options) and 
[`OptionsFactory`](#passing-options-factory-to-rocksdb).
+Note When the above described mechanism 
(`cache` and `write buffer manager`) is enabled, it will override any 
customized settings for block caches and wr

[flink] branch master updated (c42a2f9 -> 5ed371f)

2020-06-17 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from c42a2f9  [FLINK-18299][json] Fix the non SQL standard timestamp format 
in JSON format
 add 5ed371f  [FLINK-18242][state-backend-rocksdb] Remove the deprecated 
OptionsFactory and related classes

No new revisions were added by this update.

Summary of changes:
 .../generated/expert_rocksdb_section.html  |  2 +-
 .../generated/rocks_db_configuration.html  |  2 +-
 docs/ops/state/state_backends.md   |  8 +-
 docs/ops/state/state_backends.zh.md|  4 +-
 flink-python/pyflink/datastream/state_backend.py   |  8 +-
 .../state/ConfigurableOptionsFactory.java  | 39 --
 .../state/DefaultConfigurableOptionsFactory.java   |  6 +-
 .../contrib/streaming/state/OptionsFactory.java| 69 -
 .../contrib/streaming/state/RocksDBOptions.java|  2 +-
 .../streaming/state/RocksDBOptionsFactory.java | 30 +---
 .../state/RocksDBOptionsFactoryAdapter.java| 75 --
 .../streaming/state/RocksDBStateBackend.java   | 24 --
 .../RocksDBOptionsFactoryCompatibilityTest.java| 89 --
 .../state/RocksDBStateBackendConfigTest.java   |  3 +-
 14 files changed, 20 insertions(+), 341 deletions(-)
 delete mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableOptionsFactory.java
 delete mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/OptionsFactory.java
 delete mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryAdapter.java
 delete mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryCompatibilityTest.java



[flink] branch master updated (07772bd -> 204aa6d)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 07772bd  [FLINK-18238][checkpoint] Broadcast CancelCheckpointMarker 
while executing checkpoint aborted by coordinator RPC
 add 204aa6d  [FLINK-18332][state] Add error message to precondition in 
KeyGroupPartitionedPriorityQueue

No new revisions were added by this update.

Summary of changes:
 .../runtime/state/heap/KeyGroupPartitionedPriorityQueue.java  | 8 +++-
 1 file changed, 7 insertions(+), 1 deletion(-)



[flink] 01/02: [FLINK-17800][roksdb] Ensure total order seek to avoid user misuse

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit b8ddbef9a5cc5dc769ba61bd5019dd96843c932f
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 17 files changed, 256 insertions(+), 47 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class

[flink] 02/02: [FLINK-17800][roksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 88c22864504d772764c5838afe0b944f1da50a3a
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][roksdb] Support customized RocksDB write/read options and use 
RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 505bd2a..591e8b7 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 08b4864..94e0d5b 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -249,9 +247,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -289,9 +284,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] branch release-1.10 updated (5c0f827 -> 88c2286)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 5c0f827  [FLINK-17322][network] Disallowing repeated consumer creation 
for BufferBuilder.
 new b8ddbef  [FLINK-17800][roksdb] Ensure total order seek to avoid user 
misuse
 new 88c2286  [FLINK-17800][roksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 20 files changed, 328 insertions(+), 54 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 02/02: [FLINK-17800][roksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 88c22864504d772764c5838afe0b944f1da50a3a
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][roksdb] Support customized RocksDB write/read options and use 
RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 505bd2a..591e8b7 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 08b4864..94e0d5b 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -249,9 +247,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -289,9 +284,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] 01/02: [FLINK-17800][roksdb] Ensure total order seek to avoid user misuse

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit b8ddbef9a5cc5dc769ba61bd5019dd96843c932f
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 17 files changed, 256 insertions(+), 47 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class

[flink] branch release-1.10 updated (5c0f827 -> 88c2286)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 5c0f827  [FLINK-17322][network] Disallowing repeated consumer creation 
for BufferBuilder.
 new b8ddbef  [FLINK-17800][roksdb] Ensure total order seek to avoid user 
misuse
 new 88c2286  [FLINK-17800][roksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 .../api/operators/TimerHeapInternalTimer.java  |   2 +-
 20 files changed, 328 insertions(+), 54 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] branch master updated (3792f8f -> f125062)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 3792f8f  [FLINK-17666][table-planner-blink] Insert into partitioned 
table can fail with select *
 add 8ca388c  [FLINK-17800][roksdb] Ensure total order seek to avoid user 
misuse
 add f125062  [FLINK-17800][roksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

No new revisions were added by this update.

Summary of changes:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 327 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] branch release-1.11 updated (fbae8fc -> 8f31729)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from fbae8fc  [FLINK-17666][table-planner-blink] Insert into partitioned 
table can fail with select *
 new 5c0de8d  [FLINK-17800][roksdb] Ensure total order seek to avoid user 
misuse
 new 8f31729  [FLINK-17800][roksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 327 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 01/02: [FLINK-17800][roksdb] Ensure total order seek to avoid user misuse

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 5c0de8d6d5a9eea1a779cf3703412f522bece54c
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 16 files changed, 255 insertions(+), 46 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class RocksDBIncrementalCheckpointUtils {
@Nonnegative long

[flink] branch master updated (3792f8f -> f125062)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 3792f8f  [FLINK-17666][table-planner-blink] Insert into partitioned 
table can fail with select *
 add 8ca388c  [FLINK-17800][roksdb] Ensure total order seek to avoid user 
misuse
 add f125062  [FLINK-17800][roksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

No new revisions were added by this update.

Summary of changes:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 327 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 02/02: [FLINK-17800][roksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 8f3172962c5033d4a7dbf5232c18d3e872821c02
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][roksdb] Support customized RocksDB write/read options and use 
RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 f0cce0b..24b897a 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 b9bdbb8..1846bde 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -243,9 +241,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -283,9 +278,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] 02/02: [FLINK-17800][roksdb] Support customized RocksDB write/read options and use RocksDBResourceContainer to get them

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 8f3172962c5033d4a7dbf5232c18d3e872821c02
Author: Yu Li 
AuthorDate: Tue Jun 16 14:01:29 2020 +0800

[FLINK-17800][roksdb] Support customized RocksDB write/read options and use 
RocksDBResourceContainer to get them
---
 .../streaming/state/RocksDBKeyedStateBackend.java  |  8 ++---
 .../state/RocksDBKeyedStateBackendBuilder.java | 17 ++-
 .../streaming/state/RocksDBOptionsFactory.java | 34 ++
 .../streaming/state/RocksDBResourceContainer.java  | 34 ++
 .../state/RocksDBResourceContainerTest.java| 14 +
 5 files changed, 86 insertions(+), 21 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 f0cce0b..24b897a 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
@@ -219,8 +219,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
RocksDB db,
-   WriteOptions writeOptions,
-   ReadOptions readOptions,
LinkedHashMap kvStateInformation,
int keyGroupPrefixBytes,
CloseableRegistry cancelStreamRegistry,
@@ -259,8 +257,8 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
this.keyGroupPrefixBytes = keyGroupPrefixBytes;
this.kvStateInformation = kvStateInformation;
 
-   this.writeOptions = writeOptions;
-   this.readOptions = readOptions;
+   this.writeOptions = optionsContainer.getWriteOptions();
+   this.readOptions = optionsContainer.getReadOptions();
checkArgument(writeBatchSize >= 0, "Write batch size have to be 
no negative value.");
this.writeBatchSize = writeBatchSize;
this.db = db;
@@ -370,8 +368,6 @@ public class RocksDBKeyedStateBackend extends 
AbstractKeyedStateBackend {
columnFamilyOptions.forEach(IOUtils::closeQuietly);
 
IOUtils.closeQuietly(optionsContainer);
-   IOUtils.closeQuietly(readOptions);
-   IOUtils.closeQuietly(writeOptions);
 

ttlCompactFiltersManager.disposeAndClearRegisteredCompactionFactories();
 
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 b9bdbb8..1846bde 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
@@ -54,9 +54,7 @@ import org.apache.flink.util.ResourceGuard;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
-import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
-import org.rocksdb.WriteOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -243,9 +241,6 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
ColumnFamilyHandle defaultColumnFamilyHandle = null;
RocksDBNativeMetricMonitor nativeMetricMonitor = null;
CloseableRegistry cancelStreamRegistryForBackend = new 
CloseableRegistry();
-   // The write options to use in the states.
-   WriteOptions writeOptions = null;
-   ReadOptions readOptions = null;
LinkedHashMap kvStateInformation = new 
LinkedHashMap<>();
RocksDB db = null;
AbstractRocksDBRestoreOperation restoreOperation = null;
@@ -283,9 +278,7 @@ public class RocksDBKeyedStateBackendBuilder extends 
AbstractKeyedStateBacken
}
}
 
-   writeOptions = new WriteOptions().setDisableWAL(true);
-   readOptions = 
RocksDBOperationUtils.createTotalOrderSeekReadOptions();
-   writ

[flink] branch release-1.11 updated (fbae8fc -> 8f31729)

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from fbae8fc  [FLINK-17666][table-planner-blink] Insert into partitioned 
table can fail with select *
 new 5c0de8d  [FLINK-17800][roksdb] Ensure total order seek to avoid user 
misuse
 new 8f31729  [FLINK-17800][roksdb] Support customized RocksDB write/read 
options and use RocksDBResourceContainer to get them

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:
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  21 ++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  16 +--
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../streaming/state/RocksDBOptionsFactory.java |  34 +
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../streaming/state/RocksDBResourceContainer.java  |  34 +
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBResourceContainerTest.java|  14 ++
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 19 files changed, 327 insertions(+), 53 deletions(-)
 create mode 100644 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateMisuseOptionTest.java



[flink] 01/02: [FLINK-17800][roksdb] Ensure total order seek to avoid user misuse

2020-06-16 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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

commit 5c0de8d6d5a9eea1a779cf3703412f522bece54c
Author: Yun Tang 
AuthorDate: Fri Jun 5 01:16:27 2020 +0800

[FLINK-17800][roksdb] Ensure total order seek to avoid user misuse
---
 .../state/RocksDBCachingPriorityQueueSet.java  |   8 +-
 .../state/RocksDBIncrementalCheckpointUtils.java   |   4 +-
 .../streaming/state/RocksDBKeyedStateBackend.java  |  23 +++-
 .../state/RocksDBKeyedStateBackendBuilder.java |  17 ++-
 .../contrib/streaming/state/RocksDBMapState.java   |   6 +-
 .../streaming/state/RocksDBOperationUtils.java |  15 ++-
 .../state/RocksDBPriorityQueueSetFactory.java  |   9 +-
 .../RocksDBIncrementalRestoreOperation.java|   7 +-
 ...rtitionedPriorityQueueWithRocksDBStoreTest.java |   1 +
 .../contrib/streaming/state/RocksDBResource.java   |   4 +-
 .../state/RocksDBRocksStateKeysIteratorTest.java   |   2 +-
 .../state/RocksDBStateBackendConfigTest.java   |  28 +---
 .../state/RocksDBStateMisuseOptionTest.java| 147 +
 .../contrib/streaming/state/RocksDBTestUtils.java  |  21 +++
 ...RocksKeyGroupsRocksSingleStateIteratorTest.java |   6 +-
 .../state/benchmark/RocksDBPerformanceTest.java|   3 +-
 16 files changed, 255 insertions(+), 46 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
index 364185a..fb9a833 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBCachingPriorityQueueSet.java
@@ -29,6 +29,7 @@ import org.apache.flink.util.FlinkRuntimeException;
 import 
org.apache.flink.shaded.guava18.com.google.common.primitives.UnsignedBytes;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -63,6 +64,9 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull
private final RocksDB db;
 
+   @Nonnull
+   private final ReadOptions readOptions;
+
/** Handle to the column family of the RocksDB instance in which the 
elements are stored. */
@Nonnull
private final ColumnFamilyHandle columnFamilyHandle;
@@ -112,6 +116,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnegative int keyGroupId,
@Nonnegative int keyGroupPrefixBytes,
@Nonnull RocksDB db,
+   @Nonnull ReadOptions readOptions,
@Nonnull ColumnFamilyHandle columnFamilyHandle,
@Nonnull TypeSerializer byteOrderProducingSerializer,
@Nonnull DataOutputSerializer outputStream,
@@ -119,6 +124,7 @@ public class RocksDBCachingPriorityQueueSet
@Nonnull RocksDBWriteBatchWrapper batchWrapper,
@Nonnull OrderedByteArraySetCache orderedByteArraySetCache) {
this.db = db;
+   this.readOptions = readOptions;
this.columnFamilyHandle = columnFamilyHandle;
this.byteOrderProducingSerializer = 
byteOrderProducingSerializer;
this.batchWrapper = batchWrapper;
@@ -304,7 +310,7 @@ public class RocksDBCachingPriorityQueueSet
flushWriteBatch();
return new RocksBytesIterator(
new RocksIteratorWrapper(
-   db.newIterator(columnFamilyHandle)));
+   db.newIterator(columnFamilyHandle, 
readOptions)));
}
 
/**
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 1f43dd0..5bce695 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
@@ -21,6 +21,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 
 import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 
@@ -116,7 +117,8 @@ public class RocksDBIncrementalCheckpointUtils {
@Nonnegative long

[flink] branch master updated: [FLINK-17384][connectors/hbase] Support reading hbase conf dir from flink-conf.yaml

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

liyu 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 0799b5c  [FLINK-17384][connectors/hbase] Support reading hbase conf 
dir from flink-conf.yaml
0799b5c is described below

commit 0799b5c20a127110e47439668cf8f8db2e4ecbf3
Author: liuyongvs 
AuthorDate: Thu May 14 15:48:50 2020 +0800

[FLINK-17384][connectors/hbase] Support reading hbase conf dir from 
flink-conf.yaml

This closes #12144.
---
 .../generated/environment_configuration.html   |   6 +
 .../connector/hbase/HBaseDynamicTableFactory.java  |   4 +-
 .../flink/connector/hbase/HBaseTableFactory.java   |   4 +-
 .../hbase/sink/HBaseDynamicTableSink.java  |   4 +-
 .../connector/hbase/sink/HBaseSinkFunction.java|   3 +-
 .../connector/hbase/sink/HBaseUpsertTableSink.java |   4 +-
 .../hbase/source/AbstractTableInputFormat.java |   3 +-
 .../hbase/source/HBaseLookupFunction.java  |   3 +-
 .../hbase/util/HBaseConfigurationUtil.java |  66 +++-
 .../hbase/util/HBaseConfigLoadingTest.java | 176 +
 .../src/test/resources/hbase-site.xml  |  29 
 .../apache/flink/configuration/CoreOptions.java|  11 ++
 flink-dist/src/main/flink-bin/bin/config.sh|  24 +++
 13 files changed, 322 insertions(+), 15 deletions(-)

diff --git a/docs/_includes/generated/environment_configuration.html 
b/docs/_includes/generated/environment_configuration.html
index 912ecbb..0fffb88 100644
--- a/docs/_includes/generated/environment_configuration.html
+++ b/docs/_includes/generated/environment_configuration.html
@@ -15,6 +15,12 @@
 Path to hadoop configuration directory. It is required to read 
HDFS and/or YARN configuration. You can also set it via environment 
variable.
 
 
+env.hbase.conf.dir
+(none)
+String
+Path to hbase configuration directory. It is required to read 
HBASE configuration. You can also set it via environment variable.
+
+
 env.java.opts
 (none)
 String
diff --git 
a/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseDynamicTableFactory.java
 
b/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseDynamicTableFactory.java
index b0e6dee..48b70eb 100644
--- 
a/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseDynamicTableFactory.java
+++ 
b/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseDynamicTableFactory.java
@@ -25,6 +25,7 @@ import org.apache.flink.connector.hbase.options.HBaseOptions;
 import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
 import org.apache.flink.connector.hbase.sink.HBaseDynamicTableSink;
 import org.apache.flink.connector.hbase.source.HBaseDynamicTableSource;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
 import org.apache.flink.connector.hbase.util.HBaseTableSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.connector.sink.DynamicTableSink;
@@ -34,7 +35,6 @@ import 
org.apache.flink.table.factories.DynamicTableSourceFactory;
 import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 
 import java.time.Duration;
@@ -108,7 +108,7 @@ public class HBaseDynamicTableFactory implements 
DynamicTableSourceFactory, Dyna
 
String hTableName = helper.getOptions().get(TABLE_NAME);
// create default configuration from current runtime env 
(`hbase-site.xml` in classpath) first,
-   Configuration hbaseClientConf = HBaseConfiguration.create();
+   Configuration hbaseClientConf = 
HBaseConfigurationUtil.getHBaseConfiguration();
hbaseClientConf.set(HConstants.ZOOKEEPER_QUORUM, 
helper.getOptions().get(ZOOKEEPER_QUORUM));
hbaseClientConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, 
helper.getOptions().get(ZOOKEEPER_ZNODE_PARENT));
String nullStringLiteral = 
helper.getOptions().get(NULL_STRING_LITERAL);
diff --git 
a/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseTableFactory.java
 
b/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseTableFactory.java
index f64f9b9..ca3e1e5 100644
--- 
a/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseTableFactory.java
+++ 
b/flink-connectors/flink-connector-hbase/src/main/java/org/apache/flink/connector/hbase/HBaseTableFactory.java
@@ -26,6 +26,7 @@ import

[flink] branch release-1.11 updated (79cb1a1 -> 6f9042b)

2020-05-28 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 79cb1a1  [FLINK-17558][netty] Release partitions asynchronously
 add 6f9042b  [FLINK-17865][checkpoint] Increase default size of 
'state.backend.fs.memory-threshold'

No new revisions were added by this update.

Summary of changes:
 .../generated/checkpointing_configuration.html |  6 ++--
 .../generated/expert_state_backends_section.html   |  6 ++--
 .../flink/configuration/CheckpointingOptions.java  |  7 ++--
 .../state/api/output/SavepointOutputFormat.java|  2 +-
 .../pyflink/datastream/tests/test_state_backend.py |  2 +-
 .../runtime/state/filesystem/FsStateBackend.java   | 39 ++
 .../runtime/state/StateBackendLoadingTest.java | 17 +-
 .../flink/test/checkpointing/SavepointITCase.java  |  4 +--
 .../utils/SavepointMigrationTestBase.java  |  3 +-
 9 files changed, 50 insertions(+), 36 deletions(-)



[flink] branch master updated: [FLINK-17865][checkpoint] Increase default size of 'state.backend.fs.memory-threshold'

2020-05-28 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu 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 a835f31  [FLINK-17865][checkpoint] Increase default size of 
'state.backend.fs.memory-threshold'
a835f31 is described below

commit a835f31a3a78f34b4a80f9e634b34c6a6681a482
Author: Yun Tang 
AuthorDate: Thu May 21 20:37:56 2020 +0800

[FLINK-17865][checkpoint] Increase default size of 
'state.backend.fs.memory-threshold'

This closes #12282.
---
 .../generated/checkpointing_configuration.html |  6 ++--
 .../generated/expert_state_backends_section.html   |  6 ++--
 .../flink/configuration/CheckpointingOptions.java  |  7 ++--
 .../state/api/output/SavepointOutputFormat.java|  2 +-
 .../pyflink/datastream/tests/test_state_backend.py |  2 +-
 .../runtime/state/filesystem/FsStateBackend.java   | 39 ++
 .../runtime/state/StateBackendLoadingTest.java | 17 +-
 .../flink/test/checkpointing/SavepointITCase.java  |  4 +--
 .../utils/SavepointMigrationTestBase.java  |  3 +-
 9 files changed, 50 insertions(+), 36 deletions(-)

diff --git a/docs/_includes/generated/checkpointing_configuration.html 
b/docs/_includes/generated/checkpointing_configuration.html
index c8517a5..748bdb2 100644
--- a/docs/_includes/generated/checkpointing_configuration.html
+++ b/docs/_includes/generated/checkpointing_configuration.html
@@ -22,9 +22,9 @@
 
 
 state.backend.fs.memory-threshold
-1024
-Integer
-The minimum size of state data files. All state chunks smaller 
than that are stored inline in the root checkpoint metadata file.
+20 kb
+MemorySize
+The minimum size of state data files. All state chunks smaller 
than that are stored inline in the root checkpoint metadata file. The max 
memory threshold for this configuration is 1MB.
 
 
 state.backend.fs.write-buffer-size
diff --git a/docs/_includes/generated/expert_state_backends_section.html 
b/docs/_includes/generated/expert_state_backends_section.html
index 9d50be1..0fed867 100644
--- a/docs/_includes/generated/expert_state_backends_section.html
+++ b/docs/_includes/generated/expert_state_backends_section.html
@@ -16,9 +16,9 @@
 
 
 state.backend.fs.memory-threshold
-1024
-Integer
-The minimum size of state data files. All state chunks smaller 
than that are stored inline in the root checkpoint metadata file.
+20 kb
+MemorySize
+The minimum size of state data files. All state chunks smaller 
than that are stored inline in the root checkpoint metadata file. The max 
memory threshold for this configuration is 1MB.
 
 
 state.backend.fs.write-buffer-size
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
 
b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
index df19ab9..16eaf75 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java
@@ -140,11 +140,12 @@ public class CheckpointingOptions {
/** The minimum size of state data files. All state chunks smaller than 
that
 * are stored inline in the root checkpoint metadata file. */
@Documentation.Section(Documentation.Sections.EXPERT_STATE_BACKENDS)
-   public static final ConfigOption FS_SMALL_FILE_THRESHOLD = 
ConfigOptions
+   public static final ConfigOption FS_SMALL_FILE_THRESHOLD = 
ConfigOptions
.key("state.backend.fs.memory-threshold")
-   .defaultValue(1024)
+   .memoryType()
+   .defaultValue(MemorySize.parse("20kb"))
.withDescription("The minimum size of state data files. 
All state chunks smaller than that are stored" +
-   " inline in the root checkpoint metadata 
file.");
+   " inline in the root checkpoint metadata file. 
The max memory threshold for this configuration is 1MB.");
 
/**
 * The default size of the write buffer for the checkpoint streams that 
write to file systems.
diff --git 
a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SavepointOutputFormat.java
 
b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SavepointOutputFormat.java
index 8235067..bebd435 100644
--- 
a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/SavepointOutputFormat.java
+++ 
b/flink-libraries/flink-s

[flink] branch master updated (1386e0d -> f3088e0)

2020-05-28 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 1386e0d  [FLINK-17958][core] Fix MathUtils#divideRoundUp bug for 
handling zero / negative values.
 add f3088e0  [FLINK-17610][state] Align the behavior of result of internal 
map state to return empty iterator

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/flink/runtime/state/heap/HeapMapState.java   | 9 +
 .../org/apache/flink/runtime/state/StateBackendTestBase.java | 6 --
 .../runtime/state/ttl/TtlMapStateAllEntriesTestContext.java  | 7 +++
 .../org/apache/flink/runtime/state/ttl/TtlStateTestBase.java | 9 +
 .../apache/flink/runtime/state/ttl/TtlStateTestContextBase.java  | 6 ++
 .../apache/flink/contrib/streaming/state/RocksDBMapState.java| 9 +
 .../contrib/streaming/state/ttl/RocksDBTtlStateTestBase.java | 5 +++--
 7 files changed, 31 insertions(+), 20 deletions(-)



[flink] branch release-1.11 updated (2cac04d -> f3733905)

2020-05-28 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 2cac04d  [FLINK-17958][core] Fix MathUtils#divideRoundUp bug for 
handling zero / negative values.
 add f3733905 [FLINK-17610][state] Align the behavior of result of internal 
map state to return empty iterator

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/flink/runtime/state/heap/HeapMapState.java   | 9 +
 .../org/apache/flink/runtime/state/StateBackendTestBase.java | 6 --
 .../runtime/state/ttl/TtlMapStateAllEntriesTestContext.java  | 7 +++
 .../org/apache/flink/runtime/state/ttl/TtlStateTestBase.java | 9 +
 .../apache/flink/runtime/state/ttl/TtlStateTestContextBase.java  | 6 ++
 .../apache/flink/contrib/streaming/state/RocksDBMapState.java| 9 +
 .../contrib/streaming/state/ttl/RocksDBTtlStateTestBase.java | 5 +++--
 7 files changed, 31 insertions(+), 20 deletions(-)



svn commit: r39754 - in /dev/flink/flink-1.11.0-rc1: ./ python/

2020-05-24 Thread liyu
Author: liyu
Date: Mon May 25 03:08:23 2020
New Revision: 39754

Log:
release-1.11.0-rc1

Added:
dev/flink/flink-1.11.0-rc1/
dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.11.0-rc1/flink-1.11.0-src.tgz   (with props)
dev/flink/flink-1.11.0-rc1/flink-1.11.0-src.tgz.asc   (with props)
dev/flink/flink-1.11.0-rc1/flink-1.11.0-src.tgz.sha512
dev/flink/flink-1.11.0-rc1/python/
dev/flink/flink-1.11.0-rc1/python/apache-flink-1.11.0.tar.gz   (with props)
dev/flink/flink-1.11.0-rc1/python/apache-flink-1.11.0.tar.gz.asc   (with 
props)
dev/flink/flink-1.11.0-rc1/python/apache-flink-1.11.0.tar.gz.sha512

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp35-cp35m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp35-cp35m-macosx_10_6_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp36-cp36m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp36-cp36m-macosx_10_9_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp37-cp37m-linux_x86_64.whl.sha512

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.asc
   (with props)

dev/flink/flink-1.11.0-rc1/python/apache_flink-1.11.0-cp37-cp37m-macosx_10_9_x86_64.whl.sha512

Added: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.11.tgz.sha512 Mon May 
25 03:08:23 2020
@@ -0,0 +1 @@
+c7364288b18d3655d5767a9e6676bb0956e913ed550154557433d429ab5031a74533395316569d270318c9f0f44ac7fb57d5f0d6dd5dbf10541d598d509ffe45
  flink-1.11.0-bin-scala_2.11.tgz

Added: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.11.0-rc1/flink-1.11.0-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.11.0-rc1/flink

[flink] branch release-1.11 updated: [FLINK-16074][docs-zh] Translate the Overview page for State & Fault Tolerance into Chinese

2020-05-21 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu 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 570df3a  [FLINK-16074][docs-zh] Translate the Overview page for State 
& Fault Tolerance into Chinese
570df3a is described below

commit 570df3a172f5a1c12d6647546b00136f3516636e
Author: klion26 
AuthorDate: Tue May 12 16:54:27 2020 +0800

[FLINK-16074][docs-zh] Translate the Overview page for State & Fault 
Tolerance into Chinese

This closes #12096.
---
 docs/dev/stream/state/index.zh.md | 20 
 1 file changed, 8 insertions(+), 12 deletions(-)

diff --git a/docs/dev/stream/state/index.zh.md 
b/docs/dev/stream/state/index.zh.md
index ab8d9ea..97e2d97 100644
--- a/docs/dev/stream/state/index.zh.md
+++ b/docs/dev/stream/state/index.zh.md
@@ -25,21 +25,17 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-In this section you will learn about the APIs that Flink provides for writing
-stateful programs. Please take a look at [Stateful Stream
-Processing]({% link concepts/stateful-stream-processing.zh.md %})
-to learn about the concepts behind stateful stream processing.
-
+你将在本节中了解到 Flink 提供的用于编写有状态程序的 API,想了解更多有状态流处理的概念,请查看[有状态的流处理]({% link 
concepts/stateful-stream-processing.zh.md %})
 {% top %}
 
-Where to go next?
+接下来看什么?
 -
 
-* [Working with State](state.html): Shows how to use state in a Flink 
application and explains the different kinds of state.
-* [The Broadcast State Pattern](broadcast_state.html): Explains how to connect 
a broadcast stream with a non-broadcast stream and use state to exchange 
information between them. 
-* [Checkpointing](checkpointing.html): Describes how to enable and configure 
checkpointing for fault tolerance.
-* [Queryable State](queryable_state.html): Explains how to access state from 
outside of Flink during runtime.
-* [State Schema Evolution](schema_evolution.html): Shows how schema of state 
types can be evolved.
-* [Custom Serialization for Managed State](custom_serialization.html): 
Discusses how to implement custom serializers, especially for schema evolution.
+* [Working with State](state.html): 描述了如何在 Flink 应用程序中使用状态,以及不同类型的状态。
+* [The Broadcast State 模式](broadcast_state.html): 描述了如何将广播流和非广播流进行连接从而交换数据。
+* [Checkpointing](checkpointing.html): 介绍了如何开启和配置 checkpoint,以实现状态容错。
+* [Queryable State](queryable_state.html): 介绍了如何从外围访问 Flink 的状态。
+* [状态数据结构升级](schema_evolution.html): 介绍了状态数据结构升级相关的内容。
+* [Managed State 的自定义序列化器](custom_serialization.html): 
介绍了如何实现自定义的序列化器,尤其是如何支持状态数据结构升级。
 
 {% top %}



[flink] branch master updated (4d1a2f5 -> 25c151e)

2020-05-21 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 4d1a2f5  [FLINK-17474][parquet][hive] Parquet reader should be case 
insensitive for hive
 add 25c151e  [FLINK-16074][docs-zh] Translate the Overview page for State 
& Fault Tolerance into Chinese

No new revisions were added by this update.

Summary of changes:
 docs/dev/stream/state/index.zh.md | 20 
 1 file changed, 8 insertions(+), 12 deletions(-)



[flink] branch master updated (92f2ae1 -> 4cfd23a)

2020-05-18 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 92f2ae1  [FLINK-16094][docs-zh] Translate 
/dev/table/functions/udfs.zh.md into Chinese
 add 4cfd23a  [FLINK-16076][docs-zh] Translate "Queryable State" page into 
Chinese

No new revisions were added by this update.

Summary of changes:
 docs/dev/stream/state/queryable_state.zh.md | 197 +++-
 1 file changed, 75 insertions(+), 122 deletions(-)



[flink] branch master updated (8bc7442 -> f967bcb)

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

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


from 8bc7442  [FLINK-16999][table-runtime-blink] Add converters for all 
data types and conversion classes
 add 399519d  [FLINK-16075][docs-zh] Translate "The Broadcast State 
Pattern" page into Chinese
 add f967bcb  [FLINK-16075][docs-zh] modification based on klion26's review

No new revisions were added by this update.

Summary of changes:
 docs/dev/stream/state/broadcast_state.zh.md | 183 +++-
 1 file changed, 69 insertions(+), 114 deletions(-)



[flink-web] branch asf-site updated (798f215 -> 5189ce9)

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

liyu pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git.


from 798f215  Rebuild website
 new 61b3e95  Add Apache Flink release 1.10.1
 new e94acc8  Add special notice for incompatible changes
 new 5189ce9  Rebuild website

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:
 _config.yml |  58 +--
 _posts/2020-05-12-release-1.10.1.md | 392 +
 content/blog/index.html |  38 +-
 content/blog/page10/index.html  |  40 +-
 content/blog/page11/index.html  |  40 +-
 content/blog/page12/index.html  |  25 ++
 content/blog/page2/index.html   |  36 +-
 content/blog/page3/index.html   |  36 +-
 content/blog/page4/index.html   |  36 +-
 content/blog/page5/index.html   |  38 +-
 content/blog/page6/index.html   |  42 +-
 content/blog/page7/index.html   |  42 +-
 content/blog/page8/index.html   |  40 +-
 content/blog/page9/index.html   |  40 +-
 content/downloads.html  |  37 +-
 content/index.html  |   8 +-
 content/news/2020/05/12/release-1.10.1.html | 630 
 content/zh/downloads.html   |  41 +-
 content/zh/index.html   |   8 +-
 19 files changed, 1407 insertions(+), 220 deletions(-)
 create mode 100644 _posts/2020-05-12-release-1.10.1.md
 create mode 100644 content/news/2020/05/12/release-1.10.1.html



[flink-web] 01/03: Add Apache Flink release 1.10.1

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

liyu pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git

commit 61b3e95a39e3fdb345c5d7be2fac9a41048eb3cb
Author: Yu Li 
AuthorDate: Sat Apr 25 01:34:23 2020 +0800

Add Apache Flink release 1.10.1
---
 _config.yml |  58 +++---
 _posts/2020-05-12-release-1.10.1.md | 382 
 2 files changed, 413 insertions(+), 27 deletions(-)

diff --git a/_config.yml b/_config.yml
index 876e449..8ba0d26 100644
--- a/_config.yml
+++ b/_config.yml
@@ -9,7 +9,7 @@ url: https://flink.apache.org
 
 DOCS_BASE_URL: https://ci.apache.org/projects/flink/
 
-FLINK_VERSION_STABLE: 1.10.0
+FLINK_VERSION_STABLE: 1.10.1
 FLINK_VERSION_STABLE_SHORT: "1.10"
 
 FLINK_ISSUES_URL: https://issues.apache.org/jira/browse/FLINK
@@ -58,48 +58,48 @@ flink_releases:
   -
   version_short: "1.10"
   binary_release:
-  name: "Apache Flink 1.10.0"
+  name: "Apache Flink 1.10.1"
   scala_211:
-  id: "1100-download_211"
-  url: 
"https://www.apache.org/dyn/closer.lua/flink/flink-1.10.0/flink-1.10.0-bin-scala_2.11.tgz;
-  asc_url: 
"https://downloads.apache.org/flink/flink-1.10.0/flink-1.10.0-bin-scala_2.11.tgz.asc;
-  sha512_url: 
"https://downloads.apache.org/flink/flink-1.10.0/flink-1.10.0-bin-scala_2.11.tgz.sha512;
+  id: "1101-download_211"
+  url: 
"https://www.apache.org/dyn/closer.lua/flink/flink-1.10.1/flink-1.10.1-bin-scala_2.11.tgz;
+  asc_url: 
"https://downloads.apache.org/flink/flink-1.10.1/flink-1.10.1-bin-scala_2.11.tgz.asc;
+  sha512_url: 
"https://downloads.apache.org/flink/flink-1.10.1/flink-1.10.1-bin-scala_2.11.tgz.sha512;
   scala_212:
-  id: "1100-download_212"
-  url: 
"https://www.apache.org/dyn/closer.lua/flink/flink-1.10.0/flink-1.10.0-bin-scala_2.12.tgz;
-  asc_url: 
"https://downloads.apache.org/flink/flink-1.10.0/flink-1.10.0-bin-scala_2.12.tgz.asc;
-  sha512_url: 
"https://downloads.apache.org/flink/flink-1.10.0/flink-1.10.0-bin-scala_2.12.tgz.sha512;
+  id: "1101-download_212"
+  url: 
"https://www.apache.org/dyn/closer.lua/flink/flink-1.10.1/flink-1.10.1-bin-scala_2.12.tgz;
+  asc_url: 
"https://downloads.apache.org/flink/flink-1.10.1/flink-1.10.1-bin-scala_2.12.tgz.asc;
+  sha512_url: 
"https://downloads.apache.org/flink/flink-1.10.1/flink-1.10.1-bin-scala_2.12.tgz.sha512;
   source_release:
-  name: "Apache Flink 1.10.0"
-  id: "1100-download-source"
-  url: 
"https://www.apache.org/dyn/closer.lua/flink/flink-1.10.0/flink-1.10.0-src.tgz;
-  asc_url: 
"https://downloads.apache.org/flink/flink-1.10.0/flink-1.10.0-src.tgz.asc;
-  sha512_url: 
"https://downloads.apache.org/flink/flink-1.10.0/flink-1.10.0-src.tgz.sha512;
+  name: "Apache Flink 1.10.1"
+  id: "1101-download-source"
+  url: 
"https://www.apache.org/dyn/closer.lua/flink/flink-1.10.1/flink-1.10.1-src.tgz;
+  asc_url: 
"https://downloads.apache.org/flink/flink-1.10.1/flink-1.10.1-src.tgz.asc;
+  sha512_url: 
"https://downloads.apache.org/flink/flink-1.10.1/flink-1.10.1-src.tgz.sha512;
   optional_components:
 -
   name: "Avro SQL Format"
   category: "SQL Formats"
   scala_dependent: false
-  id: 1100-sql-format-avro
-  url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-avro/1.10.0/flink-avro-1.10.0.jar
-  asc_url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-avro/1.10.0/flink-avro-1.10.0.jar.asc
-  sha_url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-avro/1.10.0/flink-avro-1.10.0.jar.sha1
+  id: 1101-sql-format-avro
+  url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-avro/1.10.1/flink-avro-1.10.1.jar
+  asc_url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-avro/1.10.1/flink-avro-1.10.1.jar.asc
+  sha_url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-avro/1.10.1/flink-avro-1.10.1.jar.sha1
 -
   name: "CSV SQL Format"
   category: "SQL Formats"
   scala_dependent: false
-  id: 1100-sql-format-csv
-  url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-csv/1.10.0/flink-csv-1.10.0.jar
-  asc_url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink-csv/1.10.0/flink-csv-1.10.0.jar.asc
-  sha_url: 
https://repo.maven.apache.org/maven2/org/apache/flink/flink

[flink-web] 02/03: Add special notice for incompatible changes

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

liyu pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git

commit e94acc80cc9253d3783c98a353cbc40c243811c7
Author: Yu Li 
AuthorDate: Thu May 14 00:55:22 2020 +0800

Add special notice for incompatible changes
---
 _posts/2020-05-12-release-1.10.1.md | 10 ++
 1 file changed, 10 insertions(+)

diff --git a/_posts/2020-05-12-release-1.10.1.md 
b/_posts/2020-05-12-release-1.10.1.md
index bcfe4be..13435d3 100644
--- a/_posts/2020-05-12-release-1.10.1.md
+++ b/_posts/2020-05-12-release-1.10.1.md
@@ -15,6 +15,16 @@ This release includes 158 fixes and minor improvements for 
Flink 1.10.0. The lis
 
 We highly recommend all users to upgrade to Flink 1.10.1.
 
+
+ Note
+FLINK-16684 changed the builders of the StreamingFileSink to make them 
compilable in Scala. This change is source compatible but binary incompatible. 
If using the StreamingFileSink, please recompile your user code against 1.10.1 
before upgrading.
+
+
+
+ Note
+FLINK-16683 Flink no longer supports starting clusters with .bat scripts. 
Users should instead use environments like WSL or Cygwin and work with the .sh 
scripts.
+
+
 Updated Maven dependencies:
 
 ```xml



[flink-web] 03/03: Rebuild website

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

liyu pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/flink-web.git

commit 5189ce9d6018d9adbf6c9a18f56f742312b72fe3
Author: Yu Li 
AuthorDate: Wed May 13 11:32:47 2020 +0800

Rebuild website
---
 content/blog/index.html |  38 +-
 content/blog/page10/index.html  |  40 +-
 content/blog/page11/index.html  |  40 +-
 content/blog/page12/index.html  |  25 ++
 content/blog/page2/index.html   |  36 +-
 content/blog/page3/index.html   |  36 +-
 content/blog/page4/index.html   |  36 +-
 content/blog/page5/index.html   |  38 +-
 content/blog/page6/index.html   |  42 +-
 content/blog/page7/index.html   |  42 +-
 content/blog/page8/index.html   |  40 +-
 content/blog/page9/index.html   |  40 +-
 content/downloads.html  |  37 +-
 content/index.html  |   8 +-
 content/news/2020/05/12/release-1.10.1.html | 630 
 content/zh/downloads.html   |  41 +-
 content/zh/index.html   |   8 +-
 17 files changed, 984 insertions(+), 193 deletions(-)

diff --git a/content/blog/index.html b/content/blog/index.html
index 7a6814f..87dcbef 100644
--- a/content/blog/index.html
+++ b/content/blog/index.html
@@ -196,6 +196,21 @@
 
 
 
+  Apache Flink 1.10.1 
Released
+
+  12 May 2020
+   Yu Li (https://twitter.com/LiyuApache;>@LiyuApache)
+
+  The Apache Flink community released the first bugfix version of 
the Apache Flink 1.10 series.
+
+
+
+  Continue reading 

+
+
+
+
+
   Flink Community Update - 
May'20
 
   07 May 2020
@@ -319,19 +334,6 @@ This release marks a big milestone: Stateful Functions 2.0 
is not only an API up
 
 
 
-
-  Advanced Flink Application 
Patterns Vol.2: Dynamic Updates of Application Logic
-
-  24 Mar 2020
-   Alexander Fedulov (https://twitter.com/alex_fedulov;>@alex_fedulov)
-
-  In this series of blog posts you will learn about powerful Flink 
patterns for building streaming applications.
-
-  Continue 
reading 
-
-
-
-
 
 
 
@@ -364,6 +366,16 @@ This release marks a big milestone: Stateful Functions 2.0 
is not only an API up
 
 
   
+  Apache Flink 1.10.1 
Released
+
+  
+
+  
+
+  
+  
+
+  
   Flink Community 
Update - May'20
 
   
diff --git a/content/blog/page10/index.html b/content/blog/page10/index.html
index 371a414..6018c3d 100644
--- a/content/blog/page10/index.html
+++ b/content/blog/page10/index.html
@@ -196,6 +196,21 @@
 
 
 
+  Flink 0.10.1 released
+
+  27 Nov 2015
+  
+
+  Today, the Flink community released the first bugfix release of 
the 0.10 series of Flink.
+
+
+
+  Continue reading 

+
+
+
+
+
   Announcing Apache Flink 
0.10.0
 
   16 Nov 2015
@@ -338,21 +353,6 @@ release is a preview release that contains known 
issues.
 
 
 
-
-  March 2015 in the Flink 
community
-
-  07 Apr 2015
-  
-
-  March has been a busy month in the Flink community.
-
-
-
-  Continue reading 

-
-
-
-
 
 
 
@@ -385,6 +385,16 @@ release is a preview release that contains known 
issues.
 
 
   
+  Apache Flink 1.10.1 
Released
+
+  
+
+  
+
+  
+  
+
+  
   Flink Community 
Update - May'20
 
   
diff --git a/content/blog/page11/index.html b/content/blog/page11/index.html
index df981d4..67986ab 100644
--- a/content/blog/page11/index.html
+++ b/content/blog/page11/index.html
@@ -196,6 +196,21 @@
 
 
 
+  March 2015 in the Flink 
community
+
+  07 Apr 2015
+  
+
+  March has been a busy month in the Flink community.
+
+
+
+  Continue reading 

+
+
+
+
+
   Peeking 
into Apache Flink's Engine Room
 
   13 Mar 2015 by Fabian Hüske (https://twitter.com/;>@fhueske)
@@ -335,21 +350,6 @@ and offers a new API including definition of flexible 
windows.
 
 
 
-
-  Apache Flink 0.6.1 available
-
-  26 Sep 2014
-  
-
-  We are happy to announce the availability of Flink 0.6.1.
-
-
-
-  Continue reading 

-
-
-
-
 
 
 
@@ -382,6 +382,16 @@ and offers a new API including definition of flexible 
windows.
 
 
   
+  Apache Flink 1.10.1 
Released
+
+  
+
+  
+
+  
+  
+
+  
   Flink Community 
Update - May'20
 
   
diff --git a/content/blog/page12/index.html b/content/blog/page12/index.html
index e57b3e5..f6d6281 100644
--- a/content/blog/page12/index.html
+++ b/content/blog/page12/index.html
@@ -196,6 +196,21 @@
 
 
 
+  Apache Flink 0.6.1 available
+
+  26

[flink] annotated tag release-1.10.1 updated (c5915cf -> d7886a1)

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

liyu pushed a change to annotated tag release-1.10.1
in repository https://gitbox.apache.org/repos/asf/flink.git.


*** WARNING: tag release-1.10.1 was modified! ***

from c5915cf  (commit)
  to d7886a1  (tag)
 tagging b4af6fc070854112efcf1f71f05a76f78100523e (tag)
  length 987 bytes
  by Yu Li
  on Tue May 12 18:29:17 2020 +0800

- Log -
Apache Flink 1.10.1
-BEGIN PGP SIGNATURE-
Version: GnuPG v2.0.22 (GNU/Linux)

iQIcBAABAgAGBQJeunqJAAoJEJPAeQJ3GrdDgX8P/RiQm9gMdY+px8ibuOU47H7P
GZ4r8+BVuw13gMnD0pmeYDDW6g7hg8M2Et+Ke2XrpOpdw+ZxgbNOg7zsO8oTc5wP
myM8Yg/bij/8hNMuGCJ2+G1lXar8PI+p8dK1wN8ppi4ufhAPoSIHqIrXpkQhr3sc
zwJ5UeO7cB2BeZyLsmmtVQO40LFy7UA4BG89yYruWOzB9ki5aJ0YLEQ7S5n9xylY
3cpKHo60tIvPRAvtADtcqMoAhK9aQCsEMxjp+jFyQUTndgEZ2+sS42P7RP25an3E
0Tomff9n0HoW7lIkvDClkgMtK1qa7+2XQ8k2nsyARhFQCHuiGPdWu7ucZE9px9DI
WzYd/TAigClPfFP2lDQwZQuY/M8UX8VEYUuT4Jmv0arNcZjEG7YN/fdPU713MAHT
xFuh/E/axR4f0wZBGAaqbBzYPxGIeGLkfoyK6hmfGdsDUdADFcTVxJn9NPrqBHhx
ok0AUzFxnsWeB+r6TXTUCSVL/ulyMSEJzczrWf2EQFpQEGRRQfG3a6/oWJFrTmAN
EeB1ueW61WTJYQjWtmHd0IzPMX3h5i4NSDa5Qt4c/IzDlfBCwRg+fzZRyNLdL1lJ
rD0Y5jRuA2V9pUJF83Zs236a6X54mp8jg8HZQSdmnpNni0Mg3x9IpPaOwsW1i4YC
5F4NMxMDOGaNSewHFjNT
=B7Lv
-END PGP SIGNATURE-
---


No new revisions were added by this update.

Summary of changes:



[flink] annotated tag release-1.10.1-rc3 updated (c5915cf -> b4af6fc)

2020-05-07 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu pushed a change to annotated tag release-1.10.1-rc3
in repository https://gitbox.apache.org/repos/asf/flink.git.


*** WARNING: tag release-1.10.1-rc3 was modified! ***

from c5915cf  (commit)
  to b4af6fc  (tag)
 tagging c5915cf87f96e1c7ebd84ad00f7eabade7e7fe37 (commit)
 replaces pre-apache-rename
  by Yu Li
  on Thu May 7 14:00:03 2020 +0800

- Log -
release-1.10.1-rc3
-BEGIN PGP SIGNATURE-
Version: GnuPG v2.0.22 (GNU/Linux)

iQIcBAABAgAGBQJes6PoAAoJEJPAeQJ3GrdDY8MP/A2utEWKcpb7dAiWCE+7c6FH
281e58z6P17JemteUw5rfZLzUMd9v3EAWV/PY2zG+4Fbsz6WBU7LNcd0K5/44AuY
EkmFPBgWhtdCYYwj80B3A6UuFepEMwiRt+fNbOly0ue/L7m2YGLpdsnyW0BWR+dv
AYP3b1nV9VLPN7VmMleT04D9cz9W/RPmetyU4AVce71OPpS/4A8K66DElG47ZN60
mn+QfG6wzVXvB53t1wna1U21uIbufn9Z1apJWv2dcftcDsvDzRnxVLGl5MJVo70d
LgZmLyfU4mltQnAnnfoQN03ILYjjphe1Nr4YIXjOPVC9MdXGRclLo4mLPHufzTL3
FNqdQ/nwmaCIDkG2+hkEFKr9vl1Q+pFx9yvRNMXczSGxoSFIEmkK847vvButhOOf
AV0bTmAh6j91OhE2vEW7nnq45UKDxHCu4ytms3ZMlu01V0CtCMSE5OU8oJsdCA3A
RCYIMMTaJiqf2yUJkZ8tYVIQjMFxC6A5H6bTNqkkwtTVhgpxXWKloEET1PahI1CN
f8D9e8+B3v0uDQQzFChdQEPN6rAmT0pSvd0vpPCwJWoQBDmyzN4QiCCd4MWG7Ynx
hAWqCVD/ILPJxypgzeNW56jQh0TthBm8jF3rwUvG9TfMEIQ/XnW6wlf+cJOt/SaZ
jJ/faxdLFJz4NgApMW6M
=py4F
-END PGP SIGNATURE-
---


No new revisions were added by this update.

Summary of changes:



[flink] branch master updated (7a28dbe -> a7b2b5a)

2020-05-07 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

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


from 7a28dbe  [FLINK-12717][python] Add windows support for PyFlink (#11960)
 add a7b2b5a  [FLINK-16782][state] Avoid unnecessary check on expired entry 
when ReturnExpiredIfNotCleanedUp

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/flink/runtime/state/ttl/AbstractTtlDecorator.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)



svn commit: r39320 - /dev/flink/flink-1.10.1-rc3/

2020-05-07 Thread liyu
Author: liyu
Date: Thu May  7 11:30:40 2020
New Revision: 39320

Log:
Add flink-1.10.1-rc3

Added:
dev/flink/flink-1.10.1-rc3/
dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz   (with props)
dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.asc   (with props)
dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.sha512
dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.10.1-rc3/flink-1.10.1-src.tgz   (with props)
dev/flink/flink-1.10.1-rc3/flink-1.10.1-src.tgz.asc   (with props)
dev/flink/flink-1.10.1-rc3/flink-1.10.1-src.tgz.sha512

Added: dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.sha512
==
--- dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc3/apache-flink-1.10.1.tar.gz.sha512 Thu May  7 
11:30:40 2020
@@ -0,0 +1 @@
+aaa06d634afecf345cfc2db40596e69c35463f3b70edd2afb6d9879b802a1c8c524a7a78811f4273b310d3a08278fe3775c071ed812369cc5162593106961098
  apache-flink-1.10.1.tar.gz

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.11.tgz.sha512 Thu May  
7 11:30:40 2020
@@ -0,0 +1 @@
+d86f6c3acff35ba8552756ba41e61e928ff8457c2d92ada85c60a7998dbfa34741367e805c203584eac427db738c172322fb98956b8f0f8c55dc7208d8960fa3
  flink-1.10.1-bin-scala_2.11.tgz

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.sha512
==
--- dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc3/flink-1.10.1-bin-scala_2.12.tgz.sha512 Thu May  
7 11:30:40 2020
@@ -0,0 +1 @@
+a1bd474d57e864dc473c3f7b1522a6053e2d57d2375bf5d656c4894fde1324e57f5fe2fd4f42fe7597fcedd923d3cac391f82c71133fc4a527733d2694aa5689
  flink-1.10.1-bin-scala_2.12.tgz

Added: dev/flink/flink-1.10.1-rc3/flink-1.10.1-src.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc3/flink-1.10.1-src.tgz

[flink] branch master updated (51f373e -> 7d6aa34)

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

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


from 51f373e  [FLINK-16034][table-api, docs] Updat documentation with new 
Java Expression DSL (#11533)
 add 7d6aa34  [FLINK-17483][legal] Update 
flink-sql-connector-elasticsearch7 NOTICE file to correctly reflect bundled 
dependencies

No new revisions were added by this update.

Summary of changes:
 .../src/main/resources/META-INF/NOTICE| 4 
 1 file changed, 4 insertions(+)



[flink] branch master updated: [FLINK-16901][legal] Correctly handle the THIRD_PARTY_NOTICES file in kinesis connector bundled dependency

2020-05-01 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu 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 261e721  [FLINK-16901][legal] Correctly handle the THIRD_PARTY_NOTICES 
file in kinesis connector bundled dependency
261e721 is described below

commit 261e72119b69c4fc3e22d9bcdec50f6ca2fdc2e9
Author: Yu Li 
AuthorDate: Fri May 1 01:12:12 2020 +0800

[FLINK-16901][legal] Correctly handle the THIRD_PARTY_NOTICES file in 
kinesis connector bundled dependency

This closes #11964.
---
 flink-connectors/flink-connector-kinesis/pom.xml   |   8 +
 .../src/main/resources/META-INF/NOTICE | 226 +
 2 files changed, 234 insertions(+)

diff --git a/flink-connectors/flink-connector-kinesis/pom.xml 
b/flink-connectors/flink-connector-kinesis/pom.xml
index 2b2626c..d751d82 100644
--- a/flink-connectors/flink-connector-kinesis/pom.xml
+++ b/flink-connectors/flink-connector-kinesis/pom.xml
@@ -222,6 +222,14 @@ under the License.

org.apache.flink.kinesis.shaded.org.apache.http


+   
+   
+   
com.amazonaws:amazon-kinesis-producer
+   

+   
META-INF/THIRD_PARTY_NOTICES
+   

+   
+   



diff --git 
a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE 
b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE
index daffb7e..dc2371c 100644
--- 
a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE
+++ 
b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE
@@ -9,6 +9,7 @@ This project bundles the following dependencies under the 
Apache Software Licens
 - com.amazonaws:amazon-kinesis-client:1.11.2
 - com.amazonaws:amazon-kinesis-producer:0.13.1
 - com.amazonaws:aws-java-sdk-core:1.11.603
+- com.amazonaws:aws-java-sdk-dynamodb:1.11.603
 - com.amazonaws:aws-java-sdk-kinesis:1.11.603
 - com.amazonaws:aws-java-sdk-kms:1.11.603
 - com.amazonaws:aws-java-sdk-s3:1.11.603
@@ -22,3 +23,228 @@ This project bundles the following dependencies under the 
BSD license.
 See bundled license files for details.
 
 - com.google.protobuf:protobuf-java:2.6.1
+
+The Amazon Kinesis Producer Library includes http-parser, Copyright (c) 
Joyent, Inc. and other Node contributors, libc++, Copyright (c) 2003-2014, LLVM 
Project, and slf4j, Copyright (c) 2004-2013 QOS.ch, each of which is subject to 
the terms and conditions of the MIT license that states as follows:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy 
of this software and associated documentation files (the "Software"), to deal 
in the Software without restriction, including without limitation the rights to 
use, copy, modify, merge, publish, distribute, sublicense, and/or
+sell copies of the Software, and to permit persons to whom the Software is 
furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all 
copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 
SOFTWARE.
+
+The Amazon Kinesis Producer Library includes Protocol Buffers, Copyright (c) 
2014, Google Inc. (except for atomicops_internals_generic_gcc.h, which is 
Copyright (c) Red Hat Inc., atomicops_internals_aix.h, which is Copyright (c) 
Bloomberg Finance LP, and Andorid.mk, which is Copyright (c) The Android Open 
Source Project), base64, Copyright (c) 2013, Alfred Klomp, glog, Copyright (c) 
2008, Google Inc., libcxxrt, Copyright (c) 2010-2011 PathScale, Inc., and LLVM 
+ clang, Copyright (c) 2003 [...]
+
+Redistribution and use in source and binary forms, with or w

[flink] annotated tag release-1.10.1-rc2 updated (f92e8a9 -> 446b9cf)

2020-05-01 Thread liyu
This is an automated email from the ASF dual-hosted git repository.

liyu pushed a change to annotated tag release-1.10.1-rc2
in repository https://gitbox.apache.org/repos/asf/flink.git.


*** WARNING: tag release-1.10.1-rc2 was modified! ***

from f92e8a9  (commit)
  to 446b9cf  (tag)
 tagging f92e8a9d60ef664acd66230da43c6f0a1cd87adc (commit)
 replaces pre-apache-rename
  by Yu Li
  on Fri May 1 13:04:40 2020 +0800

- Log -
release-1.10.1-rc2
-BEGIN PGP SIGNATURE-
Version: GnuPG v2.0.22 (GNU/Linux)

iQIcBAABAgAGBQJeq63uAAoJEJPAeQJ3GrdDTHYQAK9RgCNIAHo4M4frVk3Cqu2z
jhEKOgbQ9z6T28z2Wuddt45khoSxy/1LNeeAW9awfVDzwUx50DDd9T/IypsbBDJ6
MxB4z4JiWBvgGddjUMvE4QqF8xEo/NAbN5wx39yKNohpDra/lgTHL5Bpmv9ay1u3
bqKD3QUggO4Jnjr0+BvdGuiHRZwTYXV+/8FnVGJj5LWS9+2zxpOt9iyBHgKan0fD
BCCqEX5WhNXRp6p6iOfr/cGkYAxnqMrd+7iRYFnMMWZMfBimAN6LTAPdJr/0p5un
EQuptrhu2JzbGZCP0OPzRD2UcswuvaVyjbrhSoHIciQN5qaO9qzH/csOKcYGmaV1
pzlG7mDuCuFDjdeSQMtwxueCNzD85weSW/26dsuj7kERLU6skej/UQPMgaCh42sj
kO3gJjIMvWMtPCYQnzTVTbl6MA3eVJsb6TngMyYzK1Ur73zYSBOapYsFN6BEXphQ
fP1GntWHfopsPr59ChO1HkL+fFWHmOaDzx3R7Um3mWbtU42LB32/vwrBHLHZs2JT
uD6Mi9gk45R1zpi6SmsKpZ1c0K0y2o3tY6pb7wfvfBjv0PGtN+chiGEUb/SV3FPW
r3x3nD4B7mtK/GadTiRDKTzQdeDS5X6Cniap8YmOr/1eLXwOIxl06HiJp+ci/u9N
mNMI+E7E00LGrhkxTqvy
=nElm
-END PGP SIGNATURE-
---


No new revisions were added by this update.

Summary of changes:



svn commit: r39220 - /dev/flink/flink-1.10.1-rc2/

2020-05-01 Thread liyu
Author: liyu
Date: Fri May  1 12:04:14 2020
New Revision: 39220

Log:
Add flink-1.10.1-rc2

Added:
dev/flink/flink-1.10.1-rc2/
dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz   (with props)
dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.asc   (with props)
dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.sha512
dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.10.1-rc2/flink-1.10.1-src.tgz   (with props)
dev/flink/flink-1.10.1-rc2/flink-1.10.1-src.tgz.asc   (with props)
dev/flink/flink-1.10.1-rc2/flink-1.10.1-src.tgz.sha512

Added: dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.sha512
==
--- dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc2/apache-flink-1.10.1.tar.gz.sha512 Fri May  1 
12:04:14 2020
@@ -0,0 +1 @@
+3ffae5ae7ad7a4197afc64cf724e0b43452e62459409008e645916320c0d0d2b4874912a5f5b4bca40b764a5d4d03929a05bd887882e770bfe31b969d3bbe086
  apache-flink-1.10.1.tar.gz

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.11.tgz.sha512 Fri May  
1 12:04:14 2020
@@ -0,0 +1 @@
+e58e2a44a3d90a8a54d54329a94981fa91e0babcb25ae8cb2f74483d34e5b759750f92958e99fc326eeb48f1fec5035558e700c7f7be34a2297ceded8ff6f3c7
  flink-1.10.1-bin-scala_2.11.tgz

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.sha512
==
--- dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc2/flink-1.10.1-bin-scala_2.12.tgz.sha512 Fri May  
1 12:04:14 2020
@@ -0,0 +1 @@
+c1fdc2c80168324e753bebffb23555fa422c05a473920962ceecdb8d5fb42732696d8117fe9bb5b6832d7946a877a4b64983f2a92bed7a380f2daf10e778781d
  flink-1.10.1-bin-scala_2.12.tgz

Added: dev/flink/flink-1.10.1-rc2/flink-1.10.1-src.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc2/flink-1.10.1-src.tgz

[flink] branch release-1.10 updated (752e8a4 -> 2c5751b)

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

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


from 752e8a4  [FLINK-17471][python] Move the LICENSE and NOTICE file to the 
package root of the PyFlink source distribution. (#11956)
 add 54a1dbf  [FLINK-16901][legal] Correctly handle the THIRD_PARTY_NOTICES 
file in kinesis connector bundled dependency
 add 2c5751b  [FLINK-17483][legal] Update 
flink-sql-connector-elasticsearch7 NOTICE file to correctly reflect bundled 
dependencies

No new revisions were added by this update.

Summary of changes:
 flink-connectors/flink-connector-kinesis/pom.xml   |   8 +
 .../src/main/resources/META-INF/NOTICE | 226 +
 .../src/main/resources/META-INF/NOTICE |   4 +
 3 files changed, 238 insertions(+)



[flink] branch master updated (1fe57ab -> 473ce29)

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

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


from 1fe57ab  [FLINK-17373][table] Support the NULL type for function calls
 add 473ce29  [FLINK-14652][checkpointing] Refactor 
notifyCheckpointComplete to SubtaskCheckpointCoordinator

No new revisions were added by this update.

Summary of changes:
 .../flink/runtime/state/TestTaskStateManager.java  |  8 +-
 .../flink/streaming/runtime/tasks/StreamTask.java  | 14 +---
 .../tasks/SubtaskCheckpointCoordinator.java| 13 
 .../tasks/SubtaskCheckpointCoordinatorImpl.java| 15 
 .../MockSubtaskCheckpointCoordinatorBuilder.java   | 85 ++
 .../tasks/SubtaskCheckpointCoordinatorTest.java| 61 
 6 files changed, 182 insertions(+), 14 deletions(-)
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MockSubtaskCheckpointCoordinatorBuilder.java
 create mode 100644 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorTest.java



[flink] branch master updated (4aa9cf1 -> 39345e0)

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

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


from 4aa9cf1  [FLINK-17325][orc] Integrate orc to file system connector
 add 39345e0  [FLINK-13390][checkpointing] Clarify the exact meaning of 
state size when executing incremental checkpoint

No new revisions were added by this update.

Summary of changes:
 docs/_includes/generated/checkpointing_configuration.html | 2 +-
 docs/_includes/generated/common_state_backends_section.html   | 2 +-
 docs/monitoring/checkpoint_monitoring.md  | 4 ++--
 docs/monitoring/checkpoint_monitoring.zh.md   | 4 ++--
 docs/ops/state/state_backends.md  | 3 +++
 docs/ops/state/state_backends.zh.md   | 2 ++
 .../java/org/apache/flink/configuration/CheckpointingOptions.java | 8 ++--
 .../job/checkpoints/detail/job-checkpoints-detail.component.html  | 2 +-
 .../src/app/pages/job/checkpoints/job-checkpoints.component.html  | 8 
 .../checkpoints/subtask/job-checkpoints-subtask.component.html| 4 ++--
 .../runtime/rest/messages/checkpoints/CheckpointStatistics.java   | 6 ++
 .../rest/messages/checkpoints/CheckpointingStatistics.java| 6 ++
 .../rest/messages/checkpoints/SubtaskCheckpointStatistics.java| 6 ++
 .../rest/messages/checkpoints/TaskCheckpointStatistics.java   | 6 ++
 .../checkpoints/TaskCheckpointStatisticsWithSubtaskDetails.java   | 6 ++
 15 files changed, 54 insertions(+), 15 deletions(-)



[flink] annotated tag release-1.10.1-rc1 updated (84b74cc -> ee391fd)

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

liyu pushed a change to annotated tag release-1.10.1-rc1
in repository https://gitbox.apache.org/repos/asf/flink.git.


*** WARNING: tag release-1.10.1-rc1 was modified! ***

from 84b74cc  (commit)
  to ee391fd  (tag)
 tagging 84b74cc0e21981bf6feceb74b48d7a9d3e215dc5 (commit)
 replaces pre-apache-rename
  by Yu Li
  on Fri Apr 24 15:57:51 2020 +0800

- Log -
release-1.10.1-rc1
-BEGIN PGP SIGNATURE-
Version: GnuPG v2.0.22 (GNU/Linux)

iQIcBAABAgAGBQJeopv/AAoJEJPAeQJ3GrdDacgP/0GlJzrzRLIf3n4oVTr7ZpCl
m769HJCepjWoo5IlRl7PI5f7xdtRsB+145kOOvMAoru4X7GZzCLLoY6ZxDEIEqv6
xt/1SW78LxqqnfZdHBoMLM4IFzW1JWHT4q2sUjM2GfDly2UPbgQKtOul7bCzzytN
bEo9KwOGcsKmsT3RyVZ4ybJ5TGZ+aQkTSjPbwKbzUXhj62OlCCeHDvpovo6z70zW
EN3ZSNe/cRqDBpFOSQ72cCL1frNOt6g0Y4ip4ivEZSN9LG1HWCa5p9c0MsU1wUzW
hYLm+hjFT0AzMhiD1Yxe6oHzE/RlNs7WedCWwHs6SHPamMwRfhnLKnj2Ua0NJXmK
rCWYpPgfp0qfm7ti5hmUa75UxGzCbCnV0CD8UYsXVP5LBAK40khhP3QM2aQeBSzR
JHGykcNfKjqmuloZV06QjfljrRsARpMm0aEb7WzLMQdxpjp1EYUuFBKGaF0QAiZx
fAF0JYvsEjKu6r1t4sDlDUd6iOZsfIYAkrTRB3qkkg09Bk7+RuwjRM+53jTF1TwY
lJmhR6PPrhGmdFBrMRp+0PLNXUofClqFcqYI/6zbfoI5zcUd5Ix0cIVRMa2cqJ7D
IpD+VxxeqyuTgFWD/AW3WcTcCNRDJkXMQcR6Am2fODfQxTgUs0L7+ftS7lGPjMYb
cP8nDLVJWZQAtnHmpC9j
=N9Qu
-END PGP SIGNATURE-
---


No new revisions were added by this update.

Summary of changes:



svn commit: r39106 - /dev/flink/flink-1.10.1-rc1/

2020-04-24 Thread liyu
Author: liyu
Date: Fri Apr 24 16:32:13 2020
New Revision: 39106

Log:
Add flink-1.10.1-rc1

Added:
dev/flink/flink-1.10.1-rc1/
dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz   (with props)
dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.asc   (with props)
dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.sha512
dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz   (with props)
dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.asc   (with 
props)
dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.sha512
dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz   (with props)
dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.asc   (with 
props)
dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.sha512
dev/flink/flink-1.10.1-rc1/flink-1.10.1-src.tgz   (with props)
dev/flink/flink-1.10.1-rc1/flink-1.10.1-src.tgz.asc   (with props)
dev/flink/flink-1.10.1-rc1/flink-1.10.1-src.tgz.sha512

Added: dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.sha512
==
--- dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc1/apache-flink-1.10.1.tar.gz.sha512 Fri Apr 24 
16:32:13 2020
@@ -0,0 +1 @@
+2aa83b6cf9888bae6215e376379ab4bd3f9370b708e1bb4b6e526f4099116777b66c8aaad77986a02560999dbe93d0f3079c5903b365322fba085eb76e4afb98
  apache-flink-1.10.1.tar.gz

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.sha512
==
--- dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.11.tgz.sha512 Fri Apr 
24 16:32:13 2020
@@ -0,0 +1 @@
+6fcbc3a9e3aa86e21c6a63ac66bb2522fb826e305ff72b77172fde70d7c5a000a2a94fe0cdd3d8b1d7fb2d2367eec0a8105cbfddc86c48aceabbfeece151c92a
  flink-1.10.1-bin-scala_2.11.tgz

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz
--
svn:mime-type = application/x-gzip

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.asc
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.asc
--
svn:mime-type = application/pgp-signature

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.sha512
==
--- dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.sha512 (added)
+++ dev/flink/flink-1.10.1-rc1/flink-1.10.1-bin-scala_2.12.tgz.sha512 Fri Apr 
24 16:32:13 2020
@@ -0,0 +1 @@
+b71a0acff9c228c49df3fff0e011b8ee59e7f54e3436b43e0f0db7582676c2e6bc5dd11a8860e4b02125d4e8cd11fee64120022e1a961315ebd910f66d7543b1
  flink-1.10.1-bin-scala_2.12.tgz

Added: dev/flink/flink-1.10.1-rc1/flink-1.10.1-src.tgz
==
Binary file - no diff available.

Propchange: dev/flink/flink-1.10.1-rc1/flink-1.10.1-src.tgz

[flink] 02/02: [FLINK-16576][state backends] Do not distribute KeyGroupsStateHandle which contains empty KeyGroupRange

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

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

commit e95eae275f21796f9983c4d8b0bc4be1a0483e94
Author: klion26 
AuthorDate: Tue Apr 14 12:06:43 2020 +0800

[FLINK-16576][state backends] Do not distribute KeyGroupsStateHandle which 
contains empty KeyGroupRange

This closes #11555.
---
 .../flink/runtime/state/KeyGroupsStateHandle.java  |  7 ++-
 .../flink/runtime/state/KeyedStateHandle.java  |  6 ++-
 .../runtime/state/KeyGroupsStateHandleTest.java| 59 ++
 3 files changed, 70 insertions(+), 2 deletions(-)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
index 8092f6c..f78ec51 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java
@@ -83,8 +83,13 @@ public class KeyGroupsStateHandle implements 
StreamStateHandle, KeyedStateHandle
 * @return key-group state over a range that is the intersection 
between this handle's key-group range and the
 *  provided key-group range.
 */
+   @Override
public KeyGroupsStateHandle getIntersection(KeyGroupRange 
keyGroupRange) {
-   return new 
KeyGroupsStateHandle(groupRangeOffsets.getIntersection(keyGroupRange), 
stateHandle);
+   KeyGroupRangeOffsets offsets = 
groupRangeOffsets.getIntersection(keyGroupRange);
+   if (offsets.getKeyGroupRange().getNumberOfKeyGroups() <= 0) {
+   return null;
+   }
+   return new KeyGroupsStateHandle(offsets, stateHandle);
}
 
@Override
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
index 704ec14..a048aeb 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateHandle.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.state;
 
+import javax.annotation.Nullable;
+
 /**
  * Base for the handles of the checkpointed states in keyed streams. When
  * recovering from failures, the handle will be passed to all tasks whose key
@@ -34,7 +36,9 @@ public interface KeyedStateHandle extends 
CompositeStateHandle {
 * Returns a state over a range that is the intersection between this
 * handle's key-group range and the provided key-group range.
 *
-* @param keyGroupRange The key group range to intersect with
+* @param keyGroupRange The key group range to intersect with,
+* will return null if the intersection of this handle's key-group and 
the provided key-group is empty.
 */
+   @Nullable
KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange);
 }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupsStateHandleTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupsStateHandleTest.java
new file mode 100644
index 000..ce6bf82
--- /dev/null
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupsStateHandleTest.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.runtime.state;
+
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * A test for {@link KeyGroupsStateHandle}
+ */
+public class KeyGroupsStateHandleTest {
+
+   @Test
+   public void testNonEmptyIntersection() {
+   KeyGroupRangeOffsets offsets = new KeyGroupRangeOffsets(0, 7);
+   byte[] dummy = new byte[10];
+   StreamStateHandle streamHandle = new 
ByteStreamSt

[flink] branch release-1.9 updated (58b58f4 -> e95eae2)

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

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


from 58b58f4  FLINK-16697][metrics][jmx] Disable rebinding
 new bb6de2d  [FLINK-16576][state backends] Fix the problem of wrong 
mapping between stateId and metaInfo in HeapRestoreOperation
 new e95eae2  [FLINK-16576][state backends] Do not distribute 
KeyGroupsStateHandle which contains empty KeyGroupRange

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:
 .../checkpoint/StateAssignmentOperation.java   |  32 +---
 .../flink/runtime/state/KeyGroupsStateHandle.java  |   7 +-
 .../flink/runtime/state/KeyedStateHandle.java  |   6 +-
 .../runtime/state/heap/HeapRestoreOperation.java   |   8 +-
 .../runtime/state/KeyGroupsStateHandleTest.java|  59 ++
 .../flink/runtime/state/StateBackendTestBase.java  | 209 ++---
 .../util/AbstractStreamOperatorTestHarness.java|  12 +-
 7 files changed, 225 insertions(+), 108 deletions(-)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupsStateHandleTest.java



[flink] 01/02: [FLINK-16576][state backends] Fix the problem of wrong mapping between stateId and metaInfo in HeapRestoreOperation

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

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

commit bb6de2ddb37287da0def0f9d81dbc4792512e97a
Author: klion26 
AuthorDate: Thu Apr 2 12:40:14 2020 +0800

[FLINK-16576][state backends] Fix the problem of wrong mapping between 
stateId and metaInfo in HeapRestoreOperation
---
 .../checkpoint/StateAssignmentOperation.java   |  32 +---
 .../runtime/state/heap/HeapRestoreOperation.java   |   8 +-
 .../flink/runtime/state/StateBackendTestBase.java  | 209 ++---
 .../util/AbstractStreamOperatorTestHarness.java|  12 +-
 4 files changed, 155 insertions(+), 106 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 fdb62eb..e8ec90a 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
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.checkpoint;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -461,10 +462,11 @@ public class StateAssignmentOperation {
/**
 * Extracts certain key group ranges from the given state handles and 
adds them to the collector.
 */
-   private static void extractIntersectingState(
-   Collection originalSubtaskStateHandles,
-   KeyGroupRange rangeToExtract,
-   List extractedStateCollector) {
+   @VisibleForTesting
+   public static void extractIntersectingState(
+   Collection 
originalSubtaskStateHandles,
+   KeyGroupRange rangeToExtract,
+   List extractedStateCollector) {
 
for (KeyedStateHandle keyedStateHandle : 
originalSubtaskStateHandles) {
 
@@ -620,26 +622,4 @@ public class StateAssignmentOperation {
newParallelism);
}
 
-   /**
-* Determine the subset of {@link KeyGroupsStateHandle 
KeyGroupsStateHandles} with correct
-* key group index for the given subtask {@link KeyGroupRange}.
-*
-* This is publicly visible to be used in tests.
-*/
-   public static List getKeyedStateHandles(
-   Collection keyedStateHandles,
-   KeyGroupRange subtaskKeyGroupRange) {
-
-   List subtaskKeyedStateHandles = new 
ArrayList<>(keyedStateHandles.size());
-
-   for (KeyedStateHandle keyedStateHandle : keyedStateHandles) {
-   KeyedStateHandle intersectedKeyedStateHandle = 
keyedStateHandle.getIntersection(subtaskKeyGroupRange);
-
-   if (intersectedKeyedStateHandle != null) {
-   
subtaskKeyedStateHandles.add(intersectedKeyedStateHandle);
-   }
-   }
-
-   return subtaskKeyedStateHandles;
-   }
 }
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapRestoreOperation.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapRestoreOperation.java
index b2ddbd1..90272b1 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapRestoreOperation.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapRestoreOperation.java
@@ -103,7 +103,6 @@ public class HeapRestoreOperation implements 
RestoreOperation {
@Override
public Void restore() throws Exception {
 
-   final Map kvStatesById = new 
HashMap<>();
registeredKVStates.clear();
registeredPQStates.clear();
 
@@ -148,6 +147,8 @@ public class HeapRestoreOperation implements 
RestoreOperation {
List restoredMetaInfos =

serializationProxy.getStateMetaInfoSnapshots();
 
+   final Map 
kvStatesById = new HashMap<>();
+

createOrCheckStateForMetaInfo(restoredMetaInfos, kvStatesById);
 
readStateHandleStateData(
@@ -198,9 +199,8 @@ public class HeapRestoreOperation implements 
RestoreOperation {

metaInfoSnapshot.getBackendStateType() + ".");
}
 
-   if (registeredState == null) {
-   kvStatesById.put(kvStatesById.size(), 
metaInfoSnapshot);
-   }
+   // always pu

[flink] branch release-1.10 updated (08d941d -> ac2be60)

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

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


from 08d941d  [FLINK-16822][sql-client] `table.xx` property set from CLI 
should also be set into TableEnvironment's TableConfig
 add 1e9d72d  [FLINK-16576][state backends] Fix the problem of wrong 
mapping between stateId and metaInfo in HeapRestoreOperation
 add ac2be60  [FLINK-16576][state backends] Do not distribute 
KeyGroupsStateHandle which contains empty KeyGroupRange

No new revisions were added by this update.

Summary of changes:
 .../checkpoint/StateAssignmentOperation.java   |  32 +---
 .../flink/runtime/state/KeyGroupsStateHandle.java  |   7 +-
 .../flink/runtime/state/KeyedStateHandle.java  |   6 +-
 .../runtime/state/heap/HeapRestoreOperation.java   |   8 +-
 .../runtime/state/KeyGroupsStateHandleTest.java|  59 ++
 .../flink/runtime/state/StateBackendTestBase.java  | 209 ++---
 .../util/AbstractStreamOperatorTestHarness.java|  12 +-
 7 files changed, 225 insertions(+), 108 deletions(-)
 create mode 100644 
flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupsStateHandleTest.java



  1   2   >