You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/10/30 13:39:15 UTC

[GitHub] asfgit closed pull request #6909: [FLINK-8995][tests] Add keyed state that uses custom stateful seriali…

asfgit closed pull request #6909: [FLINK-8995][tests] Add keyed state that uses custom stateful seriali…
URL: https://github.com/apache/flink/pull/6909
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java
index 513d0cfc53a..70fdade4a1c 100644
--- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java
+++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java
@@ -29,6 +29,7 @@
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.tests.artificialstate.ComplexPayload;
+import org.apache.flink.streaming.tests.artificialstate.StatefulComplexPayloadSerializer;
 import org.apache.flink.streaming.tests.avro.ComplexPayloadAvro;
 import org.apache.flink.streaming.tests.avro.InnerPayLoadAvro;
 import org.apache.flink.util.Collector;
@@ -91,11 +92,12 @@ public static void main(String[] args) throws Exception {
 							}
 							return new ComplexPayload(event, KEYED_STATE_OPER_NAME);
 						},
-					Collections.singletonList(
-						new KryoSerializer<>(ComplexPayload.class, env.getConfig())), // custom KryoSerializer
+					Arrays.asList(
+						new KryoSerializer<>(ComplexPayload.class, env.getConfig()), // KryoSerializer
+						new StatefulComplexPayloadSerializer()), // custom stateful serializer
 					Collections.singletonList(ComplexPayload.class) // KryoSerializer via type extraction
 				)
-			).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Kryo");
+			).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Kryo_and_Custom_Stateful");
 
 		// add a keyed stateful map operator, which uses Avro for state serialization
 		eventStream = eventStream
diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java
new file mode 100644
index 00000000000..f29697824fb
--- /dev/null
+++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java
@@ -0,0 +1,169 @@
+/*
+ * 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.streaming.tests.artificialstate;
+
+import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A custom stateful serializer to test that serializers are not used concurrently.
+ */
+public class StatefulComplexPayloadSerializer extends TypeSerializer<ComplexPayload> {
+
+	private static final long serialVersionUID = 8766687317209282373L;
+
+	/** This holds the thread that currently has exclusive ownership over the serializer. */
+	private final AtomicReference<Thread> currentOwnerThread;
+
+	public StatefulComplexPayloadSerializer() {
+		this.currentOwnerThread = new AtomicReference<>(null);
+	}
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public TypeSerializer<ComplexPayload> duplicate() {
+		return new StatefulComplexPayloadSerializer();
+	}
+
+	@Override
+	public ComplexPayload createInstance() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public ComplexPayload copy(ComplexPayload from) {
+		try {
+			if (currentOwnerThread.compareAndSet(null, Thread.currentThread())) {
+				return InstantiationUtil.deserializeObject(
+					InstantiationUtil.serializeObject(from), Thread.currentThread().getContextClassLoader());
+			} else {
+				throw new IllegalStateException("Concurrent access to type serializer detected!");
+			}
+		} catch (Exception e) {
+			throw new RuntimeException(e);
+		} finally {
+			currentOwnerThread.set(null);
+		}
+	}
+
+	@Override
+	public ComplexPayload copy(ComplexPayload from, ComplexPayload reuse) {
+		return copy(from);
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void serialize(ComplexPayload record, DataOutputView target) throws IOException {
+		try {
+			if (currentOwnerThread.compareAndSet(null, Thread.currentThread())) {
+				target.write(InstantiationUtil.serializeObject(record));
+			} else {
+				throw new IllegalStateException("Concurrent access to type serializer detected!");
+			}
+		} finally {
+			currentOwnerThread.set(null);
+		}
+	}
+
+	@Override
+	public ComplexPayload deserialize(DataInputView source) throws IOException {
+		try (final DataInputViewStream inViewWrapper = new DataInputViewStream(source)) {
+			Thread currentThread = Thread.currentThread();
+			if (currentOwnerThread.compareAndSet(null, currentThread)) {
+				return InstantiationUtil.deserializeObject(
+					inViewWrapper,
+					currentThread.getContextClassLoader());
+			} else {
+				throw new IllegalStateException("Concurrent access to type serializer detected!");
+			}
+		} catch (ClassNotFoundException e) {
+			throw new IOException("Could not deserialize object.", e);
+		} finally {
+			currentOwnerThread.set(null);
+		}
+	}
+
+	@Override
+	public ComplexPayload deserialize(ComplexPayload reuse, DataInputView source) throws IOException {
+		return deserialize(source);
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		serialize(deserialize(source), target);
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		return obj == this;
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return getClass().equals(obj.getClass());
+	}
+
+	@Override
+	public int hashCode() {
+		return 42;
+	}
+
+	@Override
+	public TypeSerializerConfigSnapshot<ComplexPayload> snapshotConfiguration() {
+		// type serializer singletons should always be parameter-less
+		return new ParameterlessTypeSerializerConfig<>(getSerializationFormatIdentifier());
+	}
+
+	@Override
+	public CompatibilityResult<ComplexPayload> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
+		if (configSnapshot instanceof ParameterlessTypeSerializerConfig
+			&& isCompatibleSerializationFormatIdentifier(
+			((ParameterlessTypeSerializerConfig<?>) configSnapshot).getSerializationFormatIdentifier())) {
+
+			return CompatibilityResult.compatible();
+		} else {
+			return CompatibilityResult.requiresMigration();
+		}
+	}
+
+	private boolean isCompatibleSerializationFormatIdentifier(String identifier) {
+		return identifier.equals(getSerializationFormatIdentifier());
+	}
+
+	private String getSerializationFormatIdentifier() {
+		return getClass().getCanonicalName();
+	}
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services