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/30 14:54:07 UTC

[flink] 02/18: [hotfix][cep] Split NFAStateSerializer methods into smaller ones.

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 8a8b8d80bab037bade0239547b6385d009a95168
Author: Dawid Wysakowicz <dw...@apache.org>
AuthorDate: Wed Jan 23 10:39:03 2019 +0100

    [hotfix][cep] Split NFAStateSerializer methods into smaller ones.
---
 .../apache/flink/cep/nfa/NFAStateSerializer.java   | 163 ++++++++++++---------
 1 file changed, 97 insertions(+), 66 deletions(-)

diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
index 7b83030..7d79a36 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
@@ -21,8 +21,6 @@ package org.apache.flink.cep.nfa;
 import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
 import org.apache.flink.cep.nfa.sharedbuffer.EventId;
 import org.apache.flink.cep.nfa.sharedbuffer.NodeId;
@@ -30,6 +28,7 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.types.StringValue;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -44,11 +43,12 @@ public class NFAStateSerializer extends TypeSerializerSingleton<NFAState> {
 
 	private static final long serialVersionUID = 2098282423980597010L;
 
-	private NFAStateSerializer() {
-	}
-
 	public static final NFAStateSerializer INSTANCE = new NFAStateSerializer();
 
+	private static final DeweyNumber.DeweyNumberSerializer VERSION_SERIALIZER = DeweyNumber.DeweyNumberSerializer.INSTANCE;
+	private static final NodeId.NodeIdSerializer NODE_ID_SERIALIZER = new NodeId.NodeIdSerializer();
+	private static final EventId.EventIdSerializer EVENT_ID_SERIALIZER = EventId.EventIdSerializer.INSTANCE;
+
 	@Override
 	public boolean isImmutableType() {
 		return false;
@@ -92,35 +92,12 @@ public class NFAStateSerializer extends TypeSerializerSingleton<NFAState> {
 		return -1;
 	}
 
-	private static final StringSerializer STATE_NAME_SERIALIZER = StringSerializer.INSTANCE;
-	private static final LongSerializer TIMESTAMP_SERIALIZER = LongSerializer.INSTANCE;
-	private static final DeweyNumber.DeweyNumberSerializer VERSION_SERIALIZER = DeweyNumber.DeweyNumberSerializer.INSTANCE;
-	private static final NodeId.NodeIdSerializer NODE_ID_SERIALIZER = new NodeId.NodeIdSerializer();
-	private static final EventId.EventIdSerializer EVENT_ID_SERIALIZER = EventId.EventIdSerializer.INSTANCE;
-
 	@Override
 	public void serialize(NFAState record, DataOutputView target) throws IOException {
 		serializeComputationStates(record.getPartialMatches(), target);
 		serializeComputationStates(record.getCompletedMatches(), target);
 	}
 
-	private void serializeComputationStates(Queue<ComputationState> states, DataOutputView target) throws IOException {
-		target.writeInt(states.size());
-		for (ComputationState computationState : states) {
-			STATE_NAME_SERIALIZER.serialize(computationState.getCurrentStateName(), target);
-			NODE_ID_SERIALIZER.serialize(computationState.getPreviousBufferEntry(), target);
-
-			VERSION_SERIALIZER.serialize(computationState.getVersion(), target);
-			TIMESTAMP_SERIALIZER.serialize(computationState.getStartTimestamp(), target);
-			if (computationState.getStartEventID() != null) {
-				target.writeByte(1);
-				EVENT_ID_SERIALIZER.serialize(computationState.getStartEventID(), target);
-			} else {
-				target.writeByte(0);
-			}
-		}
-	}
-
 	@Override
 	public NFAState deserialize(DataInputView source) throws IOException {
 		PriorityQueue<ComputationState> partialMatches = deserializeComputationStates(source);
@@ -128,27 +105,6 @@ public class NFAStateSerializer extends TypeSerializerSingleton<NFAState> {
 		return new NFAState(partialMatches, completedMatches);
 	}
 
-	private PriorityQueue<ComputationState> deserializeComputationStates(DataInputView source) throws IOException {
-		PriorityQueue<ComputationState> computationStates = new PriorityQueue<>(NFAState.COMPUTATION_STATE_COMPARATOR);
-
-		int computationStateNo = source.readInt();
-		for (int i = 0; i < computationStateNo; i++) {
-			String state = STATE_NAME_SERIALIZER.deserialize(source);
-			NodeId prevState = NODE_ID_SERIALIZER.deserialize(source);
-			DeweyNumber version = VERSION_SERIALIZER.deserialize(source);
-			long startTimestamp = TIMESTAMP_SERIALIZER.deserialize(source);
-
-			byte isNull = source.readByte();
-			EventId startEventId = null;
-			if (isNull == 1) {
-				startEventId = EVENT_ID_SERIALIZER.deserialize(source);
-			}
-
-			computationStates.add(ComputationState.createState(state, prevState, version, startTimestamp, startEventId));
-		}
-		return computationStates;
-	}
-
 	@Override
 	public NFAState deserialize(NFAState reuse, DataInputView source) throws IOException {
 		return deserialize(source);
@@ -165,22 +121,7 @@ public class NFAStateSerializer extends TypeSerializerSingleton<NFAState> {
 		target.writeInt(computationStateNo);
 
 		for (int i = 0; i < computationStateNo; i++) {
-			String state = STATE_NAME_SERIALIZER.deserialize(source);
-			STATE_NAME_SERIALIZER.serialize(state, target);
-			NodeId prevState = NODE_ID_SERIALIZER.deserialize(source);
-			NODE_ID_SERIALIZER.serialize(prevState, target);
-			DeweyNumber version = VERSION_SERIALIZER.deserialize(source);
-			VERSION_SERIALIZER.serialize(version, target);
-			long startTimestamp = TIMESTAMP_SERIALIZER.deserialize(source);
-			TIMESTAMP_SERIALIZER.serialize(startTimestamp, target);
-
-			byte isNull = source.readByte();
-			target.writeByte(isNull);
-
-			if (isNull == 1) {
-				EventId startEventId = EVENT_ID_SERIALIZER.deserialize(source);
-				EVENT_ID_SERIALIZER.serialize(startEventId, target);
-			}
+			copySingleComputationState(source, target);
 		}
 	}
 
@@ -196,14 +137,104 @@ public class NFAStateSerializer extends TypeSerializerSingleton<NFAState> {
 		return new NFAStateSerializerSnapshot();
 	}
 
+	private NFAStateSerializer() {
+	}
+
 	/**
 	 * Serializer configuration snapshot for compatibility and format evolution.
 	 */
 	@SuppressWarnings("WeakerAccess")
 	public static final class NFAStateSerializerSnapshot extends SimpleTypeSerializerSnapshot<NFAState> {
-
 		public NFAStateSerializerSnapshot() {
 			super(() -> INSTANCE);
 		}
 	}
+
+	/*
+		De/serialization methods
+	 */
+
+	private void serializeComputationStates(Queue<ComputationState> states, DataOutputView target) throws IOException {
+		target.writeInt(states.size());
+		for (ComputationState computationState : states) {
+			serializeSingleComputationState(computationState, target);
+		}
+	}
+
+	private PriorityQueue<ComputationState> deserializeComputationStates(DataInputView source) throws IOException {
+		PriorityQueue<ComputationState> computationStates = new PriorityQueue<>(NFAState.COMPUTATION_STATE_COMPARATOR);
+
+		int computationStateNo = source.readInt();
+		for (int i = 0; i < computationStateNo; i++) {
+			final ComputationState computationState = deserializeSingleComputationState(source);
+			computationStates.add(computationState);
+		}
+		return computationStates;
+	}
+
+	private void serializeSingleComputationState(
+			ComputationState computationState,
+			DataOutputView target) throws IOException {
+
+		StringValue.writeString(computationState.getCurrentStateName(), target);
+		NODE_ID_SERIALIZER.serialize(computationState.getPreviousBufferEntry(), target);
+		VERSION_SERIALIZER.serialize(computationState.getVersion(), target);
+		target.writeLong(computationState.getStartTimestamp());
+		serializeStartEvent(computationState.getStartEventID(), target);
+	}
+
+	private ComputationState deserializeSingleComputationState(DataInputView source) throws IOException {
+		String stateName = StringValue.readString(source);
+		NodeId prevState = NODE_ID_SERIALIZER.deserialize(source);
+		DeweyNumber version = VERSION_SERIALIZER.deserialize(source);
+		long startTimestamp = source.readLong();
+
+		EventId startEventId = deserializeStartEvent(source);
+
+		return ComputationState.createState(stateName,
+			prevState,
+			version,
+			startTimestamp,
+			startEventId);
+	}
+
+	private void copySingleComputationState(DataInputView source, DataOutputView target) throws IOException {
+		StringValue.copyString(source, target);
+		NodeId prevState = NODE_ID_SERIALIZER.deserialize(source);
+		NODE_ID_SERIALIZER.serialize(prevState, target);
+		DeweyNumber version = VERSION_SERIALIZER.deserialize(source);
+		VERSION_SERIALIZER.serialize(version, target);
+		long startTimestamp = source.readLong();
+		target.writeLong(startTimestamp);
+
+		copyStartEvent(source, target);
+	}
+
+	private void serializeStartEvent(EventId startEventID, DataOutputView target) throws IOException {
+		if (startEventID != null) {
+			target.writeByte(1);
+			EVENT_ID_SERIALIZER.serialize(startEventID, target);
+		} else {
+			target.writeByte(0);
+		}
+	}
+
+	private EventId deserializeStartEvent(DataInputView source) throws IOException {
+		byte isNull = source.readByte();
+		EventId startEventId = null;
+		if (isNull == 1) {
+			startEventId = EVENT_ID_SERIALIZER.deserialize(source);
+		}
+		return startEventId;
+	}
+
+	private void copyStartEvent(DataInputView source, DataOutputView target) throws IOException {
+		byte isNull = source.readByte();
+		target.writeByte(isNull);
+
+		if (isNull == 1) {
+			EventId startEventId = EVENT_ID_SERIALIZER.deserialize(source);
+			EVENT_ID_SERIALIZER.serialize(startEventId, target);
+		}
+	}
 }