[hotfix] [core] Consolidate serializer duplication tests in StateDescriptorTest 
where possible


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7667ddcf
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7667ddcf
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7667ddcf

Branch: refs/heads/master
Commit: 7667ddcfb9c2cfa96ffdf7594affeb74719e8ccf
Parents: 87d31f5
Author: Stephan Ewen <[email protected]>
Authored: Tue Mar 20 16:46:13 2018 +0100
Committer: Stephan Ewen <[email protected]>
Committed: Thu Mar 22 15:46:00 2018 +0100

----------------------------------------------------------------------
 .../state/AggregatingStateDescriptorTest.java   | 62 --------------------
 .../common/state/ListStateDescriptorTest.java   |  1 -
 .../state/ReducingStateDescriptorTest.java      | 27 ---------
 .../api/common/state/StateDescriptorTest.java   | 30 ++++++++++
 .../common/state/ValueStateDescriptorTest.java  | 23 --------
 5 files changed, 30 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7667ddcf/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java
deleted file mode 100644
index f62acc8..0000000
--- 
a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.common.state;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.AggregateFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertNotSame;
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for the {@link AggregatingStateDescriptor}.
- */
-public class AggregatingStateDescriptorTest extends TestLogger {
-
-       /**
-        * FLINK-6775.
-        *
-        * <p>Tests that the returned serializer is duplicated. This allows to
-        * share the state descriptor.
-        */
-       @Test
-       public void testSerializerDuplication() {
-               // we need a serializer that actually duplicates for testing (a 
stateful one)
-               // we use Kryo here, because it meets these conditions
-               TypeSerializer<Long> serializer = new 
KryoSerializer<>(Long.class, new ExecutionConfig());
-
-               AggregateFunction<Long, Long, Long> aggregatingFunction = 
mock(AggregateFunction.class);
-
-               AggregatingStateDescriptor<Long, Long, Long> descr = new 
AggregatingStateDescriptor<>(
-                       "foobar",
-                       aggregatingFunction,
-                       serializer);
-
-               TypeSerializer<Long> serializerA = descr.getSerializer();
-               TypeSerializer<Long> serializerB = descr.getSerializer();
-
-               // check that the retrieved serializers are not the same
-               assertNotSame(serializerA, serializerB);
-       }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7667ddcf/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java
index e7e33e7..b934ee0 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java
@@ -68,7 +68,6 @@ public class ListStateDescriptorTest {
         * <p>Tests that the returned serializer is duplicated. This allows to
         * share the state descriptor.
         */
-       @SuppressWarnings("unchecked")
        @Test
        public void testSerializerDuplication() {
                // we need a serializer that actually duplicates for testing (a 
stateful one)

http://git-wip-us.apache.org/repos/asf/flink/blob/7667ddcf/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java
index 81b7c38..5d9eba5 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java
@@ -29,7 +29,6 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
 
 /**
  * Tests for the {@link ReducingStateDescriptor}.
@@ -57,30 +56,4 @@ public class ReducingStateDescriptorTest extends TestLogger {
                assertNotNull(copy.getSerializer());
                assertEquals(serializer, copy.getSerializer());
        }
-
-       /**
-        * FLINK-6775.
-        *
-        * <p>Tests that the returned serializer is duplicated. This allows to
-        * share the state descriptor.
-        */
-       @SuppressWarnings("unchecked")
-       @Test
-       public void testSerializerDuplication() {
-               // we need a serializer that actually duplicates for testing (a 
stateful one)
-               // we use Kryo here, because it meets these conditions
-               TypeSerializer<String> statefulSerializer = new 
KryoSerializer<>(String.class, new ExecutionConfig());
-
-               ReducingStateDescriptor<String> descr = new 
ReducingStateDescriptor<>(
-                               "foobar",
-                               (a, b) -> a,
-                               statefulSerializer);
-
-               TypeSerializer<String> serializerA = descr.getSerializer();
-               TypeSerializer<String> serializerB = descr.getSerializer();
-
-               // check that the retrieved serializers are not the same
-               assertNotSame(serializerA, serializerB);
-       }
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7667ddcf/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java
index 59293f4..cf5327e 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java
@@ -33,6 +33,7 @@ import java.io.File;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -41,6 +42,10 @@ import static org.junit.Assert.fail;
  */
 public class StateDescriptorTest {
 
+       // 
------------------------------------------------------------------------
+       //  Tests for serializer initialization
+       // 
------------------------------------------------------------------------
+
        @Test
        public void testInitializeWithSerializer() throws Exception {
                final TypeSerializer<String> serializer = 
StringSerializer.INSTANCE;
@@ -131,6 +136,31 @@ public class StateDescriptorTest {
        }
 
        // 
------------------------------------------------------------------------
+       //  Tests for serializer initialization
+       // 
------------------------------------------------------------------------
+
+       /**
+        * FLINK-6775, tests that the returned serializer is duplicated.
+        * This allows to share the state descriptor across threads.
+        */
+       @Test
+       public void testSerializerDuplication() throws Exception {
+               // we need a serializer that actually duplicates for testing (a 
stateful one)
+               // we use Kryo here, because it meets these conditions
+               TypeSerializer<String> statefulSerializer = new 
KryoSerializer<>(String.class, new ExecutionConfig());
+
+               TestStateDescriptor<String> descr = new 
TestStateDescriptor<>("foobar", statefulSerializer);
+
+               TypeSerializer<String> serializerA = descr.getSerializer();
+               TypeSerializer<String> serializerB = descr.getSerializer();
+
+               // check that the retrieved serializers are not the same
+               assertNotSame(serializerA, serializerB);
+       }
+
+       // 
------------------------------------------------------------------------
+       //  Mock implementations and test types
+       // 
------------------------------------------------------------------------
 
        private static class TestStateDescriptor<T> extends 
StateDescriptor<State, T> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7667ddcf/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java
index 7ee58fe..67114e5 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java
@@ -29,7 +29,6 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
 
 /**
  * Tests for the {@link ValueStateDescriptor}.
@@ -64,26 +63,4 @@ public class ValueStateDescriptorTest extends TestLogger {
                assertNotNull(copy.getSerializer());
                assertEquals(serializer, copy.getSerializer());
        }
-
-       /**
-        * FLINK-6775.
-        *
-        * <p>Tests that the returned serializer is duplicated. This allows to
-        * share the state descriptor.
-        */
-       @SuppressWarnings("unchecked")
-       @Test
-       public void testSerializerDuplication() {
-               // we need a serializer that actually duplicates for testing (a 
stateful one)
-               // we use Kryo here, because it meets these conditions
-               TypeSerializer<String> statefulSerializer = new 
KryoSerializer<>(String.class, new ExecutionConfig());
-
-               ValueStateDescriptor<String> descr = new 
ValueStateDescriptor<>("foobar", statefulSerializer);
-
-               TypeSerializer<String> serializerA = descr.getSerializer();
-               TypeSerializer<String> serializerB = descr.getSerializer();
-
-               // check that the retrieved serializers are not the same
-               assertNotSame(serializerA, serializerB);
-       }
 }

Reply via email to