You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tz...@apache.org on 2019/01/09 11:55:38 UTC

[flink] branch master updated (1e2aa8e -> cad0509)

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

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


    from 1e2aa8e  [FLINK-10436] Add ConfigOption#withFallbackKeys (#6872)
     new e5ed8c8  [FLINK-11287] [rocksdb] RocksDBListState should be using registered serializer in state meta infos
     new cad0509  [FLINK-11073] [state backends, tests] Activate ignored testKeyedListStateSerializerReconfiguration test in StateBackendMigrationTestBase

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:
 .../apache/flink/runtime/state/StateBackendMigrationTestBase.java | 2 --
 .../apache/flink/contrib/streaming/state/RocksDBListState.java    | 8 +++-----
 2 files changed, 3 insertions(+), 7 deletions(-)


[flink] 02/02: [FLINK-11073] [state backends, tests] Activate ignored testKeyedListStateSerializerReconfiguration test in StateBackendMigrationTestBase

Posted by tz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit cad0509a752be57eb995a95ccebf00012e443ed5
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
AuthorDate: Tue Jan 8 17:41:38 2019 +0100

    [FLINK-11073] [state backends, tests] Activate ignored testKeyedListStateSerializerReconfiguration test in StateBackendMigrationTestBase
    
    This test was previously ignored due to 2 missing changes:
    - ListSerializerSnapshot was not respecting serializer reconfiguration
      (fixed by FLINK-11073), and
    - RocksDBListState did not use the correct registered state serializer
      in backend's state meta info (fixed by FLINK-11287)
    
    With those fixes in, the test can now be activated and expected to pass.
---
 .../org/apache/flink/runtime/state/StateBackendMigrationTestBase.java   | 2 --
 1 file changed, 2 deletions(-)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java
index 8d85e74..b636ed0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java
@@ -40,7 +40,6 @@ import org.apache.flink.util.StateMigrationException;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -192,7 +191,6 @@ public abstract class StateBackendMigrationTestBase<B extends AbstractStateBacke
 	}
 
 	@Test
-	@Ignore("This currently doesn't pass because the ListSerializer doesn't respect the reconfigured case, yet.")
 	public void testKeyedListStateSerializerReconfiguration() throws Exception {
 		final String stateName = "test-name";
 


[flink] 01/02: [FLINK-11287] [rocksdb] RocksDBListState should be using registered serializer in state meta infos

Posted by tz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e5ed8c85139d1dfdb4b6a47a63ac36143a7b5c64
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
AuthorDate: Tue Jan 8 15:02:43 2019 +0100

    [FLINK-11287] [rocksdb] RocksDBListState should be using registered serializer in state meta infos
    
    This closes #7434.
---
 .../apache/flink/contrib/streaming/state/RocksDBListState.java    | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)

diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
index 13f5559..72a5bc6 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
@@ -19,7 +19,6 @@
 package org.apache.flink.contrib.streaming.state;
 
 import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -77,7 +76,6 @@ class RocksDBListState<K, N, V>
 	 * @param namespaceSerializer The serializer for the namespace.
 	 * @param valueSerializer The serializer for the state.
 	 * @param defaultValue The default value for the state.
-	 * @param elementSerializer The serializer for elements of the list state.
 	 * @param backend The backend for which this state is bind to.
 	 */
 	private RocksDBListState(
@@ -85,11 +83,12 @@ class RocksDBListState<K, N, V>
 			TypeSerializer<N> namespaceSerializer,
 			TypeSerializer<List<V>> valueSerializer,
 			List<V> defaultValue,
-			TypeSerializer<V> elementSerializer,
 			RocksDBKeyedStateBackend<K> backend) {
 
 		super(columnFamily, namespaceSerializer, valueSerializer, defaultValue, backend);
-		this.elementSerializer = elementSerializer;
+
+		ListSerializer<V> castedListSerializer = (ListSerializer<V>) valueSerializer;
+		this.elementSerializer = castedListSerializer.getElementSerializer();
 	}
 
 	@Override
@@ -281,7 +280,6 @@ class RocksDBListState<K, N, V>
 			registerResult.f1.getNamespaceSerializer(),
 			(TypeSerializer<List<E>>) registerResult.f1.getStateSerializer(),
 			(List<E>) stateDesc.getDefaultValue(),
-			((ListStateDescriptor<E>) stateDesc).getElementSerializer(),
 			backend);
 	}