This is an automated email from the ASF dual-hosted git repository. aljoscha pushed a commit to branch release-1.11 in repository https://gitbox.apache.org/repos/asf/flink.git
commit 32cd222ac80ec324e16533bf959f582d1d5acdd2 Author: Aljoscha Krettek <[email protected]> AuthorDate: Fri Jan 31 18:02:44 2020 +0100 [FLINK-13632] Port NFASerializer upgrade test to TypeSerializerUpgradeTestBase --- .../cep/nfa/sharedbuffer/SharedBufferEdge.java | 19 ++ .../cep/nfa/sharedbuffer/SharedBufferNode.java | 18 ++ .../cep/NFASerializerSnapshotsMigrationTest.java | 84 ----- .../apache/flink/cep/NFASerializerUpgradeTest.java | 356 +++++++++++++++++++++ .../flink-1.6-dewey-number-serializer-data | Bin 80 -> 0 bytes .../flink-1.6-dewey-number-serializer-snapshot | Bin 547 -> 0 bytes .../resources/flink-1.6-event-id-serializer-data | Bin 120 -> 0 bytes .../flink-1.6-event-id-serializer-snapshot | Bin 402 -> 0 bytes .../resources/flink-1.6-nfa-state-serializer-data | Bin 1460 -> 0 bytes .../flink-1.6-nfa-state-serializer-snapshot | Bin 362 -> 0 bytes .../resources/flink-1.6-node-id-serializer-data | Bin 250 -> 0 bytes .../flink-1.6-node-id-serializer-snapshot | Bin 398 -> 0 bytes .../flink-1.6-shared-buffer-edge-serializer-data | Bin 330 -> 0 bytes ...link-1.6-shared-buffer-edge-serializer-snapshot | Bin 438 -> 0 bytes .../flink-1.6-shared-buffer-node-serializer-data | Bin 370 -> 0 bytes ...link-1.6-shared-buffer-node-serializer-snapshot | Bin 775 -> 0 bytes .../flink-1.7-dewey-number-serializer-data | Bin 80 -> 0 bytes .../flink-1.7-dewey-number-serializer-snapshot | Bin 535 -> 0 bytes .../resources/flink-1.7-event-id-serializer-data | Bin 120 -> 0 bytes .../flink-1.7-event-id-serializer-snapshot | Bin 390 -> 0 bytes .../resources/flink-1.7-nfa-state-serializer-data | Bin 1460 -> 0 bytes .../flink-1.7-nfa-state-serializer-snapshot | Bin 350 -> 0 bytes .../resources/flink-1.7-node-id-serializer-data | Bin 250 -> 0 bytes .../flink-1.7-node-id-serializer-snapshot | Bin 386 -> 0 bytes .../flink-1.7-shared-buffer-edge-serializer-data | Bin 330 -> 0 bytes ...link-1.7-shared-buffer-edge-serializer-snapshot | Bin 426 -> 0 bytes .../flink-1.7-shared-buffer-node-serializer-data | Bin 370 -> 0 bytes ...link-1.7-shared-buffer-node-serializer-snapshot | Bin 763 -> 0 bytes 28 files changed, 393 insertions(+), 84 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferEdge.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferEdge.java index 43afdb2..5f551df 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferEdge.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferEdge.java @@ -28,6 +28,7 @@ import org.apache.flink.core.memory.DataOutputView; import java.io.IOException; import java.io.ObjectInputStream; +import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -66,6 +67,24 @@ public class SharedBufferEdge { '}'; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SharedBufferEdge that = (SharedBufferEdge) o; + return Objects.equals(target, that.target) && + Objects.equals(deweyNumber, that.deweyNumber); + } + + @Override + public int hashCode() { + return Objects.hash(target, deweyNumber); + } + /** Serializer for {@link SharedBufferEdge}. */ public static class SharedBufferEdgeSerializer extends TypeSerializerSingleton<SharedBufferEdge> { diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferNode.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferNode.java index 57e5738..60100b8 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferNode.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferNode.java @@ -30,6 +30,7 @@ import org.apache.flink.core.memory.DataOutputView; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -63,6 +64,23 @@ public class SharedBufferNode { '}'; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SharedBufferNode that = (SharedBufferNode) o; + return Objects.equals(edges, that.edges); + } + + @Override + public int hashCode() { + return Objects.hash(edges); + } + /** Serializer for {@link SharedBufferNode}. */ public static class SharedBufferNodeSerializer extends TypeSerializerSingleton<SharedBufferNode> { diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/NFASerializerSnapshotsMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/NFASerializerSnapshotsMigrationTest.java deleted file mode 100644 index b5074b2..0000000 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/NFASerializerSnapshotsMigrationTest.java +++ /dev/null @@ -1,84 +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.cep; - -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; -import org.apache.flink.cep.nfa.DeweyNumber; -import org.apache.flink.cep.nfa.NFAStateSerializer; -import org.apache.flink.cep.nfa.NFAStateSerializerSnapshot; -import org.apache.flink.cep.nfa.sharedbuffer.EventId; -import org.apache.flink.cep.nfa.sharedbuffer.NodeId; -import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferEdge; -import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferNode; -import org.apache.flink.testutils.migration.MigrationVersion; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; - -/** - * Migration tests for NFA-related serializers. - */ -@RunWith(Parameterized.class) -public class NFASerializerSnapshotsMigrationTest extends TypeSerializerSnapshotMigrationTestBase<Object> { - - public NFASerializerSnapshotsMigrationTest(TestSpecification<Object> testSpecification) { - super(testSpecification); - } - - @Parameterized.Parameters(name = "Test Specification = {0}") - public static Collection<TestSpecification<?>> testSpecifications() { - - final TestSpecifications testSpecifications = new TestSpecifications(MigrationVersion.v1_6, MigrationVersion.v1_7); - - testSpecifications.add( - "event-id-serializer", - EventId.EventIdSerializer.class, - EventId.EventIdSerializer.EventIdSerializerSnapshot.class, - () -> EventId.EventIdSerializer.INSTANCE); - testSpecifications.add( - "node-id-serializer", - NodeId.NodeIdSerializer.class, - NodeId.NodeIdSerializer.NodeIdSerializerSnapshot.class, - NodeId.NodeIdSerializer::new); - testSpecifications.add( - "dewey-number-serializer", - DeweyNumber.DeweyNumberSerializer.class, - DeweyNumber.DeweyNumberSerializer.DeweyNumberSerializerSnapshot.class, - () -> DeweyNumber.DeweyNumberSerializer.INSTANCE); - testSpecifications.add( - "shared-buffer-edge-serializer", - SharedBufferEdge.SharedBufferEdgeSerializer.class, - SharedBufferEdge.SharedBufferEdgeSerializer.SharedBufferEdgeSerializerSnapshot.class, - SharedBufferEdge.SharedBufferEdgeSerializer::new); - testSpecifications.add( - "shared-buffer-node-serializer", - SharedBufferNode.SharedBufferNodeSerializer.class, - SharedBufferNode.SharedBufferNodeSerializer.SharedBufferNodeSerializerSnapshot.class, - SharedBufferNode.SharedBufferNodeSerializer::new); - testSpecifications.add( - "nfa-state-serializer", - NFAStateSerializer.class, - NFAStateSerializerSnapshot.class, - NFAStateSerializer::new); - - return testSpecifications.get(); - } -} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/NFASerializerUpgradeTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/NFASerializerUpgradeTest.java new file mode 100644 index 0000000..26c19a9 --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/NFASerializerUpgradeTest.java @@ -0,0 +1,356 @@ +/* + * 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.cep; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase; +import org.apache.flink.cep.nfa.DeweyNumber; +import org.apache.flink.cep.nfa.NFAState; +import org.apache.flink.cep.nfa.NFAStateSerializer; +import org.apache.flink.cep.nfa.sharedbuffer.EventId; +import org.apache.flink.cep.nfa.sharedbuffer.NodeId; +import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferEdge; +import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferNode; +import org.apache.flink.testutils.migration.MigrationVersion; + +import org.hamcrest.Matcher; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; + +import static org.hamcrest.Matchers.is; + +/** + * Migration tests for NFA-related serializers. + */ +@RunWith(Parameterized.class) +public class NFASerializerUpgradeTest extends TypeSerializerUpgradeTestBase<Object, Object> { + + public NFASerializerUpgradeTest(TestSpecification<Object, Object> testSpecification) { + super(testSpecification); + } + + @Parameterized.Parameters(name = "Test Specification = {0}") + public static Collection<TestSpecification<?, ?>> testSpecifications() throws Exception { + ArrayList<TestSpecification<?, ?>> testSpecifications = new ArrayList<>(); + for (MigrationVersion migrationVersion : MIGRATION_VERSIONS) { + testSpecifications.add( + new TestSpecification<>( + "event-id-serializer", + migrationVersion, + EventIdSerializerSetup.class, + EventIdSerializerVerifier.class)); + testSpecifications.add( + new TestSpecification<>( + "node-id-serializer", + migrationVersion, + NodeIdSerializerSetup.class, + NodeIdSerializerVerifier.class)); + testSpecifications.add( + new TestSpecification<>( + "dewey-number-serializer", + migrationVersion, + DeweyNumberSerializerSetup.class, + DeweyNumberSerializerVerifier.class)); + testSpecifications.add( + new TestSpecification<>( + "shared-buffer-edge-serializer", + migrationVersion, + SharedBufferEdgeSerializerSetup.class, + SharedBufferEdgeSerializerVerifier.class)); + testSpecifications.add( + new TestSpecification<>( + "shared-buffer-node-serializer", + migrationVersion, + SharedBufferNodeSerializerSetup.class, + SharedBufferNodeSerializerVerifier.class)); + testSpecifications.add( + new TestSpecification<>( + "nfa-state-serializer", + migrationVersion, + NFAStateSerializerSetup.class, + NFAStateSerializerVerifier.class)); + } + + return testSpecifications; + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "event-id-serializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class EventIdSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<EventId> { + + @Override + public TypeSerializer<EventId> createPriorSerializer() { + return EventId.EventIdSerializer.INSTANCE; + } + + @Override + public EventId createTestData() { + return new EventId(42, 42L); + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class EventIdSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<EventId> { + + @Override + public TypeSerializer<EventId> createUpgradedSerializer() { + return EventId.EventIdSerializer.INSTANCE; + } + + @Override + public Matcher<EventId> testDataMatcher() { + return is(new EventId(42, 42L)); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<EventId>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "node-id-serializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class NodeIdSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<NodeId> { + + @Override + public TypeSerializer<NodeId> createPriorSerializer() { + return new NodeId.NodeIdSerializer(); + } + + @Override + public NodeId createTestData() { + return new NodeId(new EventId(42, 42L), "ciao"); + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class NodeIdSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<NodeId> { + + @Override + public TypeSerializer<NodeId> createUpgradedSerializer() { + return new NodeId.NodeIdSerializer(); + } + + @Override + public Matcher<NodeId> testDataMatcher() { + return is(new NodeId(new EventId(42, 42L), "ciao")); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<NodeId>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "dewey-number-serializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class DeweyNumberSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<DeweyNumber> { + + @Override + public TypeSerializer<DeweyNumber> createPriorSerializer() { + return DeweyNumber.DeweyNumberSerializer.INSTANCE; + } + + @Override + public DeweyNumber createTestData() { + return new DeweyNumber(42); + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class DeweyNumberSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<DeweyNumber> { + + @Override + public TypeSerializer<DeweyNumber> createUpgradedSerializer() { + return DeweyNumber.DeweyNumberSerializer.INSTANCE; + } + + @Override + public Matcher<DeweyNumber> testDataMatcher() { + return is(new DeweyNumber(42)); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<DeweyNumber>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "shared-buffer-edge-serializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class SharedBufferEdgeSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<SharedBufferEdge> { + + @Override + public TypeSerializer<SharedBufferEdge> createPriorSerializer() { + return new SharedBufferEdge.SharedBufferEdgeSerializer(); + } + + @Override + public SharedBufferEdge createTestData() { + return new SharedBufferEdge( + new NodeId(new EventId(42, 42L), "page"), + new DeweyNumber(42)); + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class SharedBufferEdgeSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<SharedBufferEdge> { + + @Override + public TypeSerializer<SharedBufferEdge> createUpgradedSerializer() { + return new SharedBufferEdge.SharedBufferEdgeSerializer(); + } + + @Override + public Matcher<SharedBufferEdge> testDataMatcher() { + return is(new SharedBufferEdge( + new NodeId(new EventId(42, 42L), "page"), + new DeweyNumber(42))); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<SharedBufferEdge>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "shared-buffer-node-serializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class SharedBufferNodeSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<SharedBufferNode> { + + @Override + public TypeSerializer<SharedBufferNode> createPriorSerializer() { + return new SharedBufferNode.SharedBufferNodeSerializer(); + } + + @Override + public SharedBufferNode createTestData() { + SharedBufferNode result = new SharedBufferNode(); + result.addEdge(new SharedBufferEdge( + new NodeId(new EventId(42, 42L), "page"), + new DeweyNumber(42))); + return result; + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class SharedBufferNodeSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<SharedBufferNode> { + + @Override + public TypeSerializer<SharedBufferNode> createUpgradedSerializer() { + return new SharedBufferNode.SharedBufferNodeSerializer(); + } + + @Override + public Matcher<SharedBufferNode> testDataMatcher() { + SharedBufferNode result = new SharedBufferNode(); + result.addEdge(new SharedBufferEdge( + new NodeId(new EventId(42, 42L), "page"), + new DeweyNumber(42))); + return is(result); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<SharedBufferNode>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "nfa-state-serializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class NFAStateSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<NFAState> { + + @Override + public TypeSerializer<NFAState> createPriorSerializer() { + return new NFAStateSerializer(); + } + + @Override + public NFAState createTestData() { + return new NFAState(Collections.emptyList()); + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class NFAStateSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<NFAState> { + + @Override + public TypeSerializer<NFAState> createUpgradedSerializer() { + return new NFAStateSerializer(); + } + + @Override + public Matcher<NFAState> testDataMatcher() { + return is(new NFAState(Collections.emptyList())); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<NFAState>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } +} diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-dewey-number-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-dewey-number-serializer-data deleted file mode 100644 index 75d3d21..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-dewey-number-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-dewey-number-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-dewey-number-serializer-snapshot deleted file mode 100644 index 027167e..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-dewey-number-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-event-id-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-event-id-serializer-data deleted file mode 100644 index 840f83d..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-event-id-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-event-id-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-event-id-serializer-snapshot deleted file mode 100644 index 0d1a700..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-event-id-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-nfa-state-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-nfa-state-serializer-data deleted file mode 100644 index 6505244..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-nfa-state-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-nfa-state-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-nfa-state-serializer-snapshot deleted file mode 100644 index 6d5fa94..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-nfa-state-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-node-id-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-node-id-serializer-data deleted file mode 100644 index 7a4f46e..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-node-id-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-node-id-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-node-id-serializer-snapshot deleted file mode 100644 index 4f9053e..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-node-id-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-edge-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-edge-serializer-data deleted file mode 100644 index 5979627..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-edge-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-edge-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-edge-serializer-snapshot deleted file mode 100644 index af0016b..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-edge-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-node-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-node-serializer-data deleted file mode 100644 index 8fe40f6..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-node-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-node-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-node-serializer-snapshot deleted file mode 100644 index 3071b83..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.6-shared-buffer-node-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-dewey-number-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.7-dewey-number-serializer-data deleted file mode 100644 index 75d3d21..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-dewey-number-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-dewey-number-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.7-dewey-number-serializer-snapshot deleted file mode 100644 index 3dae669..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-dewey-number-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-event-id-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.7-event-id-serializer-data deleted file mode 100644 index 5e4f6bf..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-event-id-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-event-id-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.7-event-id-serializer-snapshot deleted file mode 100644 index c824699..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-event-id-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-nfa-state-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.7-nfa-state-serializer-data deleted file mode 100644 index 6505244..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-nfa-state-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-nfa-state-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.7-nfa-state-serializer-snapshot deleted file mode 100644 index 3f928a0..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-nfa-state-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-node-id-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.7-node-id-serializer-data deleted file mode 100644 index ed21712..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-node-id-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-node-id-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.7-node-id-serializer-snapshot deleted file mode 100644 index ba4c1c7..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-node-id-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-edge-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-edge-serializer-data deleted file mode 100644 index 3546983..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-edge-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-edge-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-edge-serializer-snapshot deleted file mode 100644 index db33009..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-edge-serializer-snapshot and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-node-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-node-serializer-data deleted file mode 100644 index fb81606..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-node-serializer-data and /dev/null differ diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-node-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-node-serializer-snapshot deleted file mode 100644 index fab3567..0000000 Binary files a/flink-libraries/flink-cep/src/test/resources/flink-1.7-shared-buffer-node-serializer-snapshot and /dev/null differ
