flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8995) Add a test operator with keyed state that uses custom, stateful serializer
Date Wed, 24 Oct 2018 16:26:00 GMT

    [ https://issues.apache.org/jira/browse/FLINK-8995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16662491#comment-16662491
] 

ASF GitHub Bot commented on FLINK-8995:
---------------------------------------

tzulitai commented on a change in pull request #6909: [FLINK-8995][tests] Add keyed state
that uses custom stateful serialiā€¦
URL: https://github.com/apache/flink/pull/6909#discussion_r227863415
 
 

 ##########
 File path: 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());
 
 Review comment:
   Can avoid multiple invocations on `Thread.currentThread()`:
   
   ```
   Thread currentThread = Thread.currentThread();
   if (currentOwnerThread.compareAndSet(null, currentThread)) {
       return deserializeObject(serializeObject(from), currentThread.getContextClassLoader());
   }
   ```

----------------------------------------------------------------
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


> Add a test operator with keyed state that uses custom, stateful serializer
> --------------------------------------------------------------------------
>
>                 Key: FLINK-8995
>                 URL: https://issues.apache.org/jira/browse/FLINK-8995
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Tests
>    Affects Versions: 1.5.0
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> This test should figure out problems in places where multiple threads would share the
same serializer instead of properly duplicating it.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message