You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by dw...@apache.org on 2018/06/13 14:54:18 UTC

[01/10] flink git commit: [FLINK-8725] Reverted backward compatibility with <=1.5

Repository: flink
Updated Branches:
  refs/heads/master a00a39829 -> 975f9b1b8


[FLINK-8725] Reverted backward compatibility with <=1.5

This closes #5960


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

Branch: refs/heads/master
Commit: 45d54426948493c819534bf53d2210d4140270a6
Parents: 55cd059
Author: Dawid Wysakowicz <dw...@apache.org>
Authored: Fri May 4 15:41:27 2018 +0200
Committer: Dawid Wysakowicz <dw...@apache.org>
Committed: Wed Jun 13 14:59:58 2018 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/flink/cep/nfa/NFA.java | 270 ++++++++++++++++++-
 .../flink/cep/nfa/NFASerializationUtils.java    |  70 +++++
 .../flink/cep/nfa/NFAStateSerializer.java       |  47 +---
 .../AbstractKeyedCEPPatternOperator.java        |  29 +-
 .../flink/cep/operator/CEPMigrationTest.java    |   2 +-
 ...-migration-after-branching-flink1.4-snapshot | Bin 0 -> 19058 bytes
 ...-migration-after-branching-flink1.5-snapshot | Bin 17433 -> 19390 bytes
 .../cep-migration-conditions-flink1.4-snapshot  | Bin 0 -> 19503 bytes
 .../cep-migration-conditions-flink1.5-snapshot  | Bin 17117 -> 19835 bytes
 ...-single-pattern-afterwards-flink1.4-snapshot | Bin 0 -> 16848 bytes
 ...-single-pattern-afterwards-flink1.5-snapshot | Bin 16850 -> 17180 bytes
 ...ation-starting-new-pattern-flink1.4-snapshot | Bin 0 -> 18866 bytes
 ...ation-starting-new-pattern-flink1.5-snapshot | Bin 17241 -> 19198 bytes
 .../api/operators/AbstractStreamOperator.java   |   2 +
 14 files changed, 378 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 7324db3..bc28cc5 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -19,15 +19,28 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
+import org.apache.flink.api.common.typeutils.base.EnumSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler;
 import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.util.Preconditions;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -121,13 +134,11 @@ public class NFA<T> {
 	}
 
 	private State<T> getState(String state) {
-		State<T> result = states.get(state);
-		return result;
+		return states.get(state);
 	}
 
 	private State<T> getState(ComputationState<T> state) {
-		State<T> result = states.get(state.getState());
-		return result;
+		return states.get(state.getState());
 	}
 
 	private boolean isStartState(ComputationState<T> state) {
@@ -801,4 +812,255 @@ public class NFA<T> {
 			};
 		}
 	}
+
+	/**
+	 * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state.
+	 */
+	@Deprecated
+	public static final class NFASerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
+
+		private static final int VERSION = 1;
+
+		/** This empty constructor is required for deserializing the configuration. */
+		public NFASerializerConfigSnapshot() {}
+
+		public NFASerializerConfigSnapshot(
+			TypeSerializer<T> eventSerializer,
+			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
+
+			super(eventSerializer, sharedBufferSerializer);
+		}
+
+		@Override
+		public int getVersion() {
+			return VERSION;
+		}
+	}
+
+	/**
+	 * Only for backward compatibility with <1.5.
+	 */
+	@Deprecated
+	public static class NFASerializer<T> extends TypeSerializer<NFA<T>> {
+
+		private static final long serialVersionUID = 2098282423980597010L;
+
+		private final TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer;
+
+		private final TypeSerializer<T> eventSerializer;
+
+		public NFASerializer(TypeSerializer<T> typeSerializer) {
+			this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer));
+		}
+
+		NFASerializer(
+			TypeSerializer<T> typeSerializer,
+			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
+			this.eventSerializer = typeSerializer;
+			this.sharedBufferSerializer = sharedBufferSerializer;
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return false;
+		}
+
+		@Override
+		public NFASerializer<T> duplicate() {
+			return new NFASerializer<>(eventSerializer.duplicate());
+		}
+
+		@Override
+		public NFA<T> createInstance() {
+			return null;
+		}
+
+		@Override
+		public NFA<T> copy(NFA<T> from) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public NFA<T> copy(NFA<T> from, NFA<T> reuse) {
+			return copy(from);
+		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(NFA<T> record, DataOutputView target) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public NFA<T> deserialize(DataInputView source) throws IOException {
+			deserializeStates(source);
+			source.readLong();
+			source.readBoolean();
+
+			SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
+			Queue<ComputationState<T>> computationStates = NFASerializationUtils.deserializeComputationStates(eventSerializer, source);
+
+			return new DummyNFA<>(eventSerializer, computationStates, sharedBuffer);
+		}
+
+		/**
+		 * Dummy nfa just for backwards compatibility.
+		 */
+		public static class DummyNFA<T> extends NFA<T> {
+
+			Queue<ComputationState<T>> computationStates;
+			SharedBuffer<String, T> sharedBuffer;
+
+			public SharedBuffer<String, T> getSharedBuffer() {
+				return sharedBuffer;
+			}
+
+			public Queue<ComputationState<T>> getComputationStates() {
+				return computationStates;
+			}
+
+			DummyNFA(TypeSerializer<T> eventSerializer, Queue<ComputationState<T>> computationStates, SharedBuffer<String, T> sharedBuffer) {
+				super(eventSerializer, 0, false, Collections.emptyList());
+				this.sharedBuffer = sharedBuffer;
+				this.computationStates = computationStates;
+			}
+		}
+
+		@Override
+		public NFA<T> deserialize(NFA<T> reuse, DataInputView source) throws IOException {
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj == this ||
+				(obj != null && obj.getClass().equals(getClass()) &&
+					sharedBufferSerializer.equals(((NFASerializer) obj).sharedBufferSerializer) &&
+					eventSerializer.equals(((NFASerializer) obj).eventSerializer));
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return true;
+		}
+
+		@Override
+		public int hashCode() {
+			return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode();
+		}
+
+		@Override
+		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+			return new NFASerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer);
+		}
+
+		@Override
+		public CompatibilityResult<NFA<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+			if (configSnapshot instanceof NFASerializerConfigSnapshot) {
+				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
+					((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
+
+				CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
+						serializersAndConfigs.get(0).f0,
+						UnloadableDummyTypeSerializer.class,
+						serializersAndConfigs.get(0).f1,
+						eventSerializer);
+
+				CompatibilityResult<SharedBuffer<String, T>> sharedBufCompatResult =
+						CompatibilityUtil.resolveCompatibilityResult(
+								serializersAndConfigs.get(1).f0,
+								UnloadableDummyTypeSerializer.class,
+								serializersAndConfigs.get(1).f1,
+								sharedBufferSerializer);
+
+				if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
+					return CompatibilityResult.compatible();
+				} else {
+					if (eventCompatResult.getConvertDeserializer() != null &&
+						sharedBufCompatResult.getConvertDeserializer() != null) {
+						return CompatibilityResult.requiresMigration(
+							new NFASerializer<>(
+								new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()),
+								new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer())));
+					}
+				}
+			}
+
+			return CompatibilityResult.requiresMigration();
+		}
+
+		private Set<State<T>> deserializeStates(DataInputView in) throws IOException {
+			TypeSerializer<String> nameSerializer = StringSerializer.INSTANCE;
+			TypeSerializer<State.StateType> stateTypeSerializer = new EnumSerializer<>(State.StateType.class);
+			TypeSerializer<StateTransitionAction> actionSerializer = new EnumSerializer<>(StateTransitionAction.class);
+
+			final int noOfStates = in.readInt();
+			Map<String, State<T>> states = new HashMap<>(noOfStates);
+
+			for (int i = 0; i < noOfStates; i++) {
+				String stateName = nameSerializer.deserialize(in);
+				State.StateType stateType = stateTypeSerializer.deserialize(in);
+
+				State<T> state = new State<>(stateName, stateType);
+				states.put(stateName, state);
+			}
+
+			for (int i = 0; i < noOfStates; i++) {
+				String srcName = nameSerializer.deserialize(in);
+
+				int noOfTransitions = in.readInt();
+				for (int j = 0; j < noOfTransitions; j++) {
+					String src = nameSerializer.deserialize(in);
+					Preconditions.checkState(src.equals(srcName),
+							"Source Edge names do not match (" + srcName + " - " + src + ").");
+
+					String trgt = nameSerializer.deserialize(in);
+					StateTransitionAction action = actionSerializer.deserialize(in);
+
+					IterativeCondition<T> condition = null;
+					try {
+						condition = deserializeCondition(in);
+					} catch (ClassNotFoundException e) {
+						e.printStackTrace();
+					}
+
+					State<T> srcState = states.get(src);
+					State<T> trgtState = states.get(trgt);
+					srcState.addStateTransition(action, trgtState, condition);
+				}
+
+			}
+			return new HashSet<>(states.values());
+		}
+
+		private IterativeCondition<T> deserializeCondition(DataInputView in) throws IOException, ClassNotFoundException {
+			boolean hasCondition = in.readBoolean();
+			if (hasCondition) {
+				int length = in.readInt();
+
+				byte[] serCondition = new byte[length];
+				in.read(serCondition);
+
+				ByteArrayInputStream bais = new ByteArrayInputStream(serCondition);
+				ObjectInputStream ois = new ObjectInputStream(bais);
+
+				IterativeCondition<T> condition = (IterativeCondition<T>) ois.readObject();
+				ois.close();
+				bais.close();
+
+				return condition;
+			}
+			return null;
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java
new file mode 100644
index 0000000..612eb25
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java
@@ -0,0 +1,70 @@
+/*
+ * 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.nfa;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.DataInputView;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.Queue;
+
+class NFASerializationUtils {
+
+	/**
+	 * Deserializes {@link Queue} of {@link ComputationState}s. The queue is represented as count of states then follows
+	 * n instances of the computational state.
+	 *
+	 * @param eventSerializer event serializer for deserializing accepted events
+	 * @param source          view on the serialized data
+	 * @param <T>             type of processed events
+	 * @return queue of computation states
+	 */
+	static <T> Queue<ComputationState<T>> deserializeComputationStates(TypeSerializer<T> eventSerializer,
+			DataInputView source) throws IOException {
+		Queue<ComputationState<T>> computationStates = new LinkedList<>();
+		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
+		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
+		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+
+		int computationStateNo = source.readInt();
+		for (int i = 0; i < computationStateNo; i++) {
+			String state = stateNameSerializer.deserialize(source);
+			String prevState = stateNameSerializer.deserialize(source);
+			long timestamp = timestampSerializer.deserialize(source);
+			DeweyNumber version = versionSerializer.deserialize(source);
+			long startTimestamp = timestampSerializer.deserialize(source);
+			int counter = source.readInt();
+
+			T event = null;
+			if (source.readBoolean()) {
+				event = eventSerializer.deserialize(source);
+			}
+
+			computationStates.add(ComputationState.createState(
+				state, prevState, event, counter, timestamp, version, startTimestamp));
+		}
+		return computationStates;
+	}
+
+	private NFASerializationUtils() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
----------------------------------------------------------------------
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 4dae798..5b057e1 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
@@ -35,7 +35,6 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
 
@@ -135,30 +134,8 @@ public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
 	@Override
 	public NFAState<T> deserialize(DataInputView source) throws IOException {
 		SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
-
-		Queue<ComputationState<T>> computationStates = new LinkedList<>();
-		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
-		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
-
-		int computationStateNo = source.readInt();
-		for (int i = 0; i < computationStateNo; i++) {
-			String state = stateNameSerializer.deserialize(source);
-			String prevState = stateNameSerializer.deserialize(source);
-			long timestamp = timestampSerializer.deserialize(source);
-			DeweyNumber version = versionSerializer.deserialize(source);
-			long startTimestamp = timestampSerializer.deserialize(source);
-			int counter = source.readInt();
-
-			T event = null;
-			if (source.readBoolean()) {
-				event = eventSerializer.deserialize(source);
-			}
-
-			computationStates.add(ComputationState.createState(
-					state, prevState, event, counter, timestamp, version, startTimestamp));
-		}
-
+		Queue<ComputationState<T>> computationStates = NFASerializationUtils.deserializeComputationStates(
+			eventSerializer, source);
 		return new NFAState<>(computationStates, sharedBuffer, false);
 	}
 
@@ -234,20 +211,20 @@ public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
 	public CompatibilityResult<NFAState<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
 		if (configSnapshot instanceof NFAStateSerializerConfigSnapshot) {
 			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
-					((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
+				((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
 
 			CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					serializersAndConfigs.get(0).f0,
-					UnloadableDummyTypeSerializer.class,
-					serializersAndConfigs.get(0).f1,
-					eventSerializer);
+				serializersAndConfigs.get(0).f0,
+				UnloadableDummyTypeSerializer.class,
+				serializersAndConfigs.get(0).f1,
+				eventSerializer);
 
 			CompatibilityResult<SharedBuffer<String, T>> sharedBufCompatResult =
-					CompatibilityUtil.resolveCompatibilityResult(
-							serializersAndConfigs.get(1).f0,
-							UnloadableDummyTypeSerializer.class,
-							serializersAndConfigs.get(1).f1,
-							sharedBufferSerializer);
+				CompatibilityUtil.resolveCompatibilityResult(
+					serializersAndConfigs.get(1).f0,
+					UnloadableDummyTypeSerializer.class,
+					serializersAndConfigs.get(1).f1,
+					sharedBufferSerializer);
 
 			if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
 				return CompatibilityResult.compatible();

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index 312549f..7cdf732 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -34,6 +34,7 @@ import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.NFAState;
 import org.apache.flink.cep.nfa.NFAStateSerializer;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.runtime.state.KeyedStateFunction;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
@@ -78,7 +79,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 
 	///////////////			State			//////////////
 
-	private static final String NFA_OPERATOR_STATE_NAME = "nfaOperatorStateName";
+	private static final String NFA_STATE_NAME = "nfaStateName";
 	private static final String EVENT_QUEUE_STATE_NAME = "eventQueuesStateName";
 
 	private transient ValueState<NFAState<IN>> nfaValueState;
@@ -136,7 +137,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		if (nfaValueState == null) {
 			nfaValueState = getRuntimeContext().getState(
 				new ValueStateDescriptor<>(
-						NFA_OPERATOR_STATE_NAME,
+						NFA_STATE_NAME,
 						new NFAStateSerializer<>(inputSerializer)));
 		}
 
@@ -149,6 +150,30 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 					)
 			);
 		}
+
+		migrateOldState();
+	}
+
+	private void migrateOldState() throws Exception {
+		getKeyedStateBackend().applyToAllKeys(
+			VoidNamespace.INSTANCE,
+			VoidNamespaceSerializer.INSTANCE,
+			new ValueStateDescriptor<>(
+				"nfaOperatorStateName",
+				new NFA.NFASerializer<>(inputSerializer)
+			),
+			new KeyedStateFunction<Object, ValueState<NFA<IN>>>() {
+				@Override
+				public void process(Object key, ValueState<NFA<IN>> state) throws Exception {
+					NFA<IN> oldState = state.value();
+					if (oldState instanceof NFA.NFASerializer.DummyNFA) {
+						NFA.NFASerializer.DummyNFA<IN> dummyNFA = (NFA.NFASerializer.DummyNFA<IN>) oldState;
+						nfaValueState.update(new NFAState<>(dummyNFA.getComputationStates(), dummyNFA.getSharedBuffer(), false));
+						state.clear();
+					}
+				}
+			}
+		);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
index 3151498..9b120ca 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
@@ -71,7 +71,7 @@ public class CEPMigrationTest {
 
 	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
 	public static Collection<MigrationVersion> parameters () {
-		return Arrays.asList(MigrationVersion.v1_5);
+		return Arrays.asList(MigrationVersion.v1_3, MigrationVersion.v1_4, MigrationVersion.v1_5);
 	}
 
 	public CEPMigrationTest(MigrationVersion migrateVersion) {

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot
new file mode 100644
index 0000000..34c5110
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot
index 73934e5..4a2b607 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot
new file mode 100644
index 0000000..d4d3405
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot
index 3126ff4..f5e00be 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot
new file mode 100644
index 0000000..c87f307
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot
index 63b1b49..9018043 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot
new file mode 100644
index 0000000..660a21a
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot
index 1b1fa75..53819f2 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/45d54426/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index 9915dd5..c193416 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -396,6 +396,8 @@ public abstract class AbstractStreamOperator<OUT>
 		return snapshotInProgress;
 	}
 
+
+
 	/**
 	 * Stream operators with state, which want to participate in a snapshot need to override this hook method.
 	 *


[04/10] flink git commit: [FLINK-9418] Migrate SharedBuffer to use MapState

Posted by dw...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
index c94d739..88504e9 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
@@ -36,7 +35,8 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
-import static org.junit.Assert.assertTrue;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests for handling Events that are equal in case of {@link Object#equals(Object)} and have same timestamps.
@@ -45,7 +45,7 @@ import static org.junit.Assert.assertTrue;
 public class SameElementITCase extends TestLogger {
 
 	@Test
-	public void testEagerZeroOrMoreSameElement() {
+	public void testEagerZeroOrMoreSameElement() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -87,7 +87,7 @@ public class SameElementITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -138,15 +138,16 @@ public void testClearingBuffer() throws Exception {
 		}
 	});
 
-	NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+	NFA<Event> nfa = compile(pattern, false);
 
-	NFAState<Event> nfaState = nfa.createNFAState();
+	NFAState nfaState = nfa.createInitialNFAState();
 
 	List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 	compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 		Lists.newArrayList(a1, b1, c1, d)
 	));
-	assertTrue(nfaState.isEmpty());
+	assertEquals(1, nfaState.getComputationStates().size());
+	assertEquals("a", nfaState.getComputationStates().peek().getCurrentStateName());
 }
 
 @Test
@@ -182,9 +183,9 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 		}
 	});
 
-	NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+	NFA<Event> nfa = compile(pattern, false);
 
-	NFAState<Event> nfaState = nfa.createNFAState();
+	NFAState nfaState = nfa.createInitialNFAState();
 
 	List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 	compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -192,11 +193,12 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 		Lists.newArrayList(a1, d1, d2),
 		Lists.newArrayList(a1, d1)
 	));
-	assertTrue(nfaState.isEmpty());
+	assertEquals(1, nfaState.getComputationStates().size());
+	assertEquals("a", nfaState.getComputationStates().peek().getCurrentStateName());
 }
 
 	@Test
-	public void testZeroOrMoreSameElement() {
+	public void testZeroOrMoreSameElement() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -239,7 +241,7 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -320,7 +322,7 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -375,7 +377,7 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -418,7 +420,7 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 			}
 		}).oneOrMore().optional();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -481,7 +483,7 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
deleted file mode 100644
index 566e2b9..0000000
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java
+++ /dev/null
@@ -1,239 +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.nfa;
-
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.cep.Event;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for {@link SharedBuffer}.
- */
-public class SharedBufferTest extends TestLogger {
-
-	@Test
-	public void testSharedBuffer() {
-		SharedBuffer<String, Event> sharedBuffer = new SharedBuffer<>();
-		int numberEvents = 8;
-		Event[] events = new Event[numberEvents];
-		final long timestamp = 1L;
-
-		for (int i = 0; i < numberEvents; i++) {
-			events[i] = new Event(i + 1, "e" + (i + 1), i);
-		}
-
-		Map<String, List<Event>> expectedPattern1 = new HashMap<>();
-		expectedPattern1.put("a1", new ArrayList<Event>());
-		expectedPattern1.get("a1").add(events[2]);
-
-		expectedPattern1.put("a[]", new ArrayList<Event>());
-		expectedPattern1.get("a[]").add(events[3]);
-
-		expectedPattern1.put("b", new ArrayList<Event>());
-		expectedPattern1.get("b").add(events[5]);
-
-		Map<String, List<Event>> expectedPattern2 = new HashMap<>();
-		expectedPattern2.put("a1", new ArrayList<Event>());
-		expectedPattern2.get("a1").add(events[0]);
-
-		expectedPattern2.put("a[]", new ArrayList<Event>());
-		expectedPattern2.get("a[]").add(events[1]);
-		expectedPattern2.get("a[]").add(events[2]);
-		expectedPattern2.get("a[]").add(events[3]);
-		expectedPattern2.get("a[]").add(events[4]);
-
-		expectedPattern2.put("b", new ArrayList<Event>());
-		expectedPattern2.get("b").add(events[5]);
-
-		Map<String, List<Event>> expectedPattern3 = new HashMap<>();
-		expectedPattern3.put("a1", new ArrayList<Event>());
-		expectedPattern3.get("a1").add(events[0]);
-
-		expectedPattern3.put("a[]", new ArrayList<Event>());
-		expectedPattern3.get("a[]").add(events[1]);
-		expectedPattern3.get("a[]").add(events[2]);
-		expectedPattern3.get("a[]").add(events[3]);
-		expectedPattern3.get("a[]").add(events[4]);
-		expectedPattern3.get("a[]").add(events[5]);
-		expectedPattern3.get("a[]").add(events[6]);
-
-		expectedPattern3.put("b", new ArrayList<Event>());
-		expectedPattern3.get("b").add(events[7]);
-
-		sharedBuffer.put("a1", events[0], timestamp, null, null, 0, 0, DeweyNumber.fromString("1"));
-		sharedBuffer.put("a[]", events[1], timestamp, "a1", events[0], timestamp, 0, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("a1", events[2], timestamp, null, null, 0, 0, DeweyNumber.fromString("2"));
-		sharedBuffer.put("a[]", events[2], timestamp, "a[]", events[1], timestamp, 1, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("a[]", events[3], timestamp, "a[]", events[2], timestamp, 2, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("a[]", events[3], timestamp, "a1", events[2], timestamp, 0, DeweyNumber.fromString("2.0"));
-		sharedBuffer.put("a[]", events[4], timestamp, "a[]", events[3], timestamp, 3, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("b", events[5], timestamp, "a[]", events[4], timestamp, 4, DeweyNumber.fromString("1.0.0"));
-		sharedBuffer.put("a[]", events[5], timestamp, "a[]", events[4], timestamp, 4, DeweyNumber.fromString("1.1"));
-		sharedBuffer.put("b", events[5], timestamp, "a[]", events[3], timestamp, 1, DeweyNumber.fromString("2.0.0"));
-		sharedBuffer.put("a[]", events[6], timestamp, "a[]", events[5], timestamp, 5, DeweyNumber.fromString("1.1"));
-		sharedBuffer.put("b", events[7], timestamp, "a[]", events[6], timestamp, 6, DeweyNumber.fromString("1.1.0"));
-
-		Collection<Map<String, List<Event>>> patterns3 = sharedBuffer.extractPatterns("b", events[7], timestamp, 7, DeweyNumber.fromString("1.1.0"));
-		sharedBuffer.release("b", events[7], timestamp, 7);
-		Collection<Map<String, List<Event>>> patterns4 = sharedBuffer.extractPatterns("b", events[7], timestamp, 7, DeweyNumber.fromString("1.1.0"));
-
-		Collection<Map<String, List<Event>>> patterns1 = sharedBuffer.extractPatterns("b", events[5], timestamp, 2, DeweyNumber.fromString("2.0.0"));
-		Collection<Map<String, List<Event>>> patterns2 = sharedBuffer.extractPatterns("b", events[5], timestamp, 5, DeweyNumber.fromString("1.0.0"));
-		sharedBuffer.release("b", events[5], timestamp, 2);
-		sharedBuffer.release("b", events[5], timestamp, 5);
-
-		assertEquals(1L, patterns3.size());
-		assertEquals(0L, patterns4.size());
-		assertEquals(1L, patterns1.size());
-		assertEquals(1L, patterns2.size());
-
-		assertTrue(sharedBuffer.isEmpty());
-		assertTrue(patterns4.isEmpty());
-		assertEquals(Collections.singletonList(expectedPattern1), patterns1);
-		assertEquals(Collections.singletonList(expectedPattern2), patterns2);
-		assertEquals(Collections.singletonList(expectedPattern3), patterns3);
-	}
-
-	@Test
-	public void testSharedBufferSerialization() throws IOException, ClassNotFoundException {
-		SharedBuffer<String, Event> sharedBuffer = new SharedBuffer<>();
-		int numberEvents = 8;
-		Event[] events = new Event[numberEvents];
-		final long timestamp = 1L;
-
-		for (int i = 0; i < numberEvents; i++) {
-			events[i] = new Event(i + 1, "e" + (i + 1), i);
-		}
-
-		sharedBuffer.put("a1", events[0], timestamp, null, null, 0, 0, DeweyNumber.fromString("1"));
-		sharedBuffer.put("a[]", events[1], timestamp, "a1", events[0], timestamp, 0, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("a1", events[2], timestamp, null, null, 0, 0, DeweyNumber.fromString("2"));
-		sharedBuffer.put("a[]", events[2], timestamp, "a[]", events[1], timestamp, 1, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("a[]", events[3], timestamp, "a[]", events[2], timestamp, 2, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("a[]", events[3], timestamp, "a1", events[2], timestamp, 0, DeweyNumber.fromString("2.0"));
-		sharedBuffer.put("a[]", events[4], timestamp, "a[]", events[3], timestamp, 3, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("b", events[5], timestamp, "a[]", events[4], timestamp, 4, DeweyNumber.fromString("1.0.0"));
-		sharedBuffer.put("a[]", events[5], timestamp, "a[]", events[4], timestamp, 4, DeweyNumber.fromString("1.1"));
-		sharedBuffer.put("b", events[5], timestamp, "a[]", events[3], timestamp, 1, DeweyNumber.fromString("2.0.0"));
-		sharedBuffer.put("a[]", events[6], timestamp, "a[]", events[5], timestamp, 5, DeweyNumber.fromString("1.1"));
-		sharedBuffer.put("b", events[7], timestamp, "a[]", events[6], timestamp, 6, DeweyNumber.fromString("1.1.0"));
-
-		SharedBuffer.SharedBufferSerializer serializer = new SharedBuffer.SharedBufferSerializer(
-				StringSerializer.INSTANCE, Event.createTypeSerializer());
-
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		serializer.serialize(sharedBuffer, new DataOutputViewStreamWrapper(baos));
-
-		ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-		SharedBuffer<String, Event> copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais));
-
-		assertEquals(sharedBuffer, copy);
-	}
-
-	@Test
-	public void testClearingSharedBufferWithMultipleEdgesBetweenEntries() {
-		SharedBuffer<String, Event> sharedBuffer = new SharedBuffer<>();
-		int numberEvents = 8;
-		Event[] events = new Event[numberEvents];
-		final long timestamp = 1L;
-
-		for (int i = 0; i < numberEvents; i++) {
-			events[i] = new Event(i + 1, "e" + (i + 1), i);
-		}
-
-		sharedBuffer.put("start", events[1], timestamp, DeweyNumber.fromString("1"));
-		sharedBuffer.put("branching", events[2], timestamp, "start", events[1], timestamp, 0, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("branching", events[3], timestamp, "start", events[1], timestamp, 0, DeweyNumber.fromString("1.1"));
-		sharedBuffer.put("branching", events[3], timestamp, "branching", events[2], timestamp, 1, DeweyNumber.fromString("1.0.0"));
-		sharedBuffer.put("branching", events[4], timestamp, "branching", events[3], timestamp, 2, DeweyNumber.fromString("1.0.0.0"));
-		sharedBuffer.put("branching", events[4], timestamp, "branching", events[3], timestamp, 2, DeweyNumber.fromString("1.1.0"));
-
-		//simulate IGNORE (next event can point to events[2])
-		sharedBuffer.lock("branching", events[2], timestamp, 1);
-
-		sharedBuffer.release("branching", events[4], timestamp, 3);
-
-		//There should be still events[1] and events[2] in the buffer
-		assertFalse(sharedBuffer.isEmpty());
-	}
-
-	@Test
-	public void testSharedBufferExtractOrder() {
-		SharedBuffer<String, Event> sharedBuffer = new SharedBuffer<>();
-		int numberEvents = 10;
-		Event[] events = new Event[numberEvents];
-		final long timestamp = 1L;
-
-		for (int i = 0; i < numberEvents; i++) {
-			events[i] = new Event(i + 1, "e" + (i + 1), i);
-		}
-
-		Map<String, List<Event>> expectedResult = new LinkedHashMap<>();
-		expectedResult.put("a", new ArrayList<>());
-		expectedResult.get("a").add(events[1]);
-		expectedResult.put("b", new ArrayList<>());
-		expectedResult.get("b").add(events[2]);
-		expectedResult.put("aa", new ArrayList<>());
-		expectedResult.get("aa").add(events[3]);
-		expectedResult.put("bb", new ArrayList<>());
-		expectedResult.get("bb").add(events[4]);
-		expectedResult.put("c", new ArrayList<>());
-		expectedResult.get("c").add(events[5]);
-
-		sharedBuffer.put("a", events[1], timestamp, DeweyNumber.fromString("1"));
-		sharedBuffer.put("b", events[2], timestamp, "a", events[1], timestamp, 0, DeweyNumber.fromString("1.0"));
-		sharedBuffer.put("aa", events[3], timestamp, "b", events[2], timestamp, 1, DeweyNumber.fromString("1.0.0"));
-		sharedBuffer.put("bb", events[4], timestamp, "aa", events[3], timestamp, 2, DeweyNumber.fromString("1.0.0.0"));
-		sharedBuffer.put("c", events[5], timestamp, "bb", events[4], timestamp, 3, DeweyNumber.fromString("1.0.0.0.0"));
-
-		Collection<Map<String, List<Event>>> patternsResult = sharedBuffer.extractPatterns("c", events[5], timestamp, 4, DeweyNumber.fromString("1.0.0.0.0"));
-
-		List<String> expectedOrder = new ArrayList<>();
-		expectedOrder.add("a");
-		expectedOrder.add("b");
-		expectedOrder.add("aa");
-		expectedOrder.add("bb");
-		expectedOrder.add("c");
-
-		List<String> resultOrder = new ArrayList<>();
-		for (String key: patternsResult.iterator().next().keySet()){
-			resultOrder.add(key);
-		}
-		assertEquals(expectedOrder, resultOrder);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesOrMoreITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesOrMoreITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesOrMoreITCase.java
index 4e540dd..049c84b 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesOrMoreITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesOrMoreITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -34,13 +33,14 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 
 /**
  * Tests for {@link Pattern#timesOrMore(int)}.
  */
 public class TimesOrMoreITCase extends TestLogger {
 	@Test
-	public void testTimesOrMore() {
+	public void testTimesOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -79,7 +79,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -91,7 +91,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNonStrict() {
+	public void testTimesOrMoreNonStrict() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -126,7 +126,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -139,7 +139,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreStrict() {
+	public void testTimesOrMoreStrict() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -174,7 +174,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -184,7 +184,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreStrictOptional() {
+	public void testTimesOrMoreStrictOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -219,7 +219,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -230,7 +230,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreStrictOptional2() {
+	public void testTimesOrMoreStrictOptional2() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -263,7 +263,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -275,7 +275,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNonStrictOptional() {
+	public void testTimesOrMoreNonStrictOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -306,7 +306,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -316,7 +316,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNonStrictOptional2() {
+	public void testTimesOrMoreNonStrictOptional2() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -351,7 +351,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -365,7 +365,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNonStrictOptional3() {
+	public void testTimesOrMoreNonStrictOptional3() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -400,7 +400,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -413,7 +413,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNonStrictWithNext() {
+	public void testTimesOrMoreNonStrictWithNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -448,7 +448,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -460,7 +460,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNotStrictWithFollowedBy() {
+	public void testTimesOrMoreNotStrictWithFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -493,7 +493,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -504,7 +504,7 @@ public class TimesOrMoreITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesOrMoreNotStrictWithFollowedByAny() {
+	public void testTimesOrMoreNotStrictWithFollowedByAny() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -537,7 +537,7 @@ public class TimesOrMoreITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java
index 76ed26a..203a1c2 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -34,6 +33,7 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 
 /**
  * Tests for {@link Pattern#times(int, int)}.
@@ -42,7 +42,7 @@ import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
 public class TimesRangeITCase extends TestLogger {
 
 	@Test
-	public void testTimesRange() {
+	public void testTimesRange() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -80,7 +80,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -93,7 +93,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeFromZero() {
+	public void testTimesRangeFromZero() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -131,7 +131,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -144,7 +144,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNonStrict() {
+	public void testTimesRangeNonStrict() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -178,7 +178,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -194,7 +194,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeStrict() {
+	public void testTimesRangeStrict() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -228,7 +228,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -242,7 +242,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeStrictOptional() {
+	public void testTimesRangeStrictOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -276,7 +276,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -290,7 +290,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeStrictOptional1() {
+	public void testTimesRangeStrictOptional1() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -322,7 +322,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -335,7 +335,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNonStrictOptional1() {
+	public void testTimesRangeNonStrictOptional1() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -365,7 +365,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -375,7 +375,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNonStrictOptional2() {
+	public void testTimesRangeNonStrictOptional2() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -409,7 +409,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -423,7 +423,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNonStrictOptional3() {
+	public void testTimesRangeNonStrictOptional3() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -457,7 +457,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -470,7 +470,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNonStrictWithNext() {
+	public void testTimesRangeNonStrictWithNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -504,7 +504,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -516,7 +516,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNotStrictWithFollowedBy() {
+	public void testTimesRangeNotStrictWithFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -548,7 +548,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -559,7 +559,7 @@ public class TimesRangeITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesRangeNotStrictWithFollowedByAny() {
+	public void testTimesRangeNotStrictWithFollowedByAny() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -591,7 +591,7 @@ public class TimesRangeITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
index b603174..fb7f086 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
@@ -34,7 +33,8 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
-import static org.junit.Assert.assertTrue;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests for {@link Pattern#until(IterativeCondition)}.
@@ -89,9 +89,9 @@ public class UntilConditionITCase {
 				UNTIL_CONDITION
 			);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -99,7 +99,9 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -137,9 +139,9 @@ public class UntilConditionITCase {
 		}).oneOrMore().allowCombinations().until(UNTIL_CONDITION)
 			.followedBy("end").where(UNTIL_CONDITION);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -149,7 +151,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent3, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -187,9 +190,9 @@ public class UntilConditionITCase {
 				UNTIL_CONDITION
 			);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -197,7 +200,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -237,16 +241,17 @@ public class UntilConditionITCase {
 				UNTIL_CONDITION
 			);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -284,9 +289,9 @@ public class UntilConditionITCase {
 				UNTIL_CONDITION
 			);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -295,7 +300,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, breaking),
 			Lists.newArrayList(startEvent, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -333,9 +339,9 @@ public class UntilConditionITCase {
 		}).oneOrMore().optional().allowCombinations().until(UNTIL_CONDITION)
 			.followedBy("end").where(UNTIL_CONDITION);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -346,7 +352,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, breaking),
 			Lists.newArrayList(startEvent, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -384,9 +391,9 @@ public class UntilConditionITCase {
 				UNTIL_CONDITION
 			);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -395,7 +402,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, breaking),
 			Lists.newArrayList(startEvent, breaking)
 		));
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -430,7 +438,7 @@ public class UntilConditionITCase {
 			}
 		}).oneOrMore().until(UNTIL_CONDITION);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -474,7 +482,7 @@ public class UntilConditionITCase {
 			}
 		}).oneOrMore().optional().until(UNTIL_CONDITION);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -514,9 +522,9 @@ public class UntilConditionITCase {
 			}
 		}).followedBy("middle").oneOrMore().until(UNTIL_CONDITION);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -526,7 +534,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1)
 		));
 
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -567,9 +576,9 @@ public class UntilConditionITCase {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -579,7 +588,8 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1)
 		));
 
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
@@ -620,9 +630,9 @@ public class UntilConditionITCase {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -633,6 +643,7 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent)
 		));
 
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
index d1b6d59..92f59a5 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
@@ -18,10 +18,7 @@
 
 package org.apache.flink.cep.nfa.compiler;
 
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.SubEvent;
 import org.apache.flink.cep.nfa.AfterMatchSkipStrategy;
@@ -46,6 +43,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -72,9 +70,6 @@ public class NFACompilerTest extends TestLogger {
 		}
 	};
 
-	private static final TypeSerializer<Event> serializer = TypeExtractor.createTypeInfo(Event.class)
-		.createSerializer(new ExecutionConfig());
-
 	@Rule
 	public ExpectedException expectedException = ExpectedException.none();
 
@@ -90,7 +85,7 @@ public class NFACompilerTest extends TestLogger {
 			.followedBy("start").where(new TestFilter());
 
 		// here we must have an exception because of the two "start" patterns with the same name.
-		NFACompiler.compile(invalidPattern, Event.createTypeSerializer(), false);
+		compile(invalidPattern, false);
 	}
 
 	@Test
@@ -105,7 +100,7 @@ public class NFACompilerTest extends TestLogger {
 			.notFollowedBy("end").where(new TestFilter());
 
 		// here we must have an exception because of the two "start" patterns with the same name.
-		NFACompiler.compile(invalidPattern, Event.createTypeSerializer(), false);
+		compile(invalidPattern, false);
 	}
 
 	/**
@@ -131,7 +126,7 @@ public class NFACompilerTest extends TestLogger {
 			.followedBy("middle").subtype(SubEvent.class)
 			.next("end").where(endFilter);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, serializer, false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		Collection<State<Event>> states = nfa.getStates();
 		assertEquals(4, states.size());
@@ -218,7 +213,7 @@ public class NFACompilerTest extends TestLogger {
 			}
 		});
 
-		NFACompiler.compile(invalidPattern, Event.createTypeSerializer(), false);
+		compile(invalidPattern, false);
 	}
 
 	private <T> Set<Tuple2<String, StateTransitionAction>> unfoldTransitions(final State<T> state) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
new file mode 100644
index 0000000..9be7cc1
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
@@ -0,0 +1,216 @@
+/*
+ * 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.nfa.sharedbuffer;
+
+import org.apache.flink.cep.Event;
+import org.apache.flink.cep.nfa.DeweyNumber;
+import org.apache.flink.cep.utils.TestSharedBuffer;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link SharedBuffer}.
+ */
+public class SharedBufferTest extends TestLogger {
+
+	@Test
+	public void testSharedBuffer() throws Exception {
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		int numberEvents = 8;
+		Event[] events = new Event[numberEvents];
+		EventId[] eventIds = new EventId[numberEvents];
+		final long timestamp = 1L;
+
+		for (int i = 0; i < numberEvents; i++) {
+			events[i] = new Event(i + 1, "e" + (i + 1), i);
+			eventIds[i] = sharedBuffer.registerEvent(events[i], timestamp);
+		}
+
+		Map<String, List<Event>> expectedPattern1 = new HashMap<>();
+		expectedPattern1.put("a1", new ArrayList<>());
+		expectedPattern1.get("a1").add(events[2]);
+
+		expectedPattern1.put("a[]", new ArrayList<>());
+		expectedPattern1.get("a[]").add(events[3]);
+
+		expectedPattern1.put("b", new ArrayList<>());
+		expectedPattern1.get("b").add(events[5]);
+
+		Map<String, List<Event>> expectedPattern2 = new HashMap<>();
+		expectedPattern2.put("a1", new ArrayList<>());
+		expectedPattern2.get("a1").add(events[0]);
+
+		expectedPattern2.put("a[]", new ArrayList<>());
+		expectedPattern2.get("a[]").add(events[1]);
+		expectedPattern2.get("a[]").add(events[2]);
+		expectedPattern2.get("a[]").add(events[3]);
+		expectedPattern2.get("a[]").add(events[4]);
+
+		expectedPattern2.put("b", new ArrayList<>());
+		expectedPattern2.get("b").add(events[5]);
+
+		Map<String, List<Event>> expectedPattern3 = new HashMap<>();
+		expectedPattern3.put("a1", new ArrayList<>());
+		expectedPattern3.get("a1").add(events[0]);
+
+		expectedPattern3.put("a[]", new ArrayList<>());
+		expectedPattern3.get("a[]").add(events[1]);
+		expectedPattern3.get("a[]").add(events[2]);
+		expectedPattern3.get("a[]").add(events[3]);
+		expectedPattern3.get("a[]").add(events[4]);
+		expectedPattern3.get("a[]").add(events[5]);
+		expectedPattern3.get("a[]").add(events[6]);
+
+		expectedPattern3.put("b", new ArrayList<>());
+		expectedPattern3.get("b").add(events[7]);
+
+		NodeId a10 = sharedBuffer.put("a1", eventIds[0], DeweyNumber.fromString("1"));
+		NodeId aLoop0 = sharedBuffer.put("a[]", eventIds[1], a10, DeweyNumber.fromString("1.0"));
+		NodeId a11 = sharedBuffer.put("a1", eventIds[2], DeweyNumber.fromString("2"));
+		NodeId aLoop1 = sharedBuffer.put("a[]", eventIds[2], aLoop0, DeweyNumber.fromString("1.0"));
+		NodeId aLoop2 = sharedBuffer.put("a[]", eventIds[3], aLoop1, DeweyNumber.fromString("1.0"));
+		NodeId aSecondLoop0 = sharedBuffer.put("a[]", eventIds[3], a11, DeweyNumber.fromString("2.0"));
+		NodeId aLoop3 = sharedBuffer.put("a[]", eventIds[4], aLoop2, DeweyNumber.fromString("1.0"));
+		NodeId b0 = sharedBuffer.put("b", eventIds[5], aLoop3, DeweyNumber.fromString("1.0.0"));
+		NodeId aLoop4 = sharedBuffer.put("a[]", eventIds[5], aLoop3, DeweyNumber.fromString("1.1"));
+		NodeId b1 = sharedBuffer.put("b", eventIds[5], aSecondLoop0, DeweyNumber.fromString("2.0.0"));
+		NodeId aLoop5 = sharedBuffer.put("a[]", eventIds[6], aLoop4, DeweyNumber.fromString("1.1"));
+		NodeId b3 = sharedBuffer.put("b", eventIds[7], aLoop5, DeweyNumber.fromString("1.1.0"));
+
+		Collection<Map<String, List<Event>>> patterns3 = sharedBuffer.extractPatterns(b3, DeweyNumber.fromString("1.1.0"));
+		sharedBuffer.releaseNode(b3);
+		Collection<Map<String, List<Event>>> patterns4 = sharedBuffer.extractPatterns(b3, DeweyNumber.fromString("1.1.0"));
+
+		Collection<Map<String, List<Event>>> patterns1 = sharedBuffer.extractPatterns(b1, DeweyNumber.fromString("2.0.0"));
+		Collection<Map<String, List<Event>>> patterns2 = sharedBuffer.extractPatterns(b0, DeweyNumber.fromString("1.0.0"));
+		sharedBuffer.releaseNode(b0);
+		sharedBuffer.releaseNode(b1);
+
+		for (EventId eventId : eventIds) {
+			sharedBuffer.releaseEvent(eventId);
+		}
+
+		assertEquals(1L, patterns3.size());
+		assertEquals(0L, patterns4.size());
+		assertEquals(1L, patterns1.size());
+		assertEquals(1L, patterns2.size());
+
+		assertTrue(sharedBuffer.isEmpty());
+		assertTrue(patterns4.isEmpty());
+		assertEquals(Collections.singletonList(expectedPattern1), patterns1);
+		assertEquals(Collections.singletonList(expectedPattern2), patterns2);
+		assertEquals(Collections.singletonList(expectedPattern3), patterns3);
+	}
+
+	@Test
+	public void testClearingSharedBufferWithMultipleEdgesBetweenEntries() throws Exception {
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		int numberEvents = 8;
+		Event[] events = new Event[numberEvents];
+		EventId[] eventIds = new EventId[numberEvents];
+		final long timestamp = 1L;
+
+		for (int i = 0; i < numberEvents; i++) {
+			events[i] = new Event(i + 1, "e" + (i + 1), i);
+			eventIds[i] = sharedBuffer.registerEvent(events[i], timestamp);
+		}
+
+		NodeId start = sharedBuffer.put("start", eventIds[1], DeweyNumber.fromString("1"));
+		NodeId b0 = sharedBuffer.put("branching", eventIds[2], start, DeweyNumber.fromString("1.0"));
+		NodeId b1 = sharedBuffer.put("branching", eventIds[3], start, DeweyNumber.fromString("1.1"));
+		NodeId b00 = sharedBuffer.put("branching", eventIds[3], b0, DeweyNumber.fromString("1.0.0"));
+		sharedBuffer.put("branching", eventIds[4], b00, DeweyNumber.fromString("1.0.0.0"));
+		NodeId b10 = sharedBuffer.put("branching", eventIds[4], b1, DeweyNumber.fromString("1.1.0"));
+
+		//simulate IGNORE (next event can point to events[2])
+		sharedBuffer.lockNode(b0);
+
+		sharedBuffer.releaseNode(b10);
+
+		for (EventId eventId : eventIds) {
+			sharedBuffer.releaseEvent(eventId);
+		}
+
+		//There should be still events[1] and events[2] in the buffer
+		assertFalse(sharedBuffer.isEmpty());
+	}
+
+	@Test
+	public void testSharedBufferExtractOrder() throws Exception {
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		int numberEvents = 5;
+		Event[] events = new Event[numberEvents];
+		EventId[] eventIds = new EventId[numberEvents];
+		final long timestamp = 1L;
+
+		for (int i = 0; i < numberEvents; i++) {
+			events[i] = new Event(i + 1, "e" + (i + 1), i);
+			eventIds[i] = sharedBuffer.registerEvent(events[i], timestamp);
+		}
+
+		Map<String, List<Event>> expectedResult = new LinkedHashMap<>();
+		expectedResult.put("a", new ArrayList<>());
+		expectedResult.get("a").add(events[0]);
+		expectedResult.put("b", new ArrayList<>());
+		expectedResult.get("b").add(events[1]);
+		expectedResult.put("aa", new ArrayList<>());
+		expectedResult.get("aa").add(events[2]);
+		expectedResult.put("bb", new ArrayList<>());
+		expectedResult.get("bb").add(events[3]);
+		expectedResult.put("c", new ArrayList<>());
+		expectedResult.get("c").add(events[4]);
+
+		NodeId a = sharedBuffer.put("a", eventIds[0], DeweyNumber.fromString("1"));
+		NodeId b = sharedBuffer.put("b", eventIds[1], a, DeweyNumber.fromString("1.0"));
+		NodeId aa = sharedBuffer.put("aa", eventIds[2], b, DeweyNumber.fromString("1.0.0"));
+		NodeId bb = sharedBuffer.put("bb", eventIds[3], aa, DeweyNumber.fromString("1.0.0.0"));
+		NodeId c = sharedBuffer.put("c", eventIds[4], bb, DeweyNumber.fromString("1.0.0.0.0"));
+
+		Collection<Map<String, List<Event>>> patternsResult = sharedBuffer.extractPatterns(c,
+			DeweyNumber.fromString("1.0.0.0.0"));
+
+		List<String> expectedOrder = new ArrayList<>();
+		expectedOrder.add("a");
+		expectedOrder.add("b");
+		expectedOrder.add("aa");
+		expectedOrder.add("bb");
+		expectedOrder.add("c");
+
+		for (EventId eventId : eventIds) {
+			sharedBuffer.releaseEvent(eventId);
+		}
+
+		List<String> resultOrder = new ArrayList<>(patternsResult.iterator().next().keySet());
+		assertEquals(expectedOrder, resultOrder);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
index 9b120ca..6c7543a 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
@@ -551,7 +551,7 @@ public class CEPMigrationTest {
 	}
 
 	@Test
-	public void testAndOrSubtypConditionsAfterMigration() throws Exception {
+	public void testAndOrSubtypeConditionsAfterMigration() throws Exception {
 
 		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
 			private static final long serialVersionUID = -4873366487571254798L;
@@ -625,7 +625,7 @@ public class CEPMigrationTest {
 			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
 					.within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 
@@ -653,7 +653,7 @@ public class CEPMigrationTest {
 				.times(2)
 				.within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 
@@ -684,7 +684,7 @@ public class CEPMigrationTest {
 					// priority queue in CEP operator are correctly checkpointed/restored
 					.within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
index 5a98445..4786484 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
@@ -64,6 +64,7 @@ import java.util.Map;
 import java.util.Queue;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.validateMockitoUsage;
 
@@ -462,9 +463,9 @@ public class CEPOperatorTest extends TestLogger {
 		try {
 			harness.open();
 
-			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "nfaValueState");
+			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "computationStates");
 			final ValueState nfaOperatorStateSpy = Mockito.spy(nfaOperatorState);
-			Whitebox.setInternalState(operator, "nfaValueState", nfaOperatorStateSpy);
+			Whitebox.setInternalState(operator, "computationStates", nfaOperatorStateSpy);
 
 			Event startEvent = new Event(42, "c", 1.0);
 			SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0);
@@ -476,7 +477,7 @@ public class CEPOperatorTest extends TestLogger {
 			harness.processElement(new StreamRecord<>(endEvent, 4L));
 
 			// verify the number of invocations NFA is updated
-			Mockito.verify(nfaOperatorStateSpy, Mockito.times(2)).update(Mockito.any());
+			Mockito.verify(nfaOperatorStateSpy, Mockito.times(3)).update(Mockito.any());
 
 			// get and verify the output
 			Queue<Object> result = harness.getOutput();
@@ -507,9 +508,9 @@ public class CEPOperatorTest extends TestLogger {
 
 			harness.open();
 
-			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "nfaValueState");
+			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "computationStates");
 			final ValueState nfaOperatorStateSpy = Mockito.spy(nfaOperatorState);
-			Whitebox.setInternalState(operator, "nfaValueState", nfaOperatorStateSpy);
+			Whitebox.setInternalState(operator, "computationStates", nfaOperatorStateSpy);
 
 			Event startEvent = new Event(42, "c", 1.0);
 			SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0);
@@ -521,7 +522,7 @@ public class CEPOperatorTest extends TestLogger {
 			harness.processElement(new StreamRecord<>(endEvent, 4L));
 
 			// verify the number of invocations NFA is updated
-			Mockito.verify(nfaOperatorStateSpy, Mockito.times(2)).update(Mockito.any());
+			Mockito.verify(nfaOperatorStateSpy, Mockito.times(3)).update(Mockito.any());
 
 			// get and verify the output
 			Queue<Object> result = harness.getOutput();
@@ -568,8 +569,8 @@ public class CEPOperatorTest extends TestLogger {
 			assertEquals(2L, harness.numEventTimeTimers());
 			assertEquals(4L, operator.getPQSize(42));
 			assertEquals(1L, operator.getPQSize(43));
-			assertTrue(!operator.hasNonEmptyNFAState(42));
-			assertTrue(!operator.hasNonEmptyNFAState(43));
+			assertTrue(!operator.hasNonEmptySharedBuffer(42));
+			assertTrue(!operator.hasNonEmptySharedBuffer(43));
 
 			harness.processWatermark(new Watermark(2L));
 
@@ -581,9 +582,9 @@ public class CEPOperatorTest extends TestLogger {
 			// for 43 the element entered the NFA and the PQ is empty
 
 			assertEquals(2L, harness.numEventTimeTimers());
-			assertTrue(operator.hasNonEmptyNFAState(42));
+			assertTrue(operator.hasNonEmptySharedBuffer(42));
 			assertEquals(1L, operator.getPQSize(42));
-			assertTrue(operator.hasNonEmptyNFAState(43));
+			assertTrue(operator.hasNonEmptySharedBuffer(43));
 			assertTrue(!operator.hasNonEmptyPQ(43));
 
 			harness.processElement(new StreamRecord<>(startEvent2, 4L));
@@ -605,9 +606,9 @@ public class CEPOperatorTest extends TestLogger {
 			// now we have 1 key because the 43 expired and was removed.
 			// 42 is still there due to startEvent2
 			assertEquals(1L, harness.numEventTimeTimers());
-			assertTrue(operator2.hasNonEmptyNFAState(42));
+			assertTrue(operator2.hasNonEmptySharedBuffer(42));
 			assertTrue(!operator2.hasNonEmptyPQ(42));
-			assertTrue(!operator2.hasNonEmptyNFAState(43));
+			assertTrue(!operator2.hasNonEmptySharedBuffer(43));
 			assertTrue(!operator2.hasNonEmptyPQ(43));
 
 			verifyPattern(harness.getOutput().poll(), startEvent1, middleEvent1, endEvent1);
@@ -622,7 +623,7 @@ public class CEPOperatorTest extends TestLogger {
 			harness.processWatermark(20L);
 			harness.processWatermark(21L);
 
-			assertTrue(!operator2.hasNonEmptyNFAState(42));
+			assertTrue(!operator2.hasNonEmptySharedBuffer(42));
 			assertTrue(!operator2.hasNonEmptyPQ(42));
 			assertEquals(0L, harness.numEventTimeTimers());
 
@@ -665,7 +666,7 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertEquals(1L, harness.numEventTimeTimers());
 			assertEquals(7L, operator.getPQSize(41));
-			assertTrue(!operator.hasNonEmptyNFAState(41));
+			assertTrue(!operator.hasNonEmptySharedBuffer(41));
 
 			harness.processWatermark(new Watermark(2L));
 
@@ -674,7 +675,7 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertEquals(1L, harness.numEventTimeTimers());
 			assertEquals(6L, operator.getPQSize(41));
-			assertTrue(operator.hasNonEmptyNFAState(41)); // processed the first element
+			assertTrue(operator.hasNonEmptySharedBuffer(41)); // processed the first element
 
 			harness.processWatermark(new Watermark(8L));
 
@@ -714,12 +715,12 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertEquals(1L, harness.numEventTimeTimers());
 			assertEquals(0L, operator.getPQSize(41));
-			assertTrue(operator.hasNonEmptyNFAState(41));
+			assertTrue(operator.hasNonEmptySharedBuffer(41));
 
 			harness.processWatermark(new Watermark(17L));
 			verifyWatermark(harness.getOutput().poll(), 17L);
 
-			assertTrue(!operator.hasNonEmptyNFAState(41));
+			assertTrue(!operator.hasNonEmptySharedBuffer(41));
 			assertTrue(!operator.hasNonEmptyPQ(41));
 			assertEquals(0L, harness.numEventTimeTimers());
 		} finally {
@@ -800,8 +801,8 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertTrue(!operator.hasNonEmptyPQ(42));
 			assertTrue(!operator.hasNonEmptyPQ(43));
-			assertTrue(operator.hasNonEmptyNFAState(42));
-			assertTrue(operator.hasNonEmptyNFAState(43));
+			assertTrue(operator.hasNonEmptySharedBuffer(42));
+			assertTrue(operator.hasNonEmptySharedBuffer(43));
 
 			harness.setProcessingTime(3L);
 
@@ -834,10 +835,10 @@ public class CEPOperatorTest extends TestLogger {
 
 			harness.setProcessingTime(21L);
 
-			assertTrue(operator2.hasNonEmptyNFAState(42));
+			assertTrue(operator2.hasNonEmptySharedBuffer(42));
 
 			harness.processElement(new StreamRecord<>(startEvent1, 21L));
-			assertTrue(operator2.hasNonEmptyNFAState(42));
+			assertTrue(operator2.hasNonEmptySharedBuffer(42));
 
 			harness.setProcessingTime(49L);
 
@@ -845,7 +846,7 @@ public class CEPOperatorTest extends TestLogger {
 			harness.processElement(new StreamRecord<>(new Event(42, "foobar", 1.0), 2L));
 
 			// the pattern expired
-			assertTrue(!operator2.hasNonEmptyNFAState(42));
+			assertTrue(!operator2.hasNonEmptySharedBuffer(42));
 
 			assertEquals(0L, harness.numEventTimeTimers());
 			assertTrue(!operator2.hasNonEmptyPQ(42));
@@ -910,7 +911,7 @@ public class CEPOperatorTest extends TestLogger {
 
 				@Override
 				public NFA<Event> createNFA() {
-					return NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+					return NFACompiler.compileFactory(pattern, false).createNFA();
 				}
 			});
 
@@ -988,12 +989,12 @@ public class CEPOperatorTest extends TestLogger {
 			harness
 				.processElement(new StreamRecord<>(new SubEvent(42, "barfoo", 1.0, 5.0), 0L));
 
-			assertTrue(!operator.hasNonEmptyNFAState(42));
-			assertTrue(!operator.hasNonEmptyNFAState(43));
+			assertTrue(!operator.hasNonEmptySharedBuffer(42));
+			assertTrue(!operator.hasNonEmptySharedBuffer(43));
 
 			harness.setProcessingTime(3L);
-			assertTrue(operator.hasNonEmptyNFAState(42));
-			assertTrue(operator.hasNonEmptyNFAState(43));
+			assertTrue(operator.hasNonEmptySharedBuffer(42));
+			assertTrue(operator.hasNonEmptySharedBuffer(43));
 
 			harness.processElement(new StreamRecord<>(middleEvent2, 3L));
 			harness.processElement(new StreamRecord<>(middleEvent1, 3L));
@@ -1047,14 +1048,14 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertTrue(operator.hasNonEmptyPQ(42));
 			assertTrue(operator.hasNonEmptyPQ(43));
-			assertTrue(!operator.hasNonEmptyNFAState(42));
-			assertTrue(!operator.hasNonEmptyNFAState(43));
+			assertFalse(operator.hasNonEmptySharedBuffer(42));
+			assertFalse(operator.hasNonEmptySharedBuffer(43));
 
 			harness.processWatermark(3L);
-			assertTrue(!operator.hasNonEmptyPQ(42));
-			assertTrue(!operator.hasNonEmptyPQ(43));
-			assertTrue(operator.hasNonEmptyNFAState(42));
-			assertTrue(operator.hasNonEmptyNFAState(43));
+			assertFalse(operator.hasNonEmptyPQ(42));
+			assertFalse(operator.hasNonEmptyPQ(43));
+			assertTrue(operator.hasNonEmptySharedBuffer(42));
+			assertTrue(operator.hasNonEmptySharedBuffer(43));
 
 			harness.processElement(new StreamRecord<>(startEvent2, 4L));
 			harness.processElement(new StreamRecord<Event>(middleEvent2, 5L));
@@ -1224,7 +1225,7 @@ public class CEPOperatorTest extends TestLogger {
 					// priority queue in CEP operator are correctly checkpointed/restored
 					.within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 
@@ -1275,7 +1276,7 @@ public class CEPOperatorTest extends TestLogger {
 				}
 			}).within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 
@@ -1319,7 +1320,7 @@ public class CEPOperatorTest extends TestLogger {
 				}
 			}).within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
index 8a73556..c4b147c 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java
@@ -26,8 +26,10 @@ import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
 import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -366,20 +368,23 @@ public class CEPRescalingTest {
 	}
 
 	private KeyedOneInputStreamOperatorTestHarness<Integer, Event, Map<String, List<Event>>> getTestHarness(
-		int maxParallelism,
-		int taskParallelism,
-		int subtaskIdx) throws Exception {
+			int maxParallelism,
+			int taskParallelism,
+			int subtaskIdx) throws Exception {
 
 		KeySelector<Event, Integer> keySelector = new TestKeySelector();
-		return new KeyedOneInputStreamOperatorTestHarness<>(
-			getKeyedCepOpearator(
-				false,
-				new NFAFactory()),
-			keySelector,
-			BasicTypeInfo.INT_TYPE_INFO,
-			maxParallelism,
-			taskParallelism,
-			subtaskIdx);
+		KeyedOneInputStreamOperatorTestHarness<Integer, Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						getKeyedCepOpearator(
+								false,
+								new NFAFactory()),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO,
+						maxParallelism,
+						taskParallelism,
+						subtaskIdx);
+		harness.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
+		return harness;
 	}
 
 	private static class NFAFactory implements NFACompiler.NFAFactory<Event> {
@@ -427,7 +432,7 @@ public class CEPRescalingTest {
 				// priority queue in CEP operator are correctly checkpointed/restored
 				.within(Time.milliseconds(10L));
 
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+			return NFACompiler.compileFactory(pattern, handleTimeout).createNFA();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java
index 3753f34..abc4b18 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java
@@ -84,6 +84,8 @@ public class CepOperatorTestUtilities {
 			comparator,
 			null,
 			new PatternSelectFunction<Event, Map<String, List<Event>>>() {
+				private static final long serialVersionUID = -7143807777582726991L;
+
 				@Override
 				public Map<String, List<Event>> select(Map<String, List<Event>> pattern) throws Exception {
 					return pattern;

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/NFAUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/NFAUtils.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/NFAUtils.java
new file mode 100644
index 0000000..f2c6613
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/NFAUtils.java
@@ -0,0 +1,47 @@
+/*
+ * 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.utils;
+
+import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.pattern.Pattern;
+
+import static org.apache.flink.cep.nfa.compiler.NFACompiler.compileFactory;
+
+/**
+ * Utility methods for constructing NFA.
+ */
+public class NFAUtils {
+
+	/**
+	 * Compiles the given pattern into a {@link NFA}.
+	 *
+	 * @param pattern         Definition of sequence pattern
+	 * @param timeoutHandling True if the NFA shall return timed out event patterns
+	 * @param <T>             Type of the input events
+	 * @return Non-deterministic finite automaton representing the given pattern
+	 */
+	public static <T> NFA<T> compile(Pattern<T, ?> pattern, boolean timeoutHandling) {
+		NFACompiler.NFAFactory<T> factory = compileFactory(pattern, timeoutHandling);
+		return factory.createNFA();
+	}
+
+	private NFAUtils() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
new file mode 100644
index 0000000..2c7b979
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
@@ -0,0 +1,251 @@
+/*
+ * 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.utils;
+
+import org.apache.flink.api.common.state.AggregatingState;
+import org.apache.flink.api.common.state.AggregatingStateDescriptor;
+import org.apache.flink.api.common.state.FoldingState;
+import org.apache.flink.api.common.state.FoldingStateDescriptor;
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ReducingState;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Extends {@link SharedBuffer} with methods for checking the number of state accesses. It does not use a proper
+ * StateBackend, but uses stubs over java collections.
+ */
+public class TestSharedBuffer<V> extends SharedBuffer<V> {
+
+	private final MockKeyedStateStore keyedStateStore;
+
+	private TestSharedBuffer(MockKeyedStateStore stateStore, TypeSerializer<V> valueSerializer) {
+		super(stateStore, valueSerializer);
+		this.keyedStateStore = stateStore;
+	}
+
+	public long getStateWrites() {
+		return keyedStateStore.stateWrites;
+	}
+
+	public long getStateReads() {
+		return keyedStateStore.stateReads;
+	}
+
+	public long getStateAccesses() {
+		return getStateWrites() + getStateReads();
+	}
+
+	/**
+	 * Creates instance of {@link TestSharedBuffer}.
+	 *
+	 * @param typeSerializer serializer used to serialize incoming events
+	 * @param <T>            type of incoming events
+	 * @return TestSharedBuffer instance
+	 */
+	public static <T> TestSharedBuffer<T> createTestBuffer(TypeSerializer<T> typeSerializer) {
+		return new TestSharedBuffer<>(new MockKeyedStateStore(), typeSerializer);
+	}
+
+	private static class MockKeyedStateStore implements KeyedStateStore {
+
+		private long stateWrites = 0;
+		private long stateReads = 0;
+
+		@Override
+		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
+			return new ValueState<T>() {
+
+				private T value;
+
+				@Override
+				public T value() throws IOException {
+					stateReads++;
+					return value;
+				}
+
+				@Override
+				public void update(T value) throws IOException {
+					stateWrites++;
+					this.value = value;
+				}
+
+				@Override
+				public void clear() {
+					this.value = null;
+				}
+			};
+		}
+
+		@Override
+		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public <IN, ACC, OUT> AggregatingState<IN, OUT> getAggregatingState(AggregatingStateDescriptor<IN, ACC, OUT> stateProperties) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public <T, ACC> FoldingState<T, ACC> getFoldingState(FoldingStateDescriptor<T, ACC> stateProperties) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK, UV> stateProperties) {
+			return new MapState<UK, UV>() {
+
+				private Map<UK, UV> values;
+
+				private Map<UK, UV> getOrSetMap() {
+					if (values == null) {
+						this.values = new HashMap<>();
+					}
+					return values;
+				}
+
+				@Override
+				public UV get(UK key) throws Exception {
+					stateReads++;
+					if (values == null) {
+						return null;
+					}
+
+					return values.get(key);
+				}
+
+				@Override
+				public void put(UK key, UV value) throws Exception {
+					stateWrites++;
+					getOrSetMap().put(key, value);
+				}
+
+				@Override
+				public void putAll(Map<UK, UV> map) throws Exception {
+					stateWrites++;
+					getOrSetMap().putAll(map);
+				}
+
+				@Override
+				public void remove(UK key) throws Exception {
+					if (values == null) {
+						return;
+					}
+
+					stateWrites++;
+					values.remove(key);
+				}
+
+				@Override
+				public boolean contains(UK key) throws Exception {
+					if (values == null) {
+						return false;
+					}
+
+					stateReads++;
+					return values.containsKey(key);
+				}
+
+				@Override
+				public Iterable<Map.Entry<UK, UV>> entries() throws Exception {
+					if (values == null) {
+						return Collections.emptyList();
+					}
+
+					return () -> new CountingIterator<>(values.entrySet().iterator());
+				}
+
+				@Override
+				public Iterable<UK> keys() throws Exception {
+					if (values == null) {
+						return Collections.emptyList();
+					}
+
+					return () -> new CountingIterator<>(values.keySet().iterator());
+				}
+
+				@Override
+				public Iterable<UV> values() throws Exception {
+					if (values == null) {
+						return Collections.emptyList();
+					}
+
+					return () -> new CountingIterator<>(values.values().iterator());
+				}
+
+				@Override
+				public Iterator<Map.Entry<UK, UV>> iterator() throws Exception {
+					if (values == null) {
+						return Iterators.emptyIterator();
+					}
+
+					return new CountingIterator<>(values.entrySet().iterator());
+				}
+
+				@Override
+				public void clear() {
+					stateWrites++;
+					this.values = null;
+				}
+			};
+		}
+
+		private class CountingIterator<T> implements Iterator<T> {
+
+			private final Iterator<T> iterator;
+
+			CountingIterator(Iterator<T> iterator) {
+				this.iterator = iterator;
+			}
+
+			@Override
+			public boolean hasNext() {
+				return iterator.hasNext();
+			}
+
+			@Override
+			public T next() {
+				stateReads++;
+				return iterator.next();
+			}
+		}
+	}
+
+}


[05/10] flink git commit: [FLINK-9418] Migrate SharedBuffer to use MapState

Posted by dw...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index 76bc2d0..7e904c8 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -21,10 +21,11 @@ package org.apache.flink.cep.nfa;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.SubEvent;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.Quantifier;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.cep.utils.TestSharedBuffer;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.TestLogger;
@@ -32,6 +33,7 @@ import org.apache.flink.util.TestLogger;
 import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
 
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -45,6 +47,7 @@ import java.util.Set;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -54,8 +57,15 @@ import static org.junit.Assert.assertEquals;
 @SuppressWarnings("unchecked")
 public class NFAITCase extends TestLogger {
 
+	private SharedBuffer<Event> sharedBuffer;
+
+	@Before
+	public void init() {
+		sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+	}
+
 	@Test
-	public void testNoConditionNFA() {
+	public void testNoConditionNFA() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a = new Event(40, "a", 1.0);
@@ -72,7 +82,7 @@ public class NFAITCase extends TestLogger {
 
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").followedBy("end");
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -85,7 +95,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNoConditionLoopingNFA() {
+	public void testNoConditionLoopingNFA() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a = new Event(40, "a", 1.0);
@@ -102,7 +112,7 @@ public class NFAITCase extends TestLogger {
 
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").followedBy("end").oneOrMore();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -121,7 +131,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testAnyWithNoConditionNFA() {
+	public void testAnyWithNoConditionNFA() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a = new Event(40, "a", 1.0);
@@ -138,7 +148,7 @@ public class NFAITCase extends TestLogger {
 
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").followedByAny("end");
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -157,7 +167,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testSimplePatternNFA() {
+	public void testSimplePatternNFA() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(41, "start", 1.0);
@@ -194,7 +204,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -204,7 +214,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictContinuityWithResults() {
+	public void testStrictContinuityWithResults() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event middleEvent1 = new Event(41, "a", 2.0);
@@ -229,7 +239,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -239,7 +249,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictContinuityNoResults() {
+	public void testStrictContinuityNoResults() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event middleEvent1 = new Event(41, "a", 2.0);
@@ -266,7 +276,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -278,7 +288,7 @@ public class NFAITCase extends TestLogger {
 	 * length.
 	 */
 	@Test
-	public void testSimplePatternWithTimeWindowNFA() {
+	public void testSimplePatternWithTimeWindowNFA() throws Exception {
 		List<StreamRecord<Event>> events = new ArrayList<>();
 
 		final Event startEvent;
@@ -315,7 +325,7 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(10));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(events, nfa);
 
@@ -329,7 +339,7 @@ public class NFAITCase extends TestLogger {
 	 * out.
 	 */
 	@Test
-	public void testSimplePatternWithTimeoutHandling() {
+	public void testSimplePatternWithTimeoutHandling() throws Exception {
 		List<StreamRecord<Event>> events = new ArrayList<>();
 		List<Map<String, List<Event>>> resultingPatterns = new ArrayList<>();
 		Set<Tuple2<Map<String, List<Event>>, Long>> resultingTimeoutPatterns = new HashSet<>();
@@ -384,13 +394,13 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(10));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), true);
+		NFA<Event> nfa = compile(pattern, true);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		for (StreamRecord<Event> event: events) {
 			Tuple2<Collection<Map<String, List<Event>>>, Collection<Tuple2<Map<String, List<Event>>, Long>>> patterns =
-					nfa.process(nfaState, event.getValue(), event.getTimestamp());
+					nfa.process(sharedBuffer, nfaState, event.getValue(), event.getTimestamp());
 
 			Collection<Map<String, List<Event>>> matchedPatterns = patterns.f0;
 			Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutPatterns = patterns.f1;
@@ -406,7 +416,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testBranchingPattern() {
+	public void testBranchingPattern() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "start", 1.0);
@@ -455,7 +465,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -470,7 +480,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testComplexBranchingAfterZeroOrMore() {
+	public void testComplexBranchingAfterZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -528,7 +538,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -553,7 +563,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testZeroOrMore() {
+	public void testZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -589,7 +599,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -602,7 +612,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEagerZeroOrMore() {
+	public void testEagerZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -641,7 +651,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -654,7 +664,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testBeginWithZeroOrMore() {
+	public void testBeginWithZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event middleEvent1 = new Event(40, "a", 2.0);
@@ -683,7 +693,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -699,7 +709,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testZeroOrMoreAfterZeroOrMore() {
+	public void testZeroOrMoreAfterZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -746,7 +756,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -761,7 +771,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testZeroOrMoreAfterBranching() {
+	public void testZeroOrMoreAfterBranching() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -817,7 +827,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -834,7 +844,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictContinuityNoResultsAfterZeroOrMore() {
+	public void testStrictContinuityNoResultsAfterZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event start = new Event(40, "d", 2.0);
@@ -873,7 +883,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -881,7 +891,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictContinuityResultsAfterZeroOrMore() {
+	public void testStrictContinuityResultsAfterZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event start = new Event(40, "d", 2.0);
@@ -917,7 +927,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -928,7 +938,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testAtLeastOne() {
+	public void testAtLeastOne() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -964,7 +974,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -976,7 +986,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testBeginWithAtLeastOne() {
+	public void testBeginWithAtLeastOne() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent1 = new Event(41, "a", 2.0);
@@ -1005,7 +1015,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1021,7 +1031,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNextZeroOrMore() {
+	public void testNextZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "start", 1.0);
@@ -1060,7 +1070,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1070,7 +1080,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testAtLeastOneEager() {
+	public void testAtLeastOneEager() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1108,7 +1118,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1123,7 +1133,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testOptional() {
+	public void testOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1157,7 +1167,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1168,7 +1178,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimes() {
+	public void testTimes() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1206,7 +1216,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1217,7 +1227,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStartWithTimes() {
+	public void testStartWithTimes() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event middleEvent1 = new Event(41, "a", 2.0);
@@ -1246,7 +1256,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1258,7 +1268,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesNonStrictWithNext() {
+	public void testTimesNonStrictWithNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1292,7 +1302,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1303,7 +1313,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesNotStrictWithFollowedByEager() {
+	public void testTimesNotStrictWithFollowedByEager() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1335,7 +1345,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1345,7 +1355,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesNotStrictWithFollowedByNotEager() {
+	public void testTimesNotStrictWithFollowedByNotEager() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1377,7 +1387,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1389,7 +1399,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesStrictWithNextAndConsecutive() {
+	public void testTimesStrictWithNextAndConsecutive() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1422,7 +1432,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1430,7 +1440,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStartWithOptional() {
+	public void testStartWithOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1455,7 +1465,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1466,7 +1476,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEndWithZeroOrMore() {
+	public void testEndWithZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1495,7 +1505,7 @@ public class NFAITCase extends TestLogger {
 			}
 		}).oneOrMore().optional();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1508,7 +1518,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStartAndEndWithZeroOrMore() {
+	public void testStartAndEndWithZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1536,7 +1546,7 @@ public class NFAITCase extends TestLogger {
 			}
 		}).oneOrMore().optional();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1551,7 +1561,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEndWithOptional() {
+	public void testEndWithOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1576,7 +1586,7 @@ public class NFAITCase extends TestLogger {
 			}
 		}).optional();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1587,7 +1597,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEndWithOneOrMore() {
+	public void testEndWithOneOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -1616,7 +1626,7 @@ public class NFAITCase extends TestLogger {
 			}
 		}).oneOrMore();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1630,7 +1640,7 @@ public class NFAITCase extends TestLogger {
 	///////////////////////////////         Optional           ////////////////////////////////////////
 
 	@Test
-	public void testTimesNonStrictOptional1() {
+	public void testTimesNonStrictOptional1() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1660,7 +1670,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1670,7 +1680,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesNonStrictOptional2() {
+	public void testTimesNonStrictOptional2() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1704,7 +1714,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1717,7 +1727,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesNonStrictOptional3() {
+	public void testTimesNonStrictOptional3() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1751,7 +1761,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1763,7 +1773,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesStrictOptional() {
+	public void testTimesStrictOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1797,7 +1807,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1808,7 +1818,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testOneOrMoreStrictOptional() {
+	public void testOneOrMoreStrictOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1842,7 +1852,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1856,7 +1866,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesStrictOptional1() {
+	public void testTimesStrictOptional1() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1888,7 +1898,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1899,7 +1909,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testOptionalTimesNonStrictWithNext() {
+	public void testOptionalTimesNonStrictWithNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -1933,7 +1943,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1958,7 +1968,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStrictOneOrMore() {
+	public void testStrictOneOrMore() throws Exception {
 		List<List<Event>> resultingPatterns = testOneOrMore(Quantifier.ConsumingStrategy.STRICT);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -1969,7 +1979,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testSkipTillNextOneOrMore() {
+	public void testSkipTillNextOneOrMore() throws Exception {
 		List<List<Event>> resultingPatterns = testOneOrMore(Quantifier.ConsumingStrategy.SKIP_TILL_NEXT);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -1981,7 +1991,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testSkipTillAnyOneOrMore() {
+	public void testSkipTillAnyOneOrMore() throws Exception {
 		List<List<Event>> resultingPatterns = testOneOrMore(Quantifier.ConsumingStrategy.SKIP_TILL_ANY);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -1996,7 +2006,7 @@ public class NFAITCase extends TestLogger {
 		));
 	}
 
-	private List<List<Event>> testOneOrMore(Quantifier.ConsumingStrategy strategy) {
+	private List<List<Event>> testOneOrMore(Quantifier.ConsumingStrategy strategy) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -2044,13 +2054,13 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
 
 	@Test
-	public void testStrictEagerZeroOrMore() {
+	public void testStrictEagerZeroOrMore() throws Exception {
 		List<List<Event>> resultingPatterns = testZeroOrMore(Quantifier.ConsumingStrategy.STRICT);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -2062,7 +2072,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testSkipTillAnyZeroOrMore() {
+	public void testSkipTillAnyZeroOrMore() throws Exception {
 		List<List<Event>> resultingPatterns = testZeroOrMore(Quantifier.ConsumingStrategy.SKIP_TILL_ANY);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -2079,7 +2089,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testSkipTillNextZeroOrMore() {
+	public void testSkipTillNextZeroOrMore() throws Exception {
 		List<List<Event>> resultingPatterns = testZeroOrMore(Quantifier.ConsumingStrategy.SKIP_TILL_NEXT);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
@@ -2091,7 +2101,7 @@ public class NFAITCase extends TestLogger {
 		));
 	}
 
-	private List<List<Event>> testZeroOrMore(Quantifier.ConsumingStrategy strategy) {
+	private List<List<Event>> testZeroOrMore(Quantifier.ConsumingStrategy strategy) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -2139,13 +2149,13 @@ public class NFAITCase extends TestLogger {
 					}
 				});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
 
 	@Test
-	public void testTimesStrict() {
+	public void testTimesStrict() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -2179,7 +2189,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -2189,7 +2199,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesNonStrict() {
+	public void testTimesNonStrict() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -2223,7 +2233,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -2235,7 +2245,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStartWithZeroOrMoreStrict() {
+	public void testStartWithZeroOrMoreStrict() throws Exception {
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
 
@@ -2249,7 +2259,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testStartWithOneOrMoreStrict() {
+	public void testStartWithOneOrMoreStrict() throws Exception {
 
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 5726188262756267490L;
@@ -2263,7 +2273,7 @@ public class NFAITCase extends TestLogger {
 		testStartWithOneOrZeroOrMoreStrict(pattern);
 	}
 
-	private void testStartWithOneOrZeroOrMoreStrict(Pattern<Event, ?> pattern) {
+	private void testStartWithOneOrZeroOrMoreStrict(Pattern<Event, ?> pattern) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.startEvent, 1));
@@ -2272,7 +2282,7 @@ public class NFAITCase extends TestLogger {
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent2, 5));
 		inputEvents.add(new StreamRecord<>(ConsecutiveData.middleEvent3, 6));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -2287,7 +2297,7 @@ public class NFAITCase extends TestLogger {
 	///////////////////////////////     Clearing SharedBuffer     ////////////////////////////////////////
 
 	@Test
-	public void testTimesClearingBuffer() {
+	public void testTimesClearingBuffer() throws Exception {
 		Event startEvent = new Event(40, "c", 1.0);
 		Event middleEvent1 = new Event(41, "a", 2.0);
 		Event middleEvent2 = new Event(42, "a", 3.0);
@@ -2317,24 +2327,25 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(8));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
-		nfa.process(nfaState, startEvent, 1);
-		nfa.process(nfaState, middleEvent1, 2);
-		nfa.process(nfaState, middleEvent2, 3);
-		nfa.process(nfaState, middleEvent3, 4);
-		nfa.process(nfaState, end1, 6);
+		nfa.process(sharedBuffer, nfaState, startEvent, 1);
+		nfa.process(sharedBuffer, nfaState, middleEvent1, 2);
+		nfa.process(sharedBuffer, nfaState, middleEvent2, 3);
+		nfa.process(sharedBuffer, nfaState, middleEvent3, 4);
+		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(nfaState, null, 10);
+		nfa.process(sharedBuffer, nfaState, null, 10);
 
-		assertEquals(true, nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
-	public void testOptionalClearingBuffer() {
+	public void testOptionalClearingBuffer() throws Exception {
 		Event startEvent = new Event(40, "c", 1.0);
 		Event middleEvent = new Event(43, "a", 4.0);
 		Event end1 = new Event(44, "b", 5.0);
@@ -2362,22 +2373,23 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(8));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
-		nfa.process(nfaState, startEvent, 1);
-		nfa.process(nfaState, middleEvent, 5);
-		nfa.process(nfaState, end1, 6);
+		nfa.process(sharedBuffer, nfaState, startEvent, 1);
+		nfa.process(sharedBuffer, nfaState, middleEvent, 5);
+		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(nfaState, null, 10);
+		nfa.process(sharedBuffer, nfaState, null, 10);
 
-		assertEquals(true, nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
-	public void testAtLeastOneClearingBuffer() {
+	public void testAtLeastOneClearingBuffer() throws Exception {
 		Event startEvent = new Event(40, "c", 1.0);
 		Event middleEvent1 = new Event(41, "a", 2.0);
 		Event middleEvent2 = new Event(42, "a", 3.0);
@@ -2406,23 +2418,24 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(8));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
-		nfa.process(nfaState, startEvent, 1);
-		nfa.process(nfaState, middleEvent1, 3);
-		nfa.process(nfaState, middleEvent2, 4);
-		nfa.process(nfaState, end1, 6);
+		nfa.process(sharedBuffer, nfaState, startEvent, 1);
+		nfa.process(sharedBuffer, nfaState, middleEvent1, 3);
+		nfa.process(sharedBuffer, nfaState, middleEvent2, 4);
+		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(nfaState, null, 10);
+		nfa.process(sharedBuffer, nfaState, null, 10);
 
-		assertEquals(true, nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	@Test
-	public void testZeroOrMoreClearingBuffer() {
+	public void testZeroOrMoreClearingBuffer() throws Exception {
 		Event startEvent = new Event(40, "c", 1.0);
 		Event middleEvent1 = new Event(41, "a", 2.0);
 		Event middleEvent2 = new Event(42, "a", 3.0);
@@ -2451,25 +2464,26 @@ public class NFAITCase extends TestLogger {
 			}
 		}).within(Time.milliseconds(8));
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
-		nfa.process(nfaState, startEvent, 1);
-		nfa.process(nfaState, middleEvent1, 3);
-		nfa.process(nfaState, middleEvent2, 4);
-		nfa.process(nfaState, end1, 6);
+		nfa.process(sharedBuffer, nfaState, startEvent, 1);
+		nfa.process(sharedBuffer, nfaState, middleEvent1, 3);
+		nfa.process(sharedBuffer, nfaState, middleEvent2, 4);
+		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(nfaState, null, 10);
+		nfa.process(sharedBuffer, nfaState, null, 10);
 
-		assertEquals(true, nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 	///////////////////////////////////////   Skip till next     /////////////////////////////
 
 	@Test
-	public void testBranchingPatternSkipTillNext() {
+	public void testBranchingPatternSkipTillNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "start", 1.0);
@@ -2518,7 +2532,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> patterns = feedNFA(inputEvents, nfa);
 
@@ -2528,7 +2542,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testBranchingPatternMixedFollowedBy() {
+	public void testBranchingPatternMixedFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "start", 1.0);
@@ -2577,7 +2591,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> patterns = feedNFA(inputEvents, nfa);
 
@@ -2589,7 +2603,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testMultipleTakesVersionCollision() {
+	public void testMultipleTakesVersionCollision() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent = new Event(40, "c", 1.0);
@@ -2638,7 +2652,7 @@ public class NFAITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -2671,7 +2685,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNFAResultOrdering() {
+	public void testNFAResultOrdering() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event startEvent1 = new Event(41, "a-1", 2.0);
@@ -2709,17 +2723,18 @@ public class NFAITCase extends TestLogger {
 					}
 				}).times(3).consecutive();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<Map<String, List<Event>>> resultingPatterns = new ArrayList<>();
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
-					nfaState,
-					inputEvent.getValue(),
-					inputEvent.getTimestamp()).f0;
+				sharedBuffer,
+				nfaState,
+				inputEvent.getValue(),
+				inputEvent.getTimestamp()).f0;
 
 			resultingPatterns.addAll(patterns);
 		}
@@ -2737,7 +2752,7 @@ public class NFAITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNFAResultKeyOrdering() {
+	public void testNFAResultKeyOrdering() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(41, "a", 2.0);
@@ -2783,14 +2798,15 @@ public class NFAITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<Map<String, List<Event>>> resultingPatterns = new ArrayList<>();
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
+				sharedBuffer,
 				nfaState,
 				inputEvent.getValue(),
 				inputEvent.getTimestamp()).f0;

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStateAccessTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStateAccessTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStateAccessTest.java
new file mode 100644
index 0000000..ede4b23
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStateAccessTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.nfa;
+
+import org.apache.flink.cep.Event;
+import org.apache.flink.cep.SubEvent;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.IterativeCondition;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.cep.utils.TestSharedBuffer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.cep.utils.NFAUtils.compile;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests that check if we do not degrade NFA computation in case of State accesses.
+ */
+public class NFAStateAccessTest {
+
+	@Test
+	public void testComplexBranchingAfterZeroOrMore() throws Exception {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		Event startEvent = new Event(40, "c", 1.0);
+		Event middleEvent1 = new Event(41, "a", 2.0);
+		Event middleEvent2 = new Event(42, "a", 3.0);
+		Event middleEvent3 = new Event(43, "a", 4.0);
+		Event end1 = new Event(44, "b", 5.0);
+		Event end2 = new Event(45, "d", 6.0);
+		Event end3 = new Event(46, "d", 7.0);
+		Event end4 = new Event(47, "e", 8.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent, 1));
+		inputEvents.add(new StreamRecord<>(middleEvent1, 3));
+		inputEvents.add(new StreamRecord<>(middleEvent2, 4));
+		inputEvents.add(new StreamRecord<>(middleEvent3, 5));
+		inputEvents.add(new StreamRecord<>(end1, 6));
+		inputEvents.add(new StreamRecord<>(end2, 7));
+		inputEvents.add(new StreamRecord<>(end3, 8));
+		inputEvents.add(new StreamRecord<>(end4, 9));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("c");
+			}
+		}).followedByAny("middle").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("a");
+			}
+		}).oneOrMore().allowCombinations().optional().followedByAny("end1").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("b");
+			}
+		}).followedByAny("end2").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("d");
+			}
+		}).followedByAny("end3").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 5726188262756267490L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("e");
+			}
+		});
+
+		NFA<Event> nfa = compile(pattern, false);
+
+		TestSharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			nfa.process(
+				sharedBuffer,
+				nfa.createInitialNFAState(),
+				inputEvent.getValue(),
+				inputEvent.getTimestamp());
+		}
+
+		assertEquals(5, sharedBuffer.getStateReads());
+		assertEquals(6, sharedBuffer.getStateWrites());
+		assertEquals(11, sharedBuffer.getStateAccesses());
+	}
+
+	@Test
+	public void testIterativeWithABACPattern() throws Exception {
+		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
+
+		final Event startEvent1 = new Event(40, "start", 1.0);
+		final Event startEvent2 = new Event(40, "start", 2.0);
+		final Event startEvent3 = new Event(40, "start", 3.0);
+		final Event startEvent4 = new Event(40, "start", 4.0);
+		final SubEvent middleEvent1 = new SubEvent(41, "foo1", 1.0, 10);
+		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10);
+		final SubEvent middleEvent3 = new SubEvent(43, "foo3", 3.0, 10);
+		final SubEvent middleEvent4 = new SubEvent(43, "foo4", 1.0, 10);
+		final Event endEvent = new Event(46, "end", 1.0);
+
+		inputEvents.add(new StreamRecord<>(startEvent1, 1L)); //1
+		inputEvents.add(new StreamRecord<Event>(middleEvent1, 2L)); //1
+
+		inputEvents.add(new StreamRecord<>(startEvent2, 2L)); //2
+		inputEvents.add(new StreamRecord<>(startEvent3, 2L)); //3
+		inputEvents.add(new StreamRecord<Event>(middleEvent2, 2L)); //2
+
+		inputEvents.add(new StreamRecord<>(startEvent4, 2L)); //4
+		inputEvents.add(new StreamRecord<Event>(middleEvent3, 2L)); //3
+		inputEvents.add(new StreamRecord<Event>(middleEvent4, 2L)); //1
+		inputEvents.add(new StreamRecord<>(endEvent, 4L));
+
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 6215754202506583964L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("start");
+			}
+		}).followedByAny("middle1").subtype(SubEvent.class).where(new SimpleCondition<SubEvent>() {
+			private static final long serialVersionUID = 2178338526904474690L;
+
+			@Override
+			public boolean filter(SubEvent value) throws Exception {
+				return value.getName().startsWith("foo");
+			}
+		}).followedBy("middle2").where(new IterativeCondition<Event>() {
+			private static final long serialVersionUID = -1223388426808292695L;
+
+			@Override
+			public boolean filter(Event value, Context<Event> ctx) throws Exception {
+				if (!value.getName().equals("start")) {
+					return false;
+				}
+
+				double sum = 0.0;
+				for (Event e : ctx.getEventsForPattern("middle2")) {
+					sum += e.getPrice();
+				}
+				sum += value.getPrice();
+				return Double.compare(sum, 5.0) <= 0;
+			}
+		}).oneOrMore().followedBy("end").where(new SimpleCondition<Event>() {
+			private static final long serialVersionUID = 562590474115118323L;
+
+			@Override
+			public boolean filter(Event value) throws Exception {
+				return value.getName().equals("end");
+			}
+		});
+
+		NFA<Event> nfa = compile(pattern, false);
+
+		TestSharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		for (StreamRecord<Event> inputEvent : inputEvents) {
+			nfa.process(
+				sharedBuffer,
+				nfa.createInitialNFAState(),
+				inputEvent.getValue(),
+				inputEvent.getTimestamp());
+		}
+
+		assertEquals(20, sharedBuffer.getStateReads());
+		assertEquals(24, sharedBuffer.getStateWrites());
+		assertEquals(44, sharedBuffer.getStateAccesses());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
index 3879699..ab9c9b1 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
@@ -20,18 +20,21 @@ package org.apache.flink.cep.nfa;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.cep.utils.TestSharedBuffer;
 import org.apache.flink.streaming.api.windowing.time.Time;
 
+import org.junit.Before;
 import org.junit.Test;
 
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
@@ -40,8 +43,15 @@ import static org.junit.Assert.assertTrue;
  */
 public class NFAStatusChangeITCase {
 
+	private SharedBuffer<Event> sharedBuffer;
+
+	@Before
+	public void init() {
+		this.sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+	}
+
 	@Test
-	public void testNFAChange() {
+	public void testNFAChange() throws Exception {
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 1858562682635302605L;
 
@@ -72,54 +82,53 @@ public class NFAStatusChangeITCase {
 			}
 		}).within(Time.milliseconds(10));
 
-		NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true);
-		NFA<Event> nfa = nfaFactory.createNFA();
+		NFA<Event> nfa = compile(pattern, true);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
-		nfa.process(nfaState, new Event(1, "b", 1.0), 1L);
+		nfa.process(sharedBuffer, nfaState, new Event(1, "b", 1.0), 1L);
 		assertFalse("NFA status should not change as the event does not match the take condition of the 'start' state", nfaState.isStateChanged());
 
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(2, "a", 1.0), 2L);
+		nfa.process(sharedBuffer, nfaState, new Event(2, "a", 1.0), 2L);
 		assertTrue("NFA status should change as the event matches the take condition of the 'start' state", nfaState.isStateChanged());
 
 		// the status of the queue of ComputationStatus changed,
 		// more than one ComputationStatus is generated by the event from some ComputationStatus
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(3, "f", 1.0), 3L);
+		nfa.process(sharedBuffer, nfaState, new Event(3, "f", 1.0), 3L);
 		assertTrue("NFA status should change as the event matches the ignore condition and proceed condition of the 'middle:1' state", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have not changed
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(4, "f", 1.0), 4L);
+		nfa.process(sharedBuffer, nfaState, new Event(4, "f", 1.0), 4L);
 		assertFalse("NFA status should not change as the event only matches the ignore condition of the 'middle:2' state and the target state is still 'middle:2'", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have changed
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(5, "b", 1.0), 5L);
+		nfa.process(sharedBuffer, nfaState, new Event(5, "b", 1.0), 5L);
 		assertTrue("NFA status should change as the event matches the take condition of 'middle:2' state", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have changed
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(6, "d", 1.0), 6L);
+		nfa.process(sharedBuffer, nfaState, new Event(6, "d", 1.0), 6L);
 		assertTrue("NFA status should change as the event matches the take condition of 'middle2' state", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have not changed
 		// as the timestamp is within the window
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, null, 8L);
+		nfa.process(sharedBuffer, nfaState, null, 8L);
 		assertFalse("NFA status should not change as the timestamp is within the window", nfaState.isStateChanged());
 
 		// timeout ComputationStatus will be removed from the queue of ComputationStatus and timeout event will
 		// be removed from eventSharedBuffer as the timeout happens
 		nfaState.resetStateChanged();
-		Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.process(nfaState, null, 12L).f1;
+		Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.process(sharedBuffer, nfaState, null, 12L).f1;
 		assertTrue("NFA status should change as timeout happens", nfaState.isStateChanged() && !timeoutResults.isEmpty());
 	}
 
 	@Test
-	public void testNFAChangedOnOneNewComputationState() {
+	public void testNFAChangedOnOneNewComputationState() throws Exception {
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			@Override
 			public boolean filter(Event value) throws Exception {
@@ -141,21 +150,20 @@ public class NFAStatusChangeITCase {
 			}
 		}).within(Time.milliseconds(10));
 
-		NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true);
-		NFA<Event> nfa = nfaFactory.createNFA();
+		NFA<Event> nfa = compile(pattern,  true);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(6, "start", 1.0), 6L);
+		nfa.process(sharedBuffer, nfaState, new Event(6, "start", 1.0), 6L);
 
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(6, "a", 1.0), 7L);
+		nfa.process(sharedBuffer, nfaState, new Event(6, "a", 1.0), 7L);
 		assertTrue(nfaState.isStateChanged());
 	}
 
 	@Test
-	public void testNFAChangedOnTimeoutWithoutPrune() {
+	public void testNFAChangedOnTimeoutWithoutPrune() throws Exception {
 		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new IterativeCondition<Event>() {
 			@Override
 			public boolean filter(Event value, Context<Event> ctx) throws Exception {
@@ -170,16 +178,15 @@ public class NFAStatusChangeITCase {
 			}
 		}).within(Time.milliseconds(10));
 
-		NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true);
-		NFA<Event> nfa = nfaFactory.createNFA();
+		NFA<Event> nfa = compile(pattern,  true);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(6, "start", 1.0), 6L);
+		nfa.process(sharedBuffer, nfaState, new Event(6, "start", 1.0), 6L);
 
 		nfaState.resetStateChanged();
-		nfa.process(nfaState, new Event(6, "end", 1.0), 17L);
+		nfa.process(sharedBuffer, nfaState, new Event(6, "end", 1.0), 17L);
 		assertTrue(nfaState.isStateChanged());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
index 5d43111..646455f 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
@@ -19,14 +19,14 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.BooleanConditions;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.cep.utils.TestSharedBuffer;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.TestLogger;
 
@@ -34,7 +34,6 @@ import org.junit.Test;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -44,6 +43,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -51,7 +51,7 @@ import static org.junit.Assert.assertEquals;
  */
 public class NFATest extends TestLogger {
 	@Test
-	public void testSimpleNFA() {
+	public void testSimpleNFA() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		streamEvents.add(new StreamRecord<>(new Event(1, "start", 1.0), 1L));
@@ -90,7 +90,7 @@ public class NFATest extends TestLogger {
 		states.add(endState);
 		states.add(endingState);
 
-		NFA<Event> nfa = new NFA<>(Event.createTypeSerializer(), 0, false, states);
+		NFA<Event> nfa = new NFA<>(states, 0, false);
 
 		Set<Map<String, List<Event>>> expectedPatterns = new HashSet<>();
 
@@ -105,14 +105,14 @@ public class NFATest extends TestLogger {
 		expectedPatterns.add(firstPattern);
 		expectedPatterns.add(secondPattern);
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createInitialNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
 
 	@Test
-	public void testTimeoutWindowPruning() {
-		NFA<Event> nfa = createStartEndNFA(2);
+	public void testTimeoutWindowPruning() throws Exception {
+		NFA<Event> nfa = createStartEndNFA();
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		streamEvents.add(new StreamRecord<>(new Event(1, "start", 1.0), 1L));
@@ -128,7 +128,7 @@ public class NFATest extends TestLogger {
 
 		expectedPatterns.add(secondPattern);
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createInitialNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
@@ -138,8 +138,8 @@ public class NFATest extends TestLogger {
 	 * The reason is that the right window side (later elements) is exclusive.
 	 */
 	@Test
-	public void testWindowBorders() {
-		NFA<Event> nfa = createStartEndNFA(2);
+	public void testWindowBorders() throws Exception {
+		NFA<Event> nfa = createStartEndNFA();
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		streamEvents.add(new StreamRecord<>(new Event(1, "start", 1.0), 1L));
@@ -147,7 +147,7 @@ public class NFATest extends TestLogger {
 
 		Set<Map<String, List<Event>>> expectedPatterns = Collections.emptySet();
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createInitialNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
@@ -157,8 +157,8 @@ public class NFATest extends TestLogger {
 	 * semantics (left side inclusive and right side exclusive).
 	 */
 	@Test
-	public void testTimeoutWindowPruningWindowBorders() {
-		NFA<Event> nfa = createStartEndNFA(2);
+	public void testTimeoutWindowPruningWindowBorders() throws Exception {
+		NFA<Event> nfa = createStartEndNFA();
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		streamEvents.add(new StreamRecord<>(new Event(1, "start", 1.0), 1L));
@@ -174,39 +174,19 @@ public class NFATest extends TestLogger {
 
 		expectedPatterns.add(secondPattern);
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createInitialNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
 
-	@Test
-	public void testTimeoutWindowPruning2() throws IOException {
-		NFA<Event> nfa = createLoopingNFA(2);
-		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
-
-		streamEvents.add(new StreamRecord<>(new Event(1, "loop", 1.0), 101L));
-		streamEvents.add(new StreamRecord<>(new Event(2, "loop", 2.0), 102L));
-		streamEvents.add(new StreamRecord<>(new Event(3, "loop", 3.0), 103L));
-		streamEvents.add(new StreamRecord<>(new Event(4, "loop", 4.0), 104L));
-		streamEvents.add(new StreamRecord<>(new Event(5, "loop", 5.0), 105L));
+	public Collection<Map<String, List<Event>>> runNFA(
+		NFA<Event> nfa, NFAState nfaState, List<StreamRecord<Event>> inputs) throws Exception {
+		Set<Map<String, List<Event>>> actualPatterns = new HashSet<>();
 
-		NFAState<Event> nfaState = nfa.createNFAState();
-		runNFA(nfa, nfaState, streamEvents);
-
-		NFAStateSerializer<Event> serializer = new NFAStateSerializer<>(Event.createTypeSerializer());
-
-		//serialize
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		serializer.serialize(nfaState, new DataOutputViewStreamWrapper(baos));
-		baos.close();
-	}
-
-	public <T> Collection<Map<String, List<T>>> runNFA(
-		NFA<T> nfa, NFAState<T> nfaState, List<StreamRecord<T>> inputs) {
-		Set<Map<String, List<T>>> actualPatterns = new HashSet<>();
-
-		for (StreamRecord<T> streamEvent : inputs) {
-			Collection<Map<String, List<T>>> matchedPatterns = nfa.process(
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		for (StreamRecord<Event> streamEvent : inputs) {
+			Collection<Map<String, List<Event>>> matchedPatterns = nfa.process(
+				sharedBuffer,
 				nfaState,
 				streamEvent.getValue(),
 				streamEvent.getTimestamp()).f0;
@@ -218,7 +198,7 @@ public class NFATest extends TestLogger {
 	}
 
 	@Test
-	public void testNFASerialization() throws IOException, ClassNotFoundException {
+	public void testNFASerialization() throws Exception {
 		Pattern<Event, ?> pattern1 = Pattern.<Event>begin("start").where(new SimpleCondition<Event>() {
 			private static final long serialVersionUID = 1858562682635302605L;
 
@@ -267,7 +247,7 @@ public class NFATest extends TestLogger {
 			private static final long serialVersionUID = 8061969839441121955L;
 
 			@Override
-			public boolean filter(Event value, Context<Event> ctx) throws Exception {
+			public boolean filter(Event value, IterativeCondition.Context<Event> ctx) throws Exception {
 				double sum = 0.0;
 				for (Event e : ctx.getEventsForPattern("middle")) {
 					sum += e.getPrice();
@@ -277,37 +257,37 @@ public class NFATest extends TestLogger {
 		});
 
 		Pattern<Event, ?> pattern3 = Pattern.<Event>begin("start")
-				.notFollowedBy("not").where(new SimpleCondition<Event>() {
-			private static final long serialVersionUID = -6085237016591726715L;
+			.notFollowedBy("not").where(new SimpleCondition<Event>() {
+				private static final long serialVersionUID = -6085237016591726715L;
 
-			@Override
-			public boolean filter(Event value) throws Exception {
-				return value.getName().equals("c");
-			}
-		}).followedByAny("middle").where(new SimpleCondition<Event>() {
-			private static final long serialVersionUID = 8061969839441121955L;
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("c");
+				}
+			}).followedByAny("middle").where(new SimpleCondition<Event>() {
+				private static final long serialVersionUID = 8061969839441121955L;
 
-			@Override
-			public boolean filter(Event value) throws Exception {
-				return value.getName().equals("b");
-			}
-		}).oneOrMore().allowCombinations().followedByAny("end").where(new SimpleCondition<Event>() {
-			private static final long serialVersionUID = 8061969839441121955L;
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("b");
+				}
+			}).oneOrMore().allowCombinations().followedByAny("end").where(new SimpleCondition<Event>() {
+				private static final long serialVersionUID = 8061969839441121955L;
 
-			@Override
-			public boolean filter(Event value) throws Exception {
-				return value.getName().equals("d");
-			}
-		});
+				@Override
+				public boolean filter(Event value) throws Exception {
+					return value.getName().equals("d");
+				}
+			});
 
 		List<Pattern<Event, ?>> patterns = new ArrayList<>();
 		patterns.add(pattern1);
 		patterns.add(pattern2);
 		patterns.add(pattern3);
 
-		for (Pattern<Event, ?> p: patterns) {
-			NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(p, Event.createTypeSerializer(), false);
-			NFA<Event> nfa = nfaFactory.createNFA();
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		for (Pattern<Event, ?> p : patterns) {
+			NFA<Event> nfa = compile(p, false);
 
 			Event a = new Event(40, "a", 1.0);
 			Event b = new Event(41, "b", 2.0);
@@ -317,18 +297,18 @@ public class NFATest extends TestLogger {
 			Event b3 = new Event(41, "b", 5.0);
 			Event d = new Event(43, "d", 4.0);
 
-			NFAState<Event> nfaState = nfa.createNFAState();
+			NFAState nfaState = nfa.createInitialNFAState();
 
-			nfa.process(nfaState, a, 1);
-			nfa.process(nfaState, b, 2);
-			nfa.process(nfaState, c, 3);
-			nfa.process(nfaState, b1, 4);
-			nfa.process(nfaState, b2, 5);
-			nfa.process(nfaState, b3, 6);
-			nfa.process(nfaState, d, 7);
-			nfa.process(nfaState, a, 8);
+			nfa.process(sharedBuffer, nfaState, a, 1);
+			nfa.process(sharedBuffer, nfaState, b, 2);
+			nfa.process(sharedBuffer, nfaState, c, 3);
+			nfa.process(sharedBuffer, nfaState, b1, 4);
+			nfa.process(sharedBuffer, nfaState, b2, 5);
+			nfa.process(sharedBuffer, nfaState, b3, 6);
+			nfa.process(sharedBuffer, nfaState, d, 7);
+			nfa.process(sharedBuffer, nfaState, a, 8);
 
-			NFAStateSerializer<Event> serializer = new NFAStateSerializer<>(Event.createTypeSerializer());
+			NFAStateSerializer serializer = NFAStateSerializer.INSTANCE;
 
 			//serialize
 			ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -336,23 +316,22 @@ public class NFATest extends TestLogger {
 			baos.close();
 
 			// copy
-			NFAStateSerializer<Event> copySerializer = new NFAStateSerializer<>(Event.createTypeSerializer());
 			ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray());
 			ByteArrayOutputStream out = new ByteArrayOutputStream();
-			copySerializer.duplicate().copy(new DataInputViewStreamWrapper(in), new DataOutputViewStreamWrapper(out));
+			serializer.duplicate().copy(new DataInputViewStreamWrapper(in), new DataOutputViewStreamWrapper(out));
 			in.close();
 			out.close();
 
 			// deserialize
 			ByteArrayInputStream bais = new ByteArrayInputStream(out.toByteArray());
-			NFAState<Event> copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais));
+			NFAState copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais));
 			bais.close();
 
 			assertEquals(nfaState, copy);
 		}
 	}
 
-	private NFA<Event> createStartEndNFA(long windowLength) {
+	private NFA<Event> createStartEndNFA() {
 
 		State<Event> startState = new State<>("start", State.StateType.Start);
 		State<Event> endState = new State<>("end", State.StateType.Normal);
@@ -385,19 +364,7 @@ public class NFATest extends TestLogger {
 		states.add(endState);
 		states.add(endingState);
 
-		return new NFA<>(Event.createTypeSerializer(), windowLength, false, states);
+		return new NFA<>(states, 2L, false);
 	}
 
-	private NFA<Event> createLoopingNFA(long windowLength) {
-		Pattern<Event, ?> pattern = Pattern.<Event>begin("loop").where(new SimpleCondition<Event>() {
-			private static final long serialVersionUID = 5726188262756267490L;
-
-			@Override
-			public boolean filter(Event value) throws Exception {
-				return value.getName().equals("loop");
-			}
-		}).timesOrMore(3).within(Time.milliseconds(windowLength));
-
-		return NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
index b5312cb..00a3bfd 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
@@ -19,6 +19,8 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
+import org.apache.flink.cep.utils.TestSharedBuffer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 import org.junit.Assert;
@@ -37,33 +39,35 @@ public class NFATestUtilities {
 
 	public static List<List<Event>> feedNFA(
 		List<StreamRecord<Event>> inputEvents,
-		NFA<Event> nfa) {
-		return feedNFA(inputEvents, nfa, nfa.createNFAState(), AfterMatchSkipStrategy.noSkip());
+		NFA<Event> nfa) throws Exception {
+		return feedNFA(inputEvents, nfa, nfa.createInitialNFAState(), AfterMatchSkipStrategy.noSkip());
 	}
 
 	public static List<List<Event>> feedNFA(
 			List<StreamRecord<Event>> inputEvents,
 			NFA<Event> nfa,
-			NFAState<Event> nfaState) {
+			NFAState nfaState) throws Exception {
 		return feedNFA(inputEvents, nfa, nfaState, AfterMatchSkipStrategy.noSkip());
 	}
 
 	public static List<List<Event>> feedNFA(
 		List<StreamRecord<Event>> inputEvents,
 		NFA<Event> nfa,
-		AfterMatchSkipStrategy afterMatchSkipStrategy) {
-		return feedNFA(inputEvents, nfa, nfa.createNFAState(), afterMatchSkipStrategy);
+		AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
+		return feedNFA(inputEvents, nfa, nfa.createInitialNFAState(), afterMatchSkipStrategy);
 	}
 
 	public static List<List<Event>> feedNFA(
 			List<StreamRecord<Event>> inputEvents,
 			NFA<Event> nfa,
-			NFAState<Event> nfaState,
-			AfterMatchSkipStrategy afterMatchSkipStrategy) {
+			NFAState nfaState,
+			AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
 		List<List<Event>> resultingPatterns = new ArrayList<>();
 
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
+				sharedBuffer,
 				nfaState,
 				inputEvent.getValue(),
 				inputEvent.getTimestamp(),

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java
index 9198ff8..11a8484 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -34,6 +33,7 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -43,7 +43,7 @@ import static org.junit.Assert.assertEquals;
 public class NotPatternITCase extends TestLogger {
 
 	@Test
-	public void testNotNext() {
+	public void testNotNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -88,7 +88,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -99,7 +99,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotNextNoMatches() {
+	public void testNotNextNoMatches() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -144,7 +144,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -152,7 +152,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotNextNoMatchesAtTheEnd() {
+	public void testNotNextNoMatchesAtTheEnd() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -197,7 +197,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -205,7 +205,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedBy() {
+	public void testNotFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -250,7 +250,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -260,7 +260,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeOptional() {
+	public void testNotFollowedByBeforeOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -305,7 +305,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -315,7 +315,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesWithNotFollowedBy() {
+	public void testTimesWithNotFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -360,7 +360,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -368,7 +368,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIgnoreStateOfTimesWithNotFollowedBy() {
+	public void testIgnoreStateOfTimesWithNotFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -417,7 +417,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -427,7 +427,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testTimesWithNotFollowedByAfter() {
+	public void testTimesWithNotFollowedByAfter() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -476,7 +476,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -484,7 +484,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeOptionalAtTheEnd() {
+	public void testNotFollowedByBeforeOptionalAtTheEnd() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -520,7 +520,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		}).optional();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -531,7 +531,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeOptionalTimes() {
+	public void testNotFollowedByBeforeOptionalTimes() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -576,7 +576,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -586,7 +586,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByWithBranchingAtStart() {
+	public void testNotFollowedByWithBranchingAtStart() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(40, "a", 1.0);
@@ -633,7 +633,7 @@ public class NotPatternITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> matches = feedNFA(inputEvents, nfa);
 
@@ -658,20 +658,20 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotNextAfterOneOrMoreSkipTillNext() {
+	public void testNotNextAfterOneOrMoreSkipTillNext() throws Exception {
 		final List<List<Event>> matches = testNotNextAfterOneOrMore(false);
 		assertEquals(0, matches.size());
 	}
 
 	@Test
-	public void testNotNextAfterOneOrMoreSkipTillAny() {
+	public void testNotNextAfterOneOrMoreSkipTillAny() throws Exception {
 		final List<List<Event>> matches = testNotNextAfterOneOrMore(true);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_2, NotFollowByData.D_1)
 		));
 	}
 
-	private List<List<Event>> testNotNextAfterOneOrMore(boolean allMatches) {
+	private List<List<Event>> testNotNextAfterOneOrMore(boolean allMatches) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		int i = 0;
@@ -716,19 +716,19 @@ public class NotPatternITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
 
 	@Test
-	public void testNotFollowedByNextAfterOneOrMoreEager() {
+	public void testNotFollowedByNextAfterOneOrMoreEager() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByAfterOneOrMore(true, false);
 		assertEquals(0, matches.size());
 	}
 
 	@Test
-	public void testNotFollowedByAnyAfterOneOrMoreEager() {
+	public void testNotFollowedByAnyAfterOneOrMoreEager() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByAfterOneOrMore(true, true);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1),
@@ -741,13 +741,13 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByNextAfterOneOrMoreCombinations() {
+	public void testNotFollowedByNextAfterOneOrMoreCombinations() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByAfterOneOrMore(false, false);
 		assertEquals(0, matches.size());
 	}
 
 	@Test
-	public void testNotFollowedByAnyAfterOneOrMoreCombinations() {
+	public void testNotFollowedByAnyAfterOneOrMoreCombinations() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByAfterOneOrMore(false, true);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1),
@@ -760,7 +760,7 @@ public class NotPatternITCase extends TestLogger {
 		));
 	}
 
-	private List<List<Event>> testNotFollowedByAfterOneOrMore(boolean eager, boolean allMatches) {
+	private List<List<Event>> testNotFollowedByAfterOneOrMore(boolean eager, boolean allMatches) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		int i = 0;
@@ -812,13 +812,13 @@ public class NotPatternITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
 
 	@Test
-	public void testNotFollowedByAnyBeforeOneOrMoreEager() {
+	public void testNotFollowedByAnyBeforeOneOrMoreEager() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeOneOrMore(true, true);
 
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
@@ -830,7 +830,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByAnyBeforeOneOrMoreCombinations() {
+	public void testNotFollowedByAnyBeforeOneOrMoreCombinations() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeOneOrMore(false, true);
 
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
@@ -846,7 +846,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeOneOrMoreEager() {
+	public void testNotFollowedByBeforeOneOrMoreEager() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeOneOrMore(true, false);
 
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
@@ -858,7 +858,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeOneOrMoreCombinations() {
+	public void testNotFollowedByBeforeOneOrMoreCombinations() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeOneOrMore(false, false);
 
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
@@ -873,7 +873,7 @@ public class NotPatternITCase extends TestLogger {
 		));
 	}
 
-	private List<List<Event>> testNotFollowedByBeforeOneOrMore(boolean eager, boolean allMatches) {
+	private List<List<Event>> testNotFollowedByBeforeOneOrMore(boolean eager, boolean allMatches) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		int i = 0;
@@ -923,13 +923,13 @@ public class NotPatternITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
 
 	@Test
-	public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillNext() {
+	public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillNext() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeZeroOrMore(true, false);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1),
@@ -940,7 +940,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillNext() {
+	public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillNext() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeZeroOrMore(false, false);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1),
@@ -955,7 +955,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillAny() {
+	public void testNotFollowedByBeforeZeroOrMoreEagerSkipTillAny() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeZeroOrMore(true, true);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1),
@@ -966,7 +966,7 @@ public class NotPatternITCase extends TestLogger {
 	}
 
 	@Test
-	public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillAny() {
+	public void testNotFollowedByBeforeZeroOrMoreCombinationsSkipTillAny() throws Exception {
 		final List<List<Event>> matches = testNotFollowedByBeforeZeroOrMore(false, true);
 		compareMaps(matches, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(NotFollowByData.A_1, NotFollowByData.B_1, NotFollowByData.B_4, NotFollowByData.B_5, NotFollowByData.B_6, NotFollowByData.D_1),
@@ -980,7 +980,7 @@ public class NotPatternITCase extends TestLogger {
 		));
 	}
 
-	private List<List<Event>> testNotFollowedByBeforeZeroOrMore(boolean eager, boolean allMatches) {
+	private List<List<Event>> testNotFollowedByBeforeZeroOrMore(boolean eager, boolean allMatches) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		int i = 0;
@@ -1030,7 +1030,7 @@ public class NotPatternITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}


[09/10] flink git commit: [FLINK-9418] Separated pruning and element processing paths

Posted by dw...@apache.org.
[FLINK-9418] Separated pruning and element processing paths

This closes #6059


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

Branch: refs/heads/master
Commit: 05ee3ce96ae51c5df9069564fd5cd2482a62de39
Parents: 9218df8
Author: Dawid Wysakowicz <dw...@apache.org>
Authored: Tue Jun 5 18:57:41 2018 +0200
Committer: Dawid Wysakowicz <dw...@apache.org>
Committed: Wed Jun 13 14:59:59 2018 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/flink/cep/nfa/NFA.java | 135 ++++++++++++-------
 .../org/apache/flink/cep/nfa/SharedBuffer.java  |   4 +-
 .../flink/cep/nfa/sharedbuffer/EventId.java     |  15 ++-
 .../flink/cep/nfa/sharedbuffer/Lockable.java    |   6 +-
 .../cep/nfa/sharedbuffer/SharedBuffer.java      |  87 ++++++------
 .../AbstractKeyedCEPPatternOperator.java        |  43 +++---
 .../org/apache/flink/cep/nfa/NFAITCase.java     |  40 ++++--
 .../flink/cep/nfa/NFAStatusChangeITCase.java    |   7 +-
 .../java/org/apache/flink/cep/nfa/NFATest.java  |   3 +-
 .../apache/flink/cep/nfa/NFATestUtilities.java  |   3 +-
 .../cep/nfa/sharedbuffer/SharedBufferTest.java  |  41 ++++--
 .../flink/cep/utils/TestSharedBuffer.java       |   6 +
 .../flink/runtime/state/KeyedStateFunction.java |   5 +-
 .../api/operators/AbstractStreamOperator.java   |   2 -
 14 files changed, 246 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 227a34d..041a017 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -105,8 +105,7 @@ public class NFA<T> {
 	public NFA(
 			final Collection<State<T>> validStates,
 			final long windowTime,
-			final boolean handleTimeout
-	) {
+			final boolean handleTimeout) {
 		this.windowTime = windowTime;
 		this.handleTimeout = handleTimeout;
 		this.states = loadStates(validStates);
@@ -169,7 +168,6 @@ public class NFA<T> {
 		return stateObject.isFinal();
 	}
 
-
 	/**
 	 * Processes the next input event. If some of the computations reach a final state then the
 	 * resulting event sequences are returned. If computations time out and timeout handling is
@@ -185,8 +183,9 @@ public class NFA<T> {
 	 * @return Tuple of the collection of matched patterns (e.g. the result of computations which have
 	 * reached a final state) and the collection of timed out patterns (if timeout handling is
 	 * activated)
+	 * @throws Exception Thrown if the system cannot access the state.
 	 */
-	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(
+	public Collection<Map<String, List<T>>> process(
 			final SharedBuffer<T> sharedBuffer,
 			final NFAState nfaState,
 			final T event,
@@ -210,61 +209,91 @@ public class NFA<T> {
 	 * @return Tuple of the collection of matched patterns (e.g. the result of computations which have
 	 * reached a final state) and the collection of timed out patterns (if timeout handling is
 	 * activated)
+	 * @throws Exception Thrown if the system cannot access the state.
 	 */
-	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(
+	public Collection<Map<String, List<T>>> process(
 			final SharedBuffer<T> sharedBuffer,
 			final NFAState nfaState,
 			final T event,
 			final long timestamp,
 			final AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
-
 		try (EventWrapper eventWrapper = new EventWrapper(event, timestamp, sharedBuffer)) {
 			return doProcess(sharedBuffer, nfaState, eventWrapper, afterMatchSkipStrategy);
 		}
 	}
 
-	private Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> doProcess(
+	/**
+	 * Prunes states assuming there will be no events with timestamp <b>lower</b> than the given one.
+	 * It cleares the sharedBuffer and also emits all timed out partial matches.
+	 *
+	 * @param sharedBuffer the SharedBuffer object that we need to work upon while processing
+	 * @param nfaState     The NFAState object that we need to affect while processing
+	 * @param timestamp    timestamp that indicates that there will be no more events with lower timestamp
+	 * @return all timed outed partial matches
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public Collection<Tuple2<Map<String, List<T>>, Long>> advanceTime(
 			final SharedBuffer<T> sharedBuffer,
 			final NFAState nfaState,
-			final EventWrapper event,
-			final AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
+			final long timestamp) throws Exception {
 
 		Queue<ComputationState> computationStates = nfaState.getComputationStates();
-
-		final int numberComputationStates = computationStates.size();
-		final Collection<Map<String, List<T>>> result = new ArrayList<>();
 		final Collection<Tuple2<Map<String, List<T>>, Long>> timeoutResult = new ArrayList<>();
 
-		// iterate over all current computations
+		final int numberComputationStates = computationStates.size();
 		for (int i = 0; i < numberComputationStates; i++) {
 			ComputationState computationState = computationStates.poll();
 
-			final Collection<ComputationState> newComputationStates;
-
-			if (!isStartState(computationState) &&
-				windowTime > 0L &&
-				event.getTimestamp() - computationState.getStartTimestamp() >= windowTime) {
+			if (isStateTimedOut(computationState, timestamp)) {
 
 				if (handleTimeout) {
 					// extract the timed out event pattern
 					Map<String, List<T>> timedOutPattern = extractCurrentMatches(sharedBuffer, computationState);
-					timeoutResult.add(Tuple2.of(timedOutPattern, event.getTimestamp()));
+					timeoutResult.add(Tuple2.of(timedOutPattern, timestamp));
 				}
 
 				sharedBuffer.releaseNode(computationState.getPreviousBufferEntry());
 
-				newComputationStates = Collections.emptyList();
 				nfaState.setStateChanged();
-			} else if (event.getEvent() != null) {
-				newComputationStates = computeNextStates(sharedBuffer, computationState, event, event.getTimestamp());
-
-				if (newComputationStates.size() != 1) {
-					nfaState.setStateChanged();
-				} else if (!newComputationStates.iterator().next().equals(computationState)) {
-					nfaState.setStateChanged();
-				}
 			} else {
-				newComputationStates = Collections.singleton(computationState);
+				computationStates.add(computationState);
+			}
+		}
+
+		sharedBuffer.advanceTime(timestamp);
+
+		return timeoutResult;
+	}
+
+	private boolean isStateTimedOut(final ComputationState state, final long timestamp) {
+		return !isStartState(state) && windowTime > 0L && timestamp - state.getStartTimestamp() >= windowTime;
+	}
+
+	private Collection<Map<String, List<T>>> doProcess(
+			final SharedBuffer<T> sharedBuffer,
+			final NFAState nfaState,
+			final EventWrapper event,
+			final AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
+
+		Queue<ComputationState> computationStates = nfaState.getComputationStates();
+
+		final int numberComputationStates = computationStates.size();
+		final Collection<Map<String, List<T>>> result = new ArrayList<>();
+
+		// iterate over all current computations
+		for (int i = 0; i < numberComputationStates; i++) {
+			ComputationState computationState = computationStates.poll();
+
+			final Collection<ComputationState> newComputationStates = computeNextStates(
+				sharedBuffer,
+				computationState,
+				event,
+				event.getTimestamp());
+
+			if (newComputationStates.size() != 1) {
+				nfaState.setStateChanged();
+			} else if (!newComputationStates.iterator().next().equals(computationState)) {
+				nfaState.setStateChanged();
 			}
 
 			//delay adding new computation states in case a stop state is reached and we discard the path.
@@ -299,13 +328,12 @@ public class NFA<T> {
 			} else {
 				computationStates.addAll(statesToRetain);
 			}
-
 		}
 
 		discardComputationStatesAccordingToStrategy(
 			sharedBuffer, computationStates, result, afterMatchSkipStrategy);
 
-		return Tuple2.of(result, timeoutResult);
+		return result;
 	}
 
 	private void discardComputationStatesAccordingToStrategy(
@@ -500,6 +528,7 @@ public class NFA<T> {
 	 * @param event Current event which is processed
 	 * @param timestamp Timestamp of the current event
 	 * @return Collection of computation states which result from the current one
+	 * @throws Exception Thrown if the system cannot access the state.
 	 */
 	private Collection<ComputationState> computeNextStates(
 			final SharedBuffer<T> sharedBuffer,
@@ -558,22 +587,17 @@ public class NFA<T> {
 					final DeweyNumber nextVersion = new DeweyNumber(currentVersion).addStage();
 					takeBranchesToVisit--;
 
-					final NodeId newEntry;
+					final NodeId newEntry = sharedBuffer.put(
+						currentState.getName(),
+						event.getEventId(),
+						previousEntry,
+						currentVersion);
+
 					final long startTimestamp;
 					if (isStartState(computationState)) {
 						startTimestamp = timestamp;
-						newEntry = sharedBuffer.put(
-							currentState.getName(),
-							event.getEventId(),
-							previousEntry,
-							currentVersion);
 					} else {
 						startTimestamp = computationState.getStartTimestamp();
-						newEntry = sharedBuffer.put(
-							currentState.getName(),
-							event.getEventId(),
-							previousEntry,
-							currentVersion);
 					}
 
 					addComputationState(
@@ -631,7 +655,10 @@ public class NFA<T> {
 		sharedBuffer.lockNode(previousEntry);
 	}
 
-	private State<T> findFinalStateAfterProceed(SharedBuffer<T> sharedBuffer, State<T> state, T event,
+	private State<T> findFinalStateAfterProceed(
+			SharedBuffer<T> sharedBuffer,
+			State<T> state,
+			T event,
 			ComputationState computationState) {
 		final Stack<State<T>> statesToCheck = new Stack<>();
 		statesToCheck.push(state);
@@ -661,7 +688,9 @@ public class NFA<T> {
 		return takeBranches == 0 && ignoreBranches == 0 ? 0 : ignoreBranches + Math.max(1, takeBranches);
 	}
 
-	private OutgoingEdges<T> createDecisionGraph(SharedBuffer<T> sharedBuffer, ComputationState computationState,
+	private OutgoingEdges<T> createDecisionGraph(
+			SharedBuffer<T> sharedBuffer,
+			ComputationState computationState,
 			T event) {
 		State<T> state = getState(computationState);
 		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(state);
@@ -699,8 +728,11 @@ public class NFA<T> {
 		return outgoingEdges;
 	}
 
-	private boolean checkFilterCondition(SharedBuffer<T> sharedBuffer, ComputationState computationState,
-			IterativeCondition<T> condition, T event) throws Exception {
+	private boolean checkFilterCondition(
+			SharedBuffer<T> sharedBuffer,
+			ComputationState computationState,
+			IterativeCondition<T> condition,
+			T event) throws Exception {
 		return condition == null || condition.filter(event, new ConditionContext<>(this, sharedBuffer, computationState));
 	}
 
@@ -712,8 +744,10 @@ public class NFA<T> {
 	 * @param sharedBuffer The {@link SharedBuffer} from which to extract the matches
 	 * @param computationState The end computation state of the extracted event sequences
 	 * @return Collection of event sequences which end in the given computation state
+	 * @throws Exception Thrown if the system cannot access the state.
 	 */
-	private Map<String, List<T>> extractCurrentMatches(final SharedBuffer<T> sharedBuffer,
+	private Map<String, List<T>> extractCurrentMatches(
+			final SharedBuffer<T> sharedBuffer,
 			final ComputationState computationState) throws Exception {
 		if (computationState.getPreviousBufferEntry() == null) {
 			return new HashMap<>();
@@ -819,8 +853,7 @@ public class NFA<T> {
 
 		MigratedNFA(
 				final Queue<ComputationState> computationStates,
-				final org.apache.flink.cep.nfa.SharedBuffer<T> sharedBuffer
-		) {
+				final org.apache.flink.cep.nfa.SharedBuffer<T> sharedBuffer) {
 			this.sharedBuffer = sharedBuffer;
 			this.computationStates = computationStates;
 		}
@@ -838,8 +871,8 @@ public class NFA<T> {
 		public NFASerializerConfigSnapshot() {}
 
 		public NFASerializerConfigSnapshot(
-			TypeSerializer<T> eventSerializer,
-			TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufferSerializer) {
+				TypeSerializer<T> eventSerializer,
+				TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufferSerializer) {
 
 			super(eventSerializer, sharedBufferSerializer);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index 7a43537..a4dbc00 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -251,7 +251,7 @@ public class SharedBuffer<V> {
 			Map<ValueTimeWrapper<V>, EventId> values = new HashMap<>();
 			Map<EventId, Lockable<V>> valuesWithIds = new HashMap<>();
 			Map<Tuple2<String, ValueTimeWrapper<V>>, NodeId> mappingContext = new HashMap<>();
-			Map<Long, Long> totalEventsPerTimestamp = new HashMap<>();
+			Map<Long, Integer> totalEventsPerTimestamp = new HashMap<>();
 			int totalPages = source.readInt();
 
 			for (int i = 0; i < totalPages; i++) {
@@ -263,7 +263,7 @@ public class SharedBuffer<V> {
 					ValueTimeWrapper<V> wrapper = ValueTimeWrapper.deserialize(valueSerializer, source);
 					EventId eventId = values.get(wrapper);
 					if (eventId == null) {
-						long id = totalEventsPerTimestamp.computeIfAbsent(wrapper.timestamp, k -> 0L);
+						int id = totalEventsPerTimestamp.computeIfAbsent(wrapper.timestamp, k -> 0);
 						eventId = new EventId(id, wrapper.timestamp);
 						values.put(wrapper, eventId);
 						valuesWithIds.put(eventId, new Lockable<>(wrapper.value, 1));

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
index 9b99ea1..57d244a 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
@@ -19,6 +19,7 @@
 package org.apache.flink.cep.nfa.sharedbuffer;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
 import org.apache.flink.core.memory.DataInputView;
@@ -31,15 +32,15 @@ import java.util.Objects;
  * Composite key for events in {@link SharedBuffer}.
  */
 public class EventId {
-	private final long id;
+	private final int id;
 	private final long timestamp;
 
-	public EventId(long id, long timestamp) {
+	public EventId(int id, long timestamp) {
 		this.id = id;
 		this.timestamp = timestamp;
 	}
 
-	public long getId() {
+	public int getId() {
 		return id;
 	}
 
@@ -110,14 +111,14 @@ public class EventId {
 
 		@Override
 		public void serialize(EventId record, DataOutputView target) throws IOException {
-			LongSerializer.INSTANCE.serialize(record.id, target);
+			IntSerializer.INSTANCE.serialize(record.id, target);
 			LongSerializer.INSTANCE.serialize(record.timestamp, target);
 		}
 
 		@Override
 		public EventId deserialize(DataInputView source) throws IOException {
-			Long id = LongSerializer.INSTANCE.deserialize(source);
-			Long timestamp = LongSerializer.INSTANCE.deserialize(source);
+			int id = IntSerializer.INSTANCE.deserialize(source);
+			long timestamp = LongSerializer.INSTANCE.deserialize(source);
 
 			return new EventId(id, timestamp);
 		}
@@ -129,7 +130,7 @@ public class EventId {
 
 		@Override
 		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			LongSerializer.INSTANCE.copy(source, target);
+			IntSerializer.INSTANCE.copy(source, target);
 			LongSerializer.INSTANCE.copy(source, target);
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
index ca1ecae..b782d8a 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
@@ -62,10 +62,6 @@ public final class Lockable<T> {
 		return refCounter == 0;
 	}
 
-	int getRefCounter() {
-		return refCounter;
-	}
-
 	public T getElement() {
 		return element;
 	}
@@ -143,7 +139,7 @@ public final class Lockable<T> {
 
 		@Override
 		public Lockable<E> deserialize(DataInputView source) throws IOException {
-			Integer refCount = IntSerializer.INSTANCE.deserialize(source);
+			int refCount = IntSerializer.INSTANCE.deserialize(source);
 			E record = elementSerializer.deserialize(source);
 			return new Lockable<>(record, refCount);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
index 50d997c..32be5da 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
@@ -18,10 +18,12 @@
 
 package org.apache.flink.cep.nfa.sharedbuffer;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.state.KeyedStateStore;
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.DeweyNumber;
@@ -33,6 +35,7 @@ import org.apache.commons.lang3.StringUtils;
 import javax.annotation.Nullable;
 
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -69,8 +72,8 @@ public class SharedBuffer<V> {
 	private MapState<EventId, Lockable<V>> eventsBuffer;
 
 	/** The number of events seen so far in the stream per timestamp. */
-	private MapState<Long, Long> eventsCount;
-	private MapState<NodeId, Lockable<SharedBufferNode>> pages;
+	private MapState<Long, Integer> eventsCount;
+	private MapState<NodeId, Lockable<SharedBufferNode>> entries;
 
 	public SharedBuffer(KeyedStateStore stateStore, TypeSerializer<V> valueSerializer) {
 		this.eventsBuffer = stateStore.getMapState(
@@ -79,7 +82,7 @@ public class SharedBuffer<V> {
 				EventId.EventIdSerializer.INSTANCE,
 				new Lockable.LockableTypeSerializer<>(valueSerializer)));
 
-		this.pages = stateStore.getMapState(
+		this.entries = stateStore.getMapState(
 			new MapStateDescriptor<>(
 				entriesStateName,
 				NodeId.NodeIdSerializer.INSTANCE,
@@ -89,7 +92,24 @@ public class SharedBuffer<V> {
 			new MapStateDescriptor<>(
 				eventsCountStateName,
 				LongSerializer.INSTANCE,
-				LongSerializer.INSTANCE));
+				IntSerializer.INSTANCE));
+	}
+
+	/**
+	 * Notifies shared buffer that there will be no events with timestamp &lt;&eq; the given value. I allows to clear
+	 * internal counters for number of events seen so far per timestamp.
+	 *
+	 * @param timestamp watermark, no earlier events will arrive
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public void advanceTime(long timestamp) throws Exception {
+		Iterator<Long> iterator = eventsCount.keys().iterator();
+		while (iterator.hasNext()) {
+			Long next = iterator.next();
+			if (next < timestamp) {
+				iterator.remove();
+			}
+		}
 	}
 
 	/**
@@ -104,14 +124,14 @@ public class SharedBuffer<V> {
 	 * @throws Exception Thrown if the system cannot access the state.
 	 */
 	public EventId registerEvent(V value, long timestamp) throws Exception {
-		Long id = eventsCount.get(timestamp);
+		Integer id = eventsCount.get(timestamp);
 		if (id == null) {
-			id = 0L;
+			id = 0;
 		}
 
 		EventId eventId = new EventId(id, timestamp);
 		eventsBuffer.put(eventId, new Lockable<>(value, 1));
-		eventsCount.put(timestamp, id + 1L);
+		eventsCount.put(timestamp, id + 1);
 		return eventId;
 	}
 
@@ -129,9 +149,9 @@ public class SharedBuffer<V> {
 			Map<EventId, Lockable<V>> events,
 			Map<NodeId, Lockable<SharedBufferNode>> entries) throws Exception {
 		eventsBuffer.putAll(events);
-		pages.putAll(entries);
+		this.entries.putAll(entries);
 
-		Map<Long, Long> maxIds = events.keySet().stream().collect(Collectors.toMap(
+		Map<Long, Integer> maxIds = events.keySet().stream().collect(Collectors.toMap(
 			EventId::getTimestamp,
 			EventId::getId,
 			Math::max
@@ -140,31 +160,13 @@ public class SharedBuffer<V> {
 	}
 
 	/**
-	 * Stores given value (value + timestamp) under the given state. It assigns no preceding element
-	 * relation to the entry.
-	 *
-	 * @param stateName name of the state that the event should be assigned to
-	 * @param eventId   unique id of event assigned by this SharedBuffer
-	 * @param version   Version of the previous relation
-	 * @return assigned id of this entry
-	 * @throws Exception Thrown if the system cannot access the state.
-	 */
-	public NodeId put(
-			final String stateName,
-			final EventId eventId,
-			final DeweyNumber version) throws Exception {
-
-		return put(stateName, eventId, null, version);
-	}
-
-	/**
 	 * Stores given value (value + timestamp) under the given state. It assigns a preceding element
 	 * relation to the previous entry.
 	 *
-	 * @param stateName     name of the state that the event should be assigned to
-	 * @param eventId       unique id of event assigned by this SharedBuffer
-	 * @param previousNodeId id of previous entry
-	 * @param version       Version of the previous relation
+	 * @param stateName      name of the state that the event should be assigned to
+	 * @param eventId        unique id of event assigned by this SharedBuffer
+	 * @param previousNodeId id of previous entry (might be null if start of new run)
+	 * @param version        Version of the previous relation
 	 * @return assigned id of this element
 	 * @throws Exception Thrown if the system cannot access the state.
 	 */
@@ -179,7 +181,7 @@ public class SharedBuffer<V> {
 		}
 
 		NodeId currentNodeId = new NodeId(eventId, getOriginalNameFromInternal(stateName));
-		Lockable<SharedBufferNode> currentNode = pages.get(currentNodeId);
+		Lockable<SharedBufferNode> currentNode = entries.get(currentNodeId);
 		if (currentNode == null) {
 			currentNode = new Lockable<>(new SharedBufferNode(), 0);
 			lockEvent(eventId);
@@ -188,7 +190,7 @@ public class SharedBuffer<V> {
 		currentNode.getElement().addEdge(new SharedBufferEdge(
 			previousNodeId,
 			version));
-		pages.put(currentNodeId, currentNode);
+		entries.put(currentNodeId, currentNode);
 
 		return currentNodeId;
 	}
@@ -221,7 +223,7 @@ public class SharedBuffer<V> {
 		Stack<ExtractionState> extractionStates = new Stack<>();
 
 		// get the starting shared buffer entry for the previous relation
-		Lockable<SharedBufferNode> entryLock = pages.get(nodeId);
+		Lockable<SharedBufferNode> entryLock = entries.get(nodeId);
 
 		if (entryLock != null) {
 			SharedBufferNode entry = entryLock.getElement();
@@ -271,7 +273,7 @@ public class SharedBuffer<V> {
 							}
 
 							extractionStates.push(new ExtractionState(
-								target != null ? Tuple2.of(target, pages.get(target).getElement()) : null,
+								target != null ? Tuple2.of(target, entries.get(target).getElement()) : null,
 								edge.getDeweyNumber(),
 								newPath));
 						}
@@ -291,10 +293,10 @@ public class SharedBuffer<V> {
 	 * @throws Exception Thrown if the system cannot access the state.
 	 */
 	public void lockNode(final NodeId node) throws Exception {
-		Lockable<SharedBufferNode> sharedBufferNode = pages.get(node);
+		Lockable<SharedBufferNode> sharedBufferNode = entries.get(node);
 		if (sharedBufferNode != null) {
 			sharedBufferNode.lock();
-			pages.put(node, sharedBufferNode);
+			entries.put(node, sharedBufferNode);
 		}
 	}
 
@@ -306,18 +308,18 @@ public class SharedBuffer<V> {
 	 * @throws Exception Thrown if the system cannot access the state.
 	 */
 	public void releaseNode(final NodeId node) throws Exception {
-		Lockable<SharedBufferNode> sharedBufferNode = pages.get(node);
+		Lockable<SharedBufferNode> sharedBufferNode = entries.get(node);
 		if (sharedBufferNode != null) {
 			if (sharedBufferNode.release()) {
 				removeNode(node, sharedBufferNode.getElement());
 			} else {
-				pages.put(node, sharedBufferNode);
+				entries.put(node, sharedBufferNode);
 			}
 		}
 	}
 
 	private void removeNode(NodeId node, SharedBufferNode sharedBufferNode) throws Exception {
-		pages.remove(node);
+		entries.remove(node);
 		EventId eventId = node.getEventId();
 		releaseEvent(eventId);
 
@@ -392,4 +394,9 @@ public class SharedBuffer<V> {
 		}
 	}
 
+	@VisibleForTesting
+	Iterator<Map.Entry<Long, Integer>> getEventCounters() throws Exception {
+		return eventsCount.iterator();
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index 73a9200..1e482c3 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -192,7 +192,9 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 			if (comparator == null) {
 				// there can be no out of order elements in processing time
 				NFAState nfaState = getNFAState();
-				processEvent(nfaState, element.getValue(), getProcessingTimeService().getCurrentProcessingTime());
+				long timestamp = getProcessingTimeService().getCurrentProcessingTime();
+				advanceTime(nfaState, timestamp);
+				processEvent(nfaState, element.getValue(), timestamp);
 				updateNFA(nfaState);
 			} else {
 				long currentTime = timerService.currentProcessingTime();
@@ -272,15 +274,16 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		// STEP 2
 		while (!sortedTimestamps.isEmpty() && sortedTimestamps.peek() <= timerService.currentWatermark()) {
 			long timestamp = sortedTimestamps.poll();
+			advanceTime(nfaState, timestamp);
 			try (Stream<IN> elements = sort(elementQueueState.get(timestamp))) {
 				elements.forEachOrdered(
-						event -> {
-							try {
-								processEvent(nfaState, event, timestamp);
-							} catch (Exception e) {
-								throw new RuntimeException(e);
-							}
+					event -> {
+						try {
+							processEvent(nfaState, event, timestamp);
+						} catch (Exception e) {
+							throw new RuntimeException(e);
 						}
+					}
 				);
 			}
 			elementQueueState.remove(timestamp);
@@ -318,15 +321,16 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		// STEP 2
 		while (!sortedTimestamps.isEmpty()) {
 			long timestamp = sortedTimestamps.poll();
+			advanceTime(nfa, timestamp);
 			try (Stream<IN> elements = sort(elementQueueState.get(timestamp))) {
 				elements.forEachOrdered(
-						event -> {
-							try {
-								processEvent(nfa, event, timestamp);
-							} catch (Exception e) {
-								throw new RuntimeException(e);
-							}
+					event -> {
+						try {
+							processEvent(nfa, event, timestamp);
+						} catch (Exception e) {
+							throw new RuntimeException(e);
 						}
+					}
 				);
 			}
 			elementQueueState.remove(timestamp);
@@ -377,18 +381,19 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	 * @param timestamp The timestamp of the event
 	 */
 	private void processEvent(NFAState nfaState, IN event, long timestamp) throws Exception {
-		Tuple2<Collection<Map<String, List<IN>>>, Collection<Tuple2<Map<String, List<IN>>, Long>>> patterns =
+		Collection<Map<String, List<IN>>> patterns =
 				nfa.process(partialMatches, nfaState, event, timestamp, afterMatchSkipStrategy);
-		processMatchedSequences(patterns.f0, timestamp);
-		processTimedOutSequences(patterns.f1, timestamp);
+		processMatchedSequences(patterns, timestamp);
 	}
 
 	/**
-	 * Advances the time for the given NFA to the given timestamp. This can lead to pruning and
-	 * timeouts.
+	 * Advances the time for the given NFA to the given timestamp. This means that no more events with timestamp
+	 * <b>lower</b> than the given timestamp should be passed to the nfa, This can lead to pruning and timeouts.
 	 */
 	private void advanceTime(NFAState nfaState, long timestamp) throws Exception {
-		processEvent(nfaState, null, timestamp);
+		Collection<Tuple2<Map<String, List<IN>>, Long>> timedOut =
+			nfa.advanceTime(partialMatches, nfaState, timestamp);
+		processTimedOutSequences(timedOut, timestamp);
 	}
 
 	protected abstract void processMatchedSequences(Iterable<Map<String, List<IN>>> matchingSequences, long timestamp) throws Exception;

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index 7e904c8..ae68d02 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -35,6 +35,7 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -49,6 +50,7 @@ import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
 import static org.apache.flink.cep.utils.NFAUtils.compile;
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyLong;
 
 /**
  * General tests for {@link NFA} features. See also {@link IterativeConditionsITCase}, {@link NotPatternITCase},
@@ -399,11 +401,11 @@ public class NFAITCase extends TestLogger {
 		NFAState nfaState = nfa.createInitialNFAState();
 
 		for (StreamRecord<Event> event: events) {
-			Tuple2<Collection<Map<String, List<Event>>>, Collection<Tuple2<Map<String, List<Event>>, Long>>> patterns =
-					nfa.process(sharedBuffer, nfaState, event.getValue(), event.getTimestamp());
 
-			Collection<Map<String, List<Event>>> matchedPatterns = patterns.f0;
-			Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutPatterns = patterns.f1;
+			Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutPatterns =
+				nfa.advanceTime(sharedBuffer, nfaState, event.getTimestamp());
+			Collection<Map<String, List<Event>>> matchedPatterns =
+				nfa.process(sharedBuffer, nfaState, event.getValue(), event.getTimestamp());
 
 			resultingPatterns.addAll(matchedPatterns);
 			resultingTimeoutPatterns.addAll(timeoutPatterns);
@@ -2338,7 +2340,7 @@ public class NFAITCase extends TestLogger {
 		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(sharedBuffer, nfaState, null, 10);
+		nfa.advanceTime(sharedBuffer, nfaState, 10);
 
 		assertEquals(1, nfaState.getComputationStates().size());
 		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
@@ -2382,7 +2384,7 @@ public class NFAITCase extends TestLogger {
 		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(sharedBuffer, nfaState, null, 10);
+		nfa.advanceTime(sharedBuffer, nfaState, 10);
 
 		assertEquals(1, nfaState.getComputationStates().size());
 		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
@@ -2428,7 +2430,7 @@ public class NFAITCase extends TestLogger {
 		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(sharedBuffer, nfaState, null, 10);
+		nfa.advanceTime(sharedBuffer, nfaState, 10);
 
 		assertEquals(1, nfaState.getComputationStates().size());
 		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
@@ -2474,7 +2476,7 @@ public class NFAITCase extends TestLogger {
 		nfa.process(sharedBuffer, nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(sharedBuffer, nfaState, null, 10);
+		nfa.advanceTime(sharedBuffer, nfaState, 10);
 
 		assertEquals(1, nfaState.getComputationStates().size());
 		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
@@ -2734,7 +2736,7 @@ public class NFAITCase extends TestLogger {
 				sharedBuffer,
 				nfaState,
 				inputEvent.getValue(),
-				inputEvent.getTimestamp()).f0;
+				inputEvent.getTimestamp());
 
 			resultingPatterns.addAll(patterns);
 		}
@@ -2809,7 +2811,7 @@ public class NFAITCase extends TestLogger {
 				sharedBuffer,
 				nfaState,
 				inputEvent.getValue(),
-				inputEvent.getTimestamp()).f0;
+				inputEvent.getTimestamp());
 
 			resultingPatterns.addAll(patterns);
 		}
@@ -2826,4 +2828,22 @@ public class NFAITCase extends TestLogger {
 
 		Assert.assertEquals(expectedOrder, resultOrder);
 	}
+
+	@Test
+	public void testSharedBufferClearing() throws Exception {
+		Pattern<Event, ?> pattern = Pattern.<Event>begin("start").followedBy("end");
+
+		Event a = new Event(40, "a", 1.0);
+		Event b = new Event(41, "b", 2.0);
+
+		SharedBuffer<Event> spiedBuffer = Mockito.spy(sharedBuffer);
+		NFA<Event> nfa = compile(pattern, false);
+
+		nfa.process(spiedBuffer, nfa.createInitialNFAState(), a, 1);
+		nfa.process(spiedBuffer, nfa.createInitialNFAState(), b, 2);
+		Mockito.verify(spiedBuffer, Mockito.never()).advanceTime(anyLong());
+		nfa.advanceTime(spiedBuffer, nfa.createInitialNFAState(), 2);
+		Mockito.verify(spiedBuffer, Mockito.times(1)).advanceTime(2);
+
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
index ab9c9b1..ea20bee 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
@@ -117,13 +117,13 @@ public class NFAStatusChangeITCase {
 		// both the queue of ComputationStatus and eventSharedBuffer have not changed
 		// as the timestamp is within the window
 		nfaState.resetStateChanged();
-		nfa.process(sharedBuffer, nfaState, null, 8L);
+		nfa.advanceTime(sharedBuffer, nfaState, 8L);
 		assertFalse("NFA status should not change as the timestamp is within the window", nfaState.isStateChanged());
 
 		// timeout ComputationStatus will be removed from the queue of ComputationStatus and timeout event will
 		// be removed from eventSharedBuffer as the timeout happens
 		nfaState.resetStateChanged();
-		Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.process(sharedBuffer, nfaState, null, 12L).f1;
+		Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.advanceTime(sharedBuffer, nfaState, 12L);
 		assertTrue("NFA status should change as timeout happens", nfaState.isStateChanged() && !timeoutResults.isEmpty());
 	}
 
@@ -183,10 +183,11 @@ public class NFAStatusChangeITCase {
 		NFAState nfaState = nfa.createInitialNFAState();
 
 		nfaState.resetStateChanged();
+		nfa.advanceTime(sharedBuffer, nfaState, 6L);
 		nfa.process(sharedBuffer, nfaState, new Event(6, "start", 1.0), 6L);
 
 		nfaState.resetStateChanged();
-		nfa.process(sharedBuffer, nfaState, new Event(6, "end", 1.0), 17L);
+		nfa.advanceTime(sharedBuffer, nfaState, 17L);
 		assertTrue(nfaState.isStateChanged());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
index 646455f..e626b50 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
@@ -185,11 +185,12 @@ public class NFATest extends TestLogger {
 
 		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
 		for (StreamRecord<Event> streamEvent : inputs) {
+			nfa.advanceTime(sharedBuffer, nfaState, streamEvent.getTimestamp());
 			Collection<Map<String, List<Event>>> matchedPatterns = nfa.process(
 				sharedBuffer,
 				nfaState,
 				streamEvent.getValue(),
-				streamEvent.getTimestamp()).f0;
+				streamEvent.getTimestamp());
 
 			actualPatterns.addAll(matchedPatterns);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
index 00a3bfd..58ba224 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
@@ -66,12 +66,13 @@ public class NFATestUtilities {
 
 		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
 		for (StreamRecord<Event> inputEvent : inputEvents) {
+			nfa.advanceTime(sharedBuffer, nfaState, inputEvent.getTimestamp());
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
 				sharedBuffer,
 				nfaState,
 				inputEvent.getValue(),
 				inputEvent.getTimestamp(),
-				afterMatchSkipStrategy).f0;
+				afterMatchSkipStrategy);
 
 			for (Map<String, List<Event>> p: patterns) {
 				List<Event> res = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
index 9be7cc1..342c9ef 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferTest.java
@@ -29,6 +29,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -93,9 +94,9 @@ public class SharedBufferTest extends TestLogger {
 		expectedPattern3.put("b", new ArrayList<>());
 		expectedPattern3.get("b").add(events[7]);
 
-		NodeId a10 = sharedBuffer.put("a1", eventIds[0], DeweyNumber.fromString("1"));
+		NodeId a10 = sharedBuffer.put("a1", eventIds[0], null, DeweyNumber.fromString("1"));
 		NodeId aLoop0 = sharedBuffer.put("a[]", eventIds[1], a10, DeweyNumber.fromString("1.0"));
-		NodeId a11 = sharedBuffer.put("a1", eventIds[2], DeweyNumber.fromString("2"));
+		NodeId a11 = sharedBuffer.put("a1", eventIds[2], null, DeweyNumber.fromString("2"));
 		NodeId aLoop1 = sharedBuffer.put("a[]", eventIds[2], aLoop0, DeweyNumber.fromString("1.0"));
 		NodeId aLoop2 = sharedBuffer.put("a[]", eventIds[3], aLoop1, DeweyNumber.fromString("1.0"));
 		NodeId aSecondLoop0 = sharedBuffer.put("a[]", eventIds[3], a11, DeweyNumber.fromString("2.0"));
@@ -106,12 +107,16 @@ public class SharedBufferTest extends TestLogger {
 		NodeId aLoop5 = sharedBuffer.put("a[]", eventIds[6], aLoop4, DeweyNumber.fromString("1.1"));
 		NodeId b3 = sharedBuffer.put("b", eventIds[7], aLoop5, DeweyNumber.fromString("1.1.0"));
 
-		Collection<Map<String, List<Event>>> patterns3 = sharedBuffer.extractPatterns(b3, DeweyNumber.fromString("1.1.0"));
+		Collection<Map<String, List<Event>>> patterns3 = sharedBuffer.extractPatterns(b3,
+			DeweyNumber.fromString("1.1.0"));
 		sharedBuffer.releaseNode(b3);
-		Collection<Map<String, List<Event>>> patterns4 = sharedBuffer.extractPatterns(b3, DeweyNumber.fromString("1.1.0"));
+		Collection<Map<String, List<Event>>> patterns4 = sharedBuffer.extractPatterns(b3,
+			DeweyNumber.fromString("1.1.0"));
 
-		Collection<Map<String, List<Event>>> patterns1 = sharedBuffer.extractPatterns(b1, DeweyNumber.fromString("2.0.0"));
-		Collection<Map<String, List<Event>>> patterns2 = sharedBuffer.extractPatterns(b0, DeweyNumber.fromString("1.0.0"));
+		Collection<Map<String, List<Event>>> patterns1 = sharedBuffer.extractPatterns(b1,
+			DeweyNumber.fromString("2.0.0"));
+		Collection<Map<String, List<Event>>> patterns2 = sharedBuffer.extractPatterns(b0,
+			DeweyNumber.fromString("1.0.0"));
 		sharedBuffer.releaseNode(b0);
 		sharedBuffer.releaseNode(b1);
 
@@ -144,7 +149,7 @@ public class SharedBufferTest extends TestLogger {
 			eventIds[i] = sharedBuffer.registerEvent(events[i], timestamp);
 		}
 
-		NodeId start = sharedBuffer.put("start", eventIds[1], DeweyNumber.fromString("1"));
+		NodeId start = sharedBuffer.put("start", eventIds[1], null, DeweyNumber.fromString("1"));
 		NodeId b0 = sharedBuffer.put("branching", eventIds[2], start, DeweyNumber.fromString("1.0"));
 		NodeId b1 = sharedBuffer.put("branching", eventIds[3], start, DeweyNumber.fromString("1.1"));
 		NodeId b00 = sharedBuffer.put("branching", eventIds[3], b0, DeweyNumber.fromString("1.0.0"));
@@ -189,7 +194,7 @@ public class SharedBufferTest extends TestLogger {
 		expectedResult.put("c", new ArrayList<>());
 		expectedResult.get("c").add(events[4]);
 
-		NodeId a = sharedBuffer.put("a", eventIds[0], DeweyNumber.fromString("1"));
+		NodeId a = sharedBuffer.put("a", eventIds[0], null, DeweyNumber.fromString("1"));
 		NodeId b = sharedBuffer.put("b", eventIds[1], a, DeweyNumber.fromString("1.0"));
 		NodeId aa = sharedBuffer.put("aa", eventIds[2], b, DeweyNumber.fromString("1.0.0"));
 		NodeId bb = sharedBuffer.put("bb", eventIds[3], aa, DeweyNumber.fromString("1.0.0.0"));
@@ -213,4 +218,24 @@ public class SharedBufferTest extends TestLogger {
 		assertEquals(expectedOrder, resultOrder);
 	}
 
+	@Test
+	public void testSharedBufferCountersClearing() throws Exception {
+		SharedBuffer<Event> sharedBuffer = TestSharedBuffer.createTestBuffer(Event.createTypeSerializer());
+		int numberEvents = 4;
+		Event[] events = new Event[numberEvents];
+
+		for (int i = 0; i < numberEvents; i++) {
+			events[i] = new Event(i + 1, "e" + (i + 1), i);
+			sharedBuffer.registerEvent(events[i], i);
+		}
+
+		sharedBuffer.advanceTime(3);
+
+		Iterator<Map.Entry<Long, Integer>> counters = sharedBuffer.getEventCounters();
+		Map.Entry<Long, Integer> entry = counters.next();
+		assertEquals(3, entry.getKey().longValue());
+		assertEquals(1, entry.getValue().intValue());
+		assertFalse(counters.hasNext());
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
index 2c7b979..4d510cf 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java
@@ -245,6 +245,12 @@ public class TestSharedBuffer<V> extends SharedBuffer<V> {
 				stateReads++;
 				return iterator.next();
 			}
+
+			@Override
+			public void remove() {
+				stateWrites++;
+				iterator.remove();
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
index 5adce4d..b125de9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.api.common.state.State;
@@ -22,8 +23,8 @@ import org.apache.flink.api.common.state.State;
 /**
  * A function to be applied to all keyed states.
  *
- * <p>This functionality is only available through the
- * {@code BroadcastConnectedStream.process(final KeyedBroadcastProcessFunction function)}.
+ * @param <K> The type of key.
+ * @param <S> The type of state.
  */
 public abstract class KeyedStateFunction<K, S extends State> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/05ee3ce9/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index c193416..9915dd5 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -396,8 +396,6 @@ public abstract class AbstractStreamOperator<OUT>
 		return snapshotInProgress;
 	}
 
-
-
 	/**
 	 * Stream operators with state, which want to participate in a snapshot need to override this hook method.
 	 *


[07/10] flink git commit: [FLINK-9418] Migrate SharedBuffer to use MapState

Posted by dw...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index a00a310..7a43537 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -1,8 +1,8 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOVICE file
+ * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
- * regarding copyright ownership.  Vhe ASF licenses this file
+ * 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
@@ -26,635 +26,54 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler;
+import org.apache.flink.cep.nfa.sharedbuffer.EventId;
+import org.apache.flink.cep.nfa.sharedbuffer.Lockable;
+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.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.commons.lang3.StringUtils;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Set;
-import java.util.Stack;
+import java.util.stream.Collectors;
 
 /**
- * A shared buffer implementation which stores values under a key. Additionally, the values can be
- * versioned such that it is possible to retrieve their predecessor element in the buffer.
- *
- * <p>The idea of the implementation is to have for each key a dedicated {@link SharedBufferPage}. Each
- * buffer page maintains a collection of the inserted values.
- *
- * <p>The values are wrapped in a {@link SharedBufferEntry}. The shared buffer entry allows to store
- * relations between different entries. A dewey versioning scheme allows to discriminate between
- * different relations (e.g. preceding element).
- *
- * <p>The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams".
- *
- * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">
- *     https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
- *
- * @param <K> Type of the keys
- * @param <V> Type of the values
+ * @deprecated everything in this class is deprecated. Those are only migration procedures from older versions.
  */
-public class SharedBuffer<K, V> {
-
-	private Map<K, SharedBufferPage<K, V>> pages;
-
-	public SharedBuffer() {
-		this.pages = new HashMap<>(4);
-	}
-
-	/**
-	 * Stores given value (value + timestamp) under the given key. It assigns a preceding element
-	 * relation to the entry which is defined by the previous key, value (value + timestamp).
-	 *
-	 * @param key               Key of the current value
-	 * @param value             Current value
-	 * @param timestamp         Timestamp of the current value (a value requires always a timestamp to make it uniquely referable))
-	 * @param previousKey       Key of the value for the previous relation
-	 * @param previousValue     Value for the previous relation
-	 * @param previousTimestamp Timestamp of the value for the previous relation
-	 * @param version           Version of the previous relation
-	 */
-	public int put(
-			final K key,
-			final V value,
-			final long timestamp,
-			final K previousKey,
-			final V previousValue,
-			final long previousTimestamp,
-			final int previousCounter,
-			final DeweyNumber version) {
-
-		final SharedBufferEntry<K, V> previousSharedBufferEntry =
-				get(previousKey, previousValue, previousTimestamp, previousCounter);
-
-		// sanity check whether we've found the previous element
-		if (previousSharedBufferEntry == null && previousValue != null) {
-			throw new IllegalStateException("Could not find previous entry with " +
-				"key: " + previousKey + ", value: " + previousValue + " and timestamp: " +
-				previousTimestamp + ". This can indicate that either you did not implement " +
-				"the equals() and hashCode() methods of your input elements properly or that " +
-				"the element belonging to that entry has been already pruned.");
-		}
-
-		return put(key, value, timestamp, previousSharedBufferEntry, version);
-	}
-
-	/**
-	 * Stores given value (value + timestamp) under the given key. It assigns no preceding element
-	 * relation to the entry.
-	 *
-	 * @param key       Key of the current value
-	 * @param value     Current value
-	 * @param timestamp Timestamp of the current value (a value requires always a timestamp to make it uniquely referable))
-	 * @param version   Version of the previous relation
-	 */
-	public int put(
-			final K key,
-			final V value,
-			final long timestamp,
-			final DeweyNumber version) {
+@Deprecated
+public class SharedBuffer<V> {
 
-		return put(key, value, timestamp, null, version);
-	}
+	private final Map<Tuple2<String, ValueTimeWrapper<V>>, NodeId> mappingContext;
+	private final Map<EventId, Lockable<V>> eventsBuffer;
+	private final Map<NodeId, Lockable<SharedBufferNode>> pages;
 
-	private int put(
-			final K key,
-			final V value,
-			final long timestamp,
-			final SharedBufferEntry<K, V> previousSharedBufferEntry,
-			final DeweyNumber version) {
-
-		SharedBufferPage<K, V> page = pages.get(key);
-		if (page == null) {
-			page = new SharedBufferPage<>(key);
-			pages.put(key, page);
-		}
-
-		// this assumes that elements are processed in order (in terms of time)
-		int counter = 0;
-		if (previousSharedBufferEntry != null) {
-			ValueTimeWrapper<V> prev = previousSharedBufferEntry.getValueTime();
-			if (prev != null && prev.getTimestamp() == timestamp) {
-				counter = prev.getCounter() + 1;
-			}
-		}
-		page.add(new ValueTimeWrapper<>(value, timestamp, counter), previousSharedBufferEntry, version);
-		return counter;
+	public Map<EventId, Lockable<V>> getEventsBuffer() {
+		return eventsBuffer;
 	}
 
-	public boolean isEmpty() {
-		for (SharedBufferPage<K, V> page: pages.values()) {
-			if (!page.isEmpty()) {
-				return false;
-			}
-		}
-		return true;
+	public Map<NodeId, Lockable<SharedBufferNode>> getPages() {
+		return pages;
 	}
 
-	/**
-	 * Deletes all entries in each page which have expired with respect to given pruning timestamp.
-	 *
-	 * @param pruningTimestamp The time which is used for pruning. All elements whose timestamp is
-	 *                         lower than the pruning timestamp will be removed.
-	 * @return {@code true} if pruning happened
-	 */
-	public boolean prune(long pruningTimestamp) {
-		final Set<SharedBufferEntry<K, V>> prunedEntries = new HashSet<>();
-
-		final Iterator<Map.Entry<K, SharedBufferPage<K, V>>> it = pages.entrySet().iterator();
-		while (it.hasNext()) {
-			SharedBufferPage<K, V> page = it.next().getValue();
-
-			page.prune(pruningTimestamp, prunedEntries);
-			if (page.isEmpty()) {
-				it.remove();
-			}
-		}
-
-		if (prunedEntries.isEmpty()) {
-			return false;
-		}
+	public SharedBuffer(
+			Map<EventId, Lockable<V>> eventsBuffer,
+			Map<NodeId, Lockable<SharedBufferNode>> pages,
+			Map<Tuple2<String, ValueTimeWrapper<V>>, NodeId> mappingContext) {
 
-		for (SharedBufferPage<K, V> entry : pages.values()) {
-			entry.removeEdges(prunedEntries);
-		}
-		return true;
-	}
-
-	/**
-	 * Returns all elements from the previous relation starting at the given value with the
-	 * given key and timestamp.
-	 *
-	 * @param key Key of the starting value
-	 * @param value Value of the starting element
-	 * @param timestamp Timestamp of the starting value
-	 * @param version Version of the previous relation which shall be extracted
-	 * @return Collection of previous relations starting with the given value
-	 */
-	public List<Map<K, List<V>>> extractPatterns(
-			final K key,
-			final V value,
-			final long timestamp,
-			final int counter,
-			final DeweyNumber version) {
-
-		List<Map<K, List<V>>> result = new ArrayList<>();
-
-		// stack to remember the current extraction states
-		Stack<ExtractionState<K, V>> extractionStates = new Stack<>();
-
-		// get the starting shared buffer entry for the previous relation
-		SharedBufferEntry<K, V> entry = get(key, value, timestamp, counter);
-
-		if (entry != null) {
-			extractionStates.add(new ExtractionState<>(entry, version, new Stack<>()));
-
-			// use a depth first search to reconstruct the previous relations
-			while (!extractionStates.isEmpty()) {
-				final ExtractionState<K, V> extractionState = extractionStates.pop();
-				// current path of the depth first search
-				final Stack<SharedBufferEntry<K, V>> currentPath = extractionState.getPath();
-				final SharedBufferEntry<K, V> currentEntry = extractionState.getEntry();
-
-				// termination criterion
-				if (currentEntry == null) {
-					final Map<K, List<V>> completePath = new LinkedHashMap<>();
-
-					while (!currentPath.isEmpty()) {
-						final SharedBufferEntry<K, V> currentPathEntry = currentPath.pop();
-
-						K k = currentPathEntry.getKey();
-						List<V> values = completePath.get(k);
-						if (values == null) {
-							values = new ArrayList<>();
-							completePath.put(k, values);
-						}
-						values.add(currentPathEntry.getValueTime().getValue());
-					}
-					result.add(completePath);
-				} else {
-
-					// append state to the path
-					currentPath.push(currentEntry);
-
-					boolean firstMatch = true;
-					for (SharedBufferEdge<K, V> edge : currentEntry.getEdges()) {
-						// we can only proceed if the current version is compatible to the version
-						// of this previous relation
-						final DeweyNumber currentVersion = extractionState.getVersion();
-						if (currentVersion.isCompatibleWith(edge.getVersion())) {
-							if (firstMatch) {
-								// for the first match we don't have to copy the current path
-								extractionStates.push(new ExtractionState<>(edge.getTarget(), edge.getVersion(), currentPath));
-								firstMatch = false;
-							} else {
-								final Stack<SharedBufferEntry<K, V>> copy = new Stack<>();
-								copy.addAll(currentPath);
-
-								extractionStates.push(
-									new ExtractionState<>(
-										edge.getTarget(),
-										edge.getVersion(),
-										copy));
-							}
-						}
-					}
-				}
-
-			}
-		}
-		return result;
-	}
-
-	/**
-	 * Increases the reference counter for the given value, key, timestamp entry so that it is not
-	 * accidentally removed.
-	 *
-	 * @param key       Key of the value to lock
-	 * @param value     Value to lock
-	 * @param timestamp Timestamp of the value to lock
-	 */
-	public void lock(final K key, final V value, final long timestamp, int counter) {
-		SharedBufferEntry<K, V> entry = get(key, value, timestamp, counter);
-		if (entry != null) {
-			entry.increaseReferenceCounter();
-		}
-	}
-
-	/**
-	 * Decreases the reference counter for the given value, key, timestamp entry so that it can be
-	 * removed once the reference counter reaches 0.
-	 *
-	 * @param key       Key of the value to release
-	 * @param value     Value to release
-	 * @param timestamp Timestamp of the value to release
-	 */
-	public void release(final K key, final V value, final long timestamp, int counter) {
-		SharedBufferEntry<K, V> entry = get(key, value, timestamp, counter);
-		if (entry != null) {
-			internalRemove(entry);
-		}
-	}
-
-	private SharedBuffer(Map<K, SharedBufferPage<K, V>> pages) {
+		this.eventsBuffer = eventsBuffer;
 		this.pages = pages;
+		this.mappingContext = mappingContext;
 	}
 
-	private SharedBufferEntry<K, V> get(
-			final K key,
-			final V value,
-			final long timestamp,
-			final int counter) {
-		SharedBufferPage<K, V> page = pages.get(key);
-		return page == null ? null : page.get(new ValueTimeWrapper<>(value, timestamp, counter));
-	}
-
-	private void internalRemove(final SharedBufferEntry<K, V> entry) {
-		Stack<SharedBufferEntry<K, V>> entriesToRemove = new Stack<>();
-		entriesToRemove.add(entry);
-
-		while (!entriesToRemove.isEmpty()) {
-			SharedBufferEntry<K, V> currentEntry = entriesToRemove.pop();
-			currentEntry.decreaseReferenceCounter();
-
-			if (currentEntry.getReferenceCounter() == 0) {
-				currentEntry.remove();
-
-				for (SharedBufferEdge<K, V> edge : currentEntry.getEdges()) {
-					if (edge.getTarget() != null) {
-						entriesToRemove.push(edge.getTarget());
-					}
-				}
-			}
-		}
-	}
-
-	@Override
-	public String toString() {
-		StringBuilder builder = new StringBuilder();
-
-		for (Map.Entry<K, SharedBufferPage<K, V>> entry : pages.entrySet()) {
-			builder.append("Key: ").append(entry.getKey()).append(System.lineSeparator());
-			builder.append("Value: ").append(entry.getValue()).append(System.lineSeparator());
-		}
-
-		return builder.toString();
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof SharedBuffer) {
-			@SuppressWarnings("unchecked")
-			SharedBuffer<K, V> other = (SharedBuffer<K, V>) obj;
-
-			return pages.equals(other.pages);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return Objects.hash(pages);
-	}
-
-	/**
-	 * The SharedBufferPage represents a set of elements which have been stored under the same key.
-	 *
-	 * @param <K> Type of the key
-	 * @param <V> Type of the value
-	 */
-	private static class SharedBufferPage<K, V> {
-
-		private final K key;
-		private final Map<ValueTimeWrapper<V>, SharedBufferEntry<K, V>> entries;
-
-		SharedBufferPage(final K key) {
-			this.key = key;
-			entries = new HashMap<>();
-		}
-
-		public K getKey() {
-			return key;
-		}
-
-		/**
-		 * Adds a new value time pair to the page. The new entry is linked to the previous entry
-		 * with the given version.
-		 *
-		 * @param valueTime Value time pair to be stored
-		 * @param previous Previous shared buffer entry to which the new entry shall be linked
-		 * @param version Version of the relation between the new and the previous entry
-		 */
-		public void add(final ValueTimeWrapper<V> valueTime, final SharedBufferEntry<K, V> previous, final DeweyNumber version) {
-			SharedBufferEntry<K, V> sharedBufferEntry = entries.get(valueTime);
-			if (sharedBufferEntry == null) {
-				sharedBufferEntry = new SharedBufferEntry<>(valueTime, this);
-				entries.put(valueTime, sharedBufferEntry);
-			}
-
-			SharedBufferEdge<K, V> newEdge;
-			if (previous != null) {
-				newEdge = new SharedBufferEdge<>(previous, version);
-				previous.increaseReferenceCounter();
-			} else {
-				newEdge = new SharedBufferEdge<>(null, version);
-			}
-			sharedBufferEntry.addEdge(newEdge);
-		}
-
-		public SharedBufferEntry<K, V> get(final ValueTimeWrapper<V> valueTime) {
-			return entries.get(valueTime);
-		}
-
-		/**
-		 * Removes all entries from the map whose timestamp is smaller than the pruning timestamp.
-		 * @param pruningTimestamp Timestamp for the pruning
-		 * @param prunedEntries a {@link Set} to put the removed {@link SharedBufferEntry SharedBufferEntries}.
-		 */
-		private void prune(final long pruningTimestamp, final Set<SharedBufferEntry<K, V>> prunedEntries) {
-			Iterator<Map.Entry<ValueTimeWrapper<V>, SharedBufferEntry<K, V>>> it = entries.entrySet().iterator();
-			while (it.hasNext()) {
-				SharedBufferEntry<K, V> entry = it.next().getValue();
-				if (entry.getValueTime().getTimestamp() <= pruningTimestamp) {
-					prunedEntries.add(entry);
-					it.remove();
-				}
-			}
-		}
-
-		/**
-		 * Remove edges with the specified targets for the entries.
-		 */
-		private void removeEdges(final Set<SharedBufferEntry<K, V>> prunedEntries) {
-			for (SharedBufferEntry<K, V> entry : entries.values()) {
-				entry.removeEdges(prunedEntries);
-			}
-		}
-
-		public SharedBufferEntry<K, V> remove(final ValueTimeWrapper<V> valueTime) {
-			return entries.remove(valueTime);
-		}
-
-		public boolean isEmpty() {
-			return entries.isEmpty();
-		}
-
-		@Override
-		public String toString() {
-			StringBuilder builder = new StringBuilder();
-			builder.append("SharedBufferPage(" + System.lineSeparator());
-			for (SharedBufferEntry<K, V> entry: entries.values()) {
-				builder.append(entry).append(System.lineSeparator());
-			}
-			builder.append(")");
-			return builder.toString();
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (!(obj instanceof SharedBufferPage)) {
-				return false;
-			}
-			SharedBufferPage<K, V> other = (SharedBufferPage<K, V>) obj;
-			return key.equals(other.getKey()) && entries.equals(other.entries);
-		}
-
-		@Override
-		public int hashCode() {
-			int result = 1;
-			result += 31 * result + key.hashCode();
-			result += 31 * result + entries.hashCode();
-			return result;
-		}
-	}
-
-	/**
-	 * Entry of a {@link SharedBufferPage}. The entry contains the value timestamp pair, a set of
-	 * edges to other shared buffer entries denoting a relation, a reference to the owning page and
-	 * a reference counter. The reference counter counts how many references are kept to this entry.
-	 *
-	 * @param <K> Type of the key
-	 * @param <V> Type of the value
-	 */
-	private static class SharedBufferEntry<K, V> {
-
-		private final ValueTimeWrapper<V> valueTime;
-		private final Set<SharedBufferEdge<K, V>> edges;
-		private final SharedBufferPage<K, V> page;
-
-		private int referenceCounter;
-		private int entryId;
-
-		SharedBufferEntry(
-				final ValueTimeWrapper<V> valueTime,
-				final SharedBufferPage<K, V> page) {
-			this(valueTime, null, page);
-		}
-
-		SharedBufferEntry(
-				final ValueTimeWrapper<V> valueTime,
-				final SharedBufferEdge<K, V> edge,
-				final SharedBufferPage<K, V> page) {
-
-			this.valueTime = valueTime;
-			edges = new HashSet<>();
-			if (edge != null) {
-				edges.add(edge);
-			}
-			referenceCounter = 0;
-			entryId = -1;
-			this.page = page;
-		}
-
-		public ValueTimeWrapper<V> getValueTime() {
-			return valueTime;
-		}
-
-		public Set<SharedBufferEdge<K, V>> getEdges() {
-			return edges;
-		}
-
-		public K getKey() {
-			return page.getKey();
-		}
-
-		public void addEdge(SharedBufferEdge<K, V> edge) {
-			edges.add(edge);
-		}
-
-		/**
-		 * Remove edges with the specified targets.
-		 */
-		private void removeEdges(final Set<SharedBufferEntry<K, V>> prunedEntries) {
-			Iterator<SharedBufferEdge<K, V>> it = edges.iterator();
-			while (it.hasNext()) {
-				SharedBufferEdge<K, V> edge = it.next();
-				if (prunedEntries.contains(edge.getTarget())) {
-					it.remove();
-				}
-			}
-		}
-
-		public void remove() {
-			page.remove(valueTime);
-		}
-
-		public void increaseReferenceCounter() {
-			referenceCounter++;
-		}
-
-		public void decreaseReferenceCounter() {
-			if (referenceCounter > 0) {
-				referenceCounter--;
-			}
-		}
-
-		public int getReferenceCounter() {
-			return referenceCounter;
-		}
-
-		@Override
-		public String toString() {
-			return "SharedBufferEntry(" + valueTime + ", [" + StringUtils.join(edges, ", ") + "], " + referenceCounter + ")";
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (!(obj instanceof SharedBufferEntry)) {
-				return false;
-			}
-
-			@SuppressWarnings("unchecked")
-			SharedBufferEntry<K, V> other = (SharedBufferEntry<K, V>) obj;
-
-			return valueTime.equals(other.valueTime) &&
-					getKey().equals(other.getKey()) &&
-					referenceCounter == other.referenceCounter &&
-					Objects.equals(edges, other.edges);
-		}
-
-		@Override
-		public int hashCode() {
-			int result = 1;
-			result += 31 * result + valueTime.hashCode();
-			result += 31 * result + getKey().hashCode();
-			result += 31 * result + referenceCounter;
-			result += 31 * result + edges.hashCode();
-			return result;
-		}
-	}
-
-	/**
-	 * Versioned edge between two shared buffer entries.
-	 *
-	 * @param <K> Type of the key
-	 * @param <V> Type of the value
-	 */
-	private static class SharedBufferEdge<K, V> {
-		private final SharedBufferEntry<K, V> target;
-		private final DeweyNumber version;
-
-		SharedBufferEdge(final SharedBufferEntry<K, V> target, final DeweyNumber version) {
-			this.target = target;
-			this.version = version;
-		}
-
-		public SharedBufferEntry<K, V> getTarget() {
-			return target;
-		}
-
-		public DeweyNumber getVersion() {
-			return version;
-		}
-
-		@Override
-		public String toString() {
-			return "SharedBufferEdge(" + target + ", " + version + ")";
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (!(obj instanceof SharedBufferEdge)) {
-				return false;
-			}
-
-			@SuppressWarnings("unchecked")
-			SharedBufferEdge<K, V> other = (SharedBufferEdge<K, V>) obj;
-			if (!version.equals(other.getVersion())) {
-				return false;
-			}
-
-			if (target == null && other.getTarget() == null) {
-				return true;
-			} else if (target != null && other.getTarget() != null) {
-				return target.getKey().equals(other.getTarget().getKey()) &&
-						target.getValueTime().equals(other.getTarget().getValueTime());
-			} else {
-				return false;
-			}
-		}
-
-		@Override
-		public int hashCode() {
-			if (target != null) {
-				return Objects.hash(target.getKey(), target.getValueTime(), version);
-			} else {
-				return version.hashCode();
-			}
-		}
+	public NodeId getNodeId(String prevState, long timestamp, int counter, V event) {
+		return mappingContext.get(Tuple2.of(NFAStateNameHandler.getOriginalNameFromInternal(prevState),
+			new ValueTimeWrapper<>(event, timestamp, counter)));
 	}
 
 	/**
@@ -706,8 +125,8 @@ public class SharedBuffer<K, V> {
 			ValueTimeWrapper<V> other = (ValueTimeWrapper<V>) obj;
 
 			return timestamp == other.getTimestamp()
-					&& Objects.equals(value, other.getValue())
-					&& counter == other.getCounter();
+				&& Objects.equals(value, other.getValue())
+				&& counter == other.getCounter();
 		}
 
 		@Override
@@ -715,17 +134,9 @@ public class SharedBuffer<K, V> {
 			return (int) (31 * (31 * (timestamp ^ timestamp >>> 32) + value.hashCode()) + counter);
 		}
 
-		public void serialize(
-				final TypeSerializer<V> valueSerializer,
-				final DataOutputView target) throws IOException {
-			valueSerializer.serialize(value, target);
-			target.writeLong(timestamp);
-			target.writeInt(counter);
-		}
-
 		public static <V> ValueTimeWrapper<V> deserialize(
-				final TypeSerializer<V> valueSerializer,
-				final DataInputView source) throws IOException {
+			final TypeSerializer<V> valueSerializer,
+			final DataInputView source) throws IOException {
 
 			final V value = valueSerializer.deserialize(source);
 			final long timestamp = source.readLong();
@@ -736,48 +147,6 @@ public class SharedBuffer<K, V> {
 	}
 
 	/**
-	 * Helper class to store the extraction state while extracting a sequence of values following
-	 * the versioned entry edges.
-	 *
-	 * @param <K> Type of the key
-	 * @param <V> Type of the value
-	 */
-	private static class ExtractionState<K, V> {
-
-		private final SharedBufferEntry<K, V> entry;
-		private final DeweyNumber version;
-		private final Stack<SharedBufferEntry<K, V>> path;
-
-		ExtractionState(
-				final SharedBufferEntry<K, V> entry,
-				final DeweyNumber version,
-				final Stack<SharedBufferEntry<K, V>> path) {
-			this.entry = entry;
-			this.version = version;
-			this.path = path;
-		}
-
-		public SharedBufferEntry<K, V> getEntry() {
-			return entry;
-		}
-
-		public DeweyNumber getVersion() {
-			return version;
-		}
-
-		public Stack<SharedBufferEntry<K, V>> getPath() {
-			return path;
-		}
-
-		@Override
-		public String toString() {
-			return "ExtractionState(" + entry + ", " + version + ", [" +  StringUtils.join(path, ", ") + "])";
-		}
-	}
-
-	//////////////				New Serialization				////////////////////
-
-	/**
 	 * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state.
 	 */
 	public static final class SharedBufferSerializerConfigSnapshot<K, V> extends CompositeTypeSerializerConfigSnapshot {
@@ -785,12 +154,13 @@ public class SharedBuffer<K, V> {
 		private static final int VERSION = 1;
 
 		/** This empty constructor is required for deserializing the configuration. */
-		public SharedBufferSerializerConfigSnapshot() {}
+		public SharedBufferSerializerConfigSnapshot() {
+		}
 
 		public SharedBufferSerializerConfigSnapshot(
-				final TypeSerializer<K> keySerializer,
-				final TypeSerializer<V> valueSerializer,
-				final TypeSerializer<DeweyNumber> versionSerializer) {
+			final TypeSerializer<K> keySerializer,
+			final TypeSerializer<V> valueSerializer,
+			final TypeSerializer<DeweyNumber> versionSerializer) {
 
 			super(keySerializer, valueSerializer, versionSerializer);
 		}
@@ -804,7 +174,7 @@ public class SharedBuffer<K, V> {
 	/**
 	 * A {@link TypeSerializer} for the {@link SharedBuffer}.
 	 */
-	public static class SharedBufferSerializer<K, V> extends TypeSerializer<SharedBuffer<K, V>> {
+	public static class SharedBufferSerializer<K, V> extends TypeSerializer<SharedBuffer<V>> {
 
 		private static final long serialVersionUID = -3254176794680331560L;
 
@@ -815,13 +185,13 @@ public class SharedBuffer<K, V> {
 		public SharedBufferSerializer(
 				final TypeSerializer<K> keySerializer,
 				final TypeSerializer<V> valueSerializer) {
-			this(keySerializer, valueSerializer, new DeweyNumber.DeweyNumberSerializer());
+			this(keySerializer, valueSerializer, DeweyNumber.DeweyNumberSerializer.INSTANCE);
 		}
 
 		public SharedBufferSerializer(
-				final TypeSerializer<K> keySerializer,
-				final TypeSerializer<V> valueSerializer,
-				final TypeSerializer<DeweyNumber> versionSerializer) {
+			final TypeSerializer<K> keySerializer,
+			final TypeSerializer<V> valueSerializer,
+			final TypeSerializer<DeweyNumber> versionSerializer) {
 
 			this.keySerializer = keySerializer;
 			this.valueSerializer = valueSerializer;
@@ -851,32 +221,18 @@ public class SharedBuffer<K, V> {
 		}
 
 		@Override
-		public SharedBuffer<K, V> createInstance() {
-			return new SharedBuffer<>();
+		public SharedBuffer<V> createInstance() {
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public SharedBuffer<K, V> copy(SharedBuffer<K, V> from) {
-			try {
-				ByteArrayOutputStream baos = new ByteArrayOutputStream();
-				serialize(from, new DataOutputViewStreamWrapper(baos));
-				baos.close();
-
-				byte[] data = baos.toByteArray();
-
-				ByteArrayInputStream bais = new ByteArrayInputStream(data);
-				SharedBuffer<K, V> copy = deserialize(new DataInputViewStreamWrapper(bais));
-				bais.close();
-
-				return copy;
-			} catch (IOException e) {
-				throw new RuntimeException("Could not copy SharredBuffer.", e);
-			}
+		public SharedBuffer<V> copy(SharedBuffer<V> from) {
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public SharedBuffer<K, V> copy(SharedBuffer<K, V> from, SharedBuffer<K, V> reuse) {
-			return copy(from);
+		public SharedBuffer<V> copy(SharedBuffer<V> from, SharedBuffer<V> reuse) {
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
@@ -885,83 +241,43 @@ public class SharedBuffer<K, V> {
 		}
 
 		@Override
-		public void serialize(SharedBuffer<K, V> record, DataOutputView target) throws IOException {
-			Map<K, SharedBufferPage<K, V>> pages = record.pages;
-
-			int totalEdges = 0;
-			int entryCounter = 0;
-
-			// number of pages
-			target.writeInt(pages.size());
-
-			for (SharedBufferPage<K, V> page: pages.values()) {
-
-				// key for the current page
-				keySerializer.serialize(page.getKey(), target);
-
-				target.writeInt(page.entries.size());
-				for (SharedBufferEntry<K, V> sharedBuffer: page.entries.values()) {
-
-					// assign id to the sharedBufferEntry for the future
-					// serialization of the previous relation
-					sharedBuffer.entryId = entryCounter++;
-
-					ValueTimeWrapper<V> valueTimeWrapper = sharedBuffer.getValueTime();
-					valueTimeWrapper.serialize(valueSerializer, target);
-					target.writeInt(sharedBuffer.getReferenceCounter());
-
-					totalEdges += sharedBuffer.getEdges().size();
-				}
-			}
-
-			// write the edges between the shared buffer entries
-			target.writeInt(totalEdges);
-
-			for (SharedBufferPage<K, V> page: pages.values()) {
-				for (SharedBufferEntry<K, V> sharedBuffer: page.entries.values()) {
-
-					// in order to serialize the previous relation we simply serialize
-					// the ids of the source and target SharedBufferEntry
-
-					int sourceId = sharedBuffer.entryId;
-
-					for (SharedBufferEdge<K, V> edge: sharedBuffer.edges) {
-						int targetId = -1;
-						if (edge.getTarget() != null) {
-							targetId = edge.getTarget().entryId;
-						}
-
-						target.writeInt(sourceId);
-						target.writeInt(targetId);
-						versionSerializer.serialize(edge.getVersion(), target);
-					}
-				}
-			}
+		public void serialize(SharedBuffer<V> record, DataOutputView target) throws IOException {
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public SharedBuffer<K, V> deserialize(DataInputView source) throws IOException {
-			List<SharedBufferEntry<K, V>> entryList = new ArrayList<>();
-			Map<K, SharedBufferPage<K, V>> pages = new HashMap<>();
-
+		public SharedBuffer<V> deserialize(DataInputView source) throws IOException {
+			List<Tuple2<NodeId, Lockable<SharedBufferNode>>> entries = new ArrayList<>();
+			Map<ValueTimeWrapper<V>, EventId> values = new HashMap<>();
+			Map<EventId, Lockable<V>> valuesWithIds = new HashMap<>();
+			Map<Tuple2<String, ValueTimeWrapper<V>>, NodeId> mappingContext = new HashMap<>();
+			Map<Long, Long> totalEventsPerTimestamp = new HashMap<>();
 			int totalPages = source.readInt();
 
 			for (int i = 0; i < totalPages; i++) {
-
 				// key of the page
-				K key = keySerializer.deserialize(source);
-				SharedBufferPage<K, V> page = new SharedBufferPage<>(key);
-				pages.put(key, page);
+				K stateName = keySerializer.deserialize(source);
 
 				int numberEntries = source.readInt();
 				for (int j = 0; j < numberEntries; j++) {
-					ValueTimeWrapper<V> valueTimeWrapper = ValueTimeWrapper.deserialize(valueSerializer, source);
-					SharedBufferEntry<K, V> sharedBufferEntry = new SharedBufferEntry<>(valueTimeWrapper, page);
-					sharedBufferEntry.referenceCounter = source.readInt();
+					ValueTimeWrapper<V> wrapper = ValueTimeWrapper.deserialize(valueSerializer, source);
+					EventId eventId = values.get(wrapper);
+					if (eventId == null) {
+						long id = totalEventsPerTimestamp.computeIfAbsent(wrapper.timestamp, k -> 0L);
+						eventId = new EventId(id, wrapper.timestamp);
+						values.put(wrapper, eventId);
+						valuesWithIds.put(eventId, new Lockable<>(wrapper.value, 1));
+						totalEventsPerTimestamp.computeIfPresent(wrapper.timestamp, (k, v) -> v + 1);
+					} else {
+						Lockable<V> eventWrapper = valuesWithIds.get(eventId);
+						eventWrapper.lock();
+					}
 
-					page.entries.put(valueTimeWrapper, sharedBufferEntry);
+					NodeId nodeId = new NodeId(eventId, (String) stateName);
+					int refCount = source.readInt();
 
-					entryList.add(sharedBufferEntry);
+					entries.add(Tuple2.of(nodeId, new Lockable<>(new SharedBufferNode(), refCount)));
+					mappingContext.put(Tuple2.of((String) stateName, wrapper), nodeId);
 				}
 			}
 
@@ -977,61 +293,25 @@ public class SharedBuffer<K, V> {
 
 				// We've already deserialized the shared buffer entry. Simply read its ID and
 				// retrieve the buffer entry from the list of entries
-				SharedBufferEntry<K, V> sourceEntry = entryList.get(sourceIdx);
-				SharedBufferEntry<K, V> targetEntry = targetIdx < 0 ? null : entryList.get(targetIdx);
-				sourceEntry.edges.add(new SharedBufferEdge<>(targetEntry, version));
+				Tuple2<NodeId, Lockable<SharedBufferNode>> sourceEntry = entries.get(sourceIdx);
+				Tuple2<NodeId, Lockable<SharedBufferNode>> targetEntry =
+					targetIdx < 0 ? Tuple2.of(null, null) : entries.get(targetIdx);
+				sourceEntry.f1.getElement().addEdge(new SharedBufferEdge(targetEntry.f0, version));
 			}
-			return new SharedBuffer<>(pages);
+
+			Map<NodeId, Lockable<SharedBufferNode>> entriesMap = entries.stream().collect(Collectors.toMap(e -> e.f0, e -> e.f1));
+
+			return new SharedBuffer<>(valuesWithIds, entriesMap, mappingContext);
 		}
 
 		@Override
-		public SharedBuffer<K, V> deserialize(SharedBuffer<K, V> reuse, DataInputView source) throws IOException {
+		public SharedBuffer<V> deserialize(SharedBuffer<V> reuse, DataInputView source) throws IOException {
 			return deserialize(source);
 		}
 
 		@Override
 		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			int numberPages = source.readInt();
-			target.writeInt(numberPages);
-
-			for (int i = 0; i < numberPages; i++) {
-				// key of the page
-				@SuppressWarnings("unchecked")
-				K key = keySerializer.deserialize(source);
-				keySerializer.serialize(key, target);
-
-				int numberEntries = source.readInt();
-
-				for (int j = 0; j < numberEntries; j++) {
-					// restore the SharedBufferEntries for the given page
-					V value = valueSerializer.deserialize(source);
-					valueSerializer.serialize(value, target);
-
-					long timestamp = source.readLong();
-					target.writeLong(timestamp);
-
-					int counter = source.readInt();
-					target.writeInt(counter);
-
-					int referenceCounter = source.readInt();
-					target.writeInt(referenceCounter);
-				}
-			}
-
-			// read the edges of the shared buffer entries
-			int numberEdges = source.readInt();
-			target.writeInt(numberEdges);
-
-			for (int j = 0; j < numberEdges; j++) {
-				int sourceIndex = source.readInt();
-				int targetIndex = source.readInt();
-
-				target.writeInt(sourceIndex);
-				target.writeInt(targetIndex);
-
-				DeweyNumber version = versionSerializer.deserialize(source);
-				versionSerializer.serialize(version, target);
-			}
+			throw new UnsupportedOperationException();
 		}
 
 		@Override
@@ -1046,7 +326,7 @@ public class SharedBuffer<K, V> {
 
 			SharedBufferSerializer other = (SharedBufferSerializer) obj;
 			return
-					Objects.equals(keySerializer, other.getKeySerializer()) &&
+				Objects.equals(keySerializer, other.getKeySerializer()) &&
 					Objects.equals(valueSerializer, other.getValueSerializer()) &&
 					Objects.equals(versionSerializer, other.getVersionSerializer());
 		}
@@ -1064,47 +344,48 @@ public class SharedBuffer<K, V> {
 		@Override
 		public TypeSerializerConfigSnapshot snapshotConfiguration() {
 			return new SharedBufferSerializerConfigSnapshot<>(
-					keySerializer,
-					valueSerializer,
-					versionSerializer);
+				keySerializer,
+				valueSerializer,
+				versionSerializer);
 		}
 
 		@Override
-		public CompatibilityResult<SharedBuffer<K, V>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public CompatibilityResult<SharedBuffer<V>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
 			if (configSnapshot instanceof SharedBufferSerializerConfigSnapshot) {
 				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializerConfigSnapshots =
-						((SharedBufferSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
+					((SharedBufferSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
 
 				CompatibilityResult<K> keyCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						serializerConfigSnapshots.get(0).f0,
-						UnloadableDummyTypeSerializer.class,
-						serializerConfigSnapshots.get(0).f1,
-						keySerializer);
+					serializerConfigSnapshots.get(0).f0,
+					UnloadableDummyTypeSerializer.class,
+					serializerConfigSnapshots.get(0).f1,
+					keySerializer);
 
 				CompatibilityResult<V> valueCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						serializerConfigSnapshots.get(1).f0,
-						UnloadableDummyTypeSerializer.class,
-						serializerConfigSnapshots.get(1).f1,
-						valueSerializer);
+					serializerConfigSnapshots.get(1).f0,
+					UnloadableDummyTypeSerializer.class,
+					serializerConfigSnapshots.get(1).f1,
+					valueSerializer);
 
 				CompatibilityResult<DeweyNumber> versionCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						serializerConfigSnapshots.get(2).f0,
-						UnloadableDummyTypeSerializer.class,
-						serializerConfigSnapshots.get(2).f1,
-						versionSerializer);
+					serializerConfigSnapshots.get(2).f0,
+					UnloadableDummyTypeSerializer.class,
+					serializerConfigSnapshots.get(2).f1,
+					versionSerializer);
 
-				if (!keyCompatResult.isRequiresMigration() && !valueCompatResult.isRequiresMigration() && !versionCompatResult.isRequiresMigration()) {
+				if (!keyCompatResult.isRequiresMigration() && !valueCompatResult.isRequiresMigration() &&
+					!versionCompatResult.isRequiresMigration()) {
 					return CompatibilityResult.compatible();
 				} else {
 					if (keyCompatResult.getConvertDeserializer() != null
-							&& valueCompatResult.getConvertDeserializer() != null
-							&& versionCompatResult.getConvertDeserializer() != null) {
+						&& valueCompatResult.getConvertDeserializer() != null
+						&& versionCompatResult.getConvertDeserializer() != null) {
 						return CompatibilityResult.requiresMigration(
-								new SharedBufferSerializer<>(
-										new TypeDeserializerAdapter<>(keyCompatResult.getConvertDeserializer()),
-										new TypeDeserializerAdapter<>(valueCompatResult.getConvertDeserializer()),
-										new TypeDeserializerAdapter<>(versionCompatResult.getConvertDeserializer())
-								));
+							new SharedBufferSerializer<>(
+								new TypeDeserializerAdapter<>(keyCompatResult.getConvertDeserializer()),
+								new TypeDeserializerAdapter<>(valueCompatResult.getConvertDeserializer()),
+								new TypeDeserializerAdapter<>(versionCompatResult.getConvertDeserializer())
+							));
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index 5b9522b..dbb654c 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.cep.nfa.compiler;
 
-import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.AfterMatchSkipStrategy;
 import org.apache.flink.cep.nfa.NFA;
@@ -53,29 +52,10 @@ public class NFACompiler {
 	protected static final String ENDING_STATE_NAME = "$endState$";
 
 	/**
-	 * Compiles the given pattern into a {@link NFA}.
-	 *
-	 * @param pattern Definition of sequence pattern
-	 * @param inputTypeSerializer Serializer for the input type
-	 * @param timeoutHandling True if the NFA shall return timed out event patterns
-	 * @param <T> Type of the input events
-	 * @return Non-deterministic finite automaton representing the given pattern
-	 */
-	public static <T> NFA<T> compile(
-		Pattern<T, ?> pattern,
-		TypeSerializer<T> inputTypeSerializer,
-		boolean timeoutHandling) {
-		NFAFactory<T> factory = compileFactory(pattern, inputTypeSerializer, timeoutHandling);
-
-		return factory.createNFA();
-	}
-
-	/**
 	 * Compiles the given pattern into a {@link NFAFactory}. The NFA factory can be used to create
 	 * multiple NFAs.
 	 *
 	 * @param pattern Definition of sequence pattern
-	 * @param inputTypeSerializer Serializer for the input type
 	 * @param timeoutHandling True if the NFA shall return timed out event patterns
 	 * @param <T> Type of the input events
 	 * @return Factory for NFAs corresponding to the given pattern
@@ -83,15 +63,14 @@ public class NFACompiler {
 	@SuppressWarnings("unchecked")
 	public static <T> NFAFactory<T> compileFactory(
 		final Pattern<T, ?> pattern,
-		final TypeSerializer<T> inputTypeSerializer,
 		boolean timeoutHandling) {
 		if (pattern == null) {
 			// return a factory for empty NFAs
-			return new NFAFactoryImpl<>(inputTypeSerializer, 0, Collections.<State<T>>emptyList(), timeoutHandling);
+			return new NFAFactoryImpl<>(0, Collections.<State<T>>emptyList(), timeoutHandling);
 		} else {
 			final NFAFactoryCompiler<T> nfaFactoryCompiler = new NFAFactoryCompiler<>(pattern);
 			nfaFactoryCompiler.compileFactory();
-			return new NFAFactoryImpl<>(inputTypeSerializer, nfaFactoryCompiler.getWindowTime(), nfaFactoryCompiler.getStates(), timeoutHandling);
+			return new NFAFactoryImpl<>(nfaFactoryCompiler.getWindowTime(), nfaFactoryCompiler.getStates(), timeoutHandling);
 		}
 	}
 
@@ -900,18 +879,15 @@ public class NFACompiler {
 
 		private static final long serialVersionUID = 8939783698296714379L;
 
-		private final TypeSerializer<T> inputTypeSerializer;
 		private final long windowTime;
 		private final Collection<State<T>> states;
 		private final boolean timeoutHandling;
 
 		private NFAFactoryImpl(
-			TypeSerializer<T> inputTypeSerializer,
-			long windowTime,
-			Collection<State<T>> states,
-			boolean timeoutHandling) {
+				long windowTime,
+				Collection<State<T>> states,
+				boolean timeoutHandling) {
 
-			this.inputTypeSerializer = inputTypeSerializer;
 			this.windowTime = windowTime;
 			this.states = states;
 			this.timeoutHandling = timeoutHandling;
@@ -919,8 +895,7 @@ public class NFACompiler {
 
 		@Override
 		public NFA<T> createNFA() {
-			return new NFA<>(
-				inputTypeSerializer.duplicate(), windowTime, timeoutHandling, states);
+			return new NFA<>(states, windowTime, timeoutHandling);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
new file mode 100644
index 0000000..9b99ea1
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/EventId.java
@@ -0,0 +1,141 @@
+/*
+ * 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.nfa.sharedbuffer;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Composite key for events in {@link SharedBuffer}.
+ */
+public class EventId {
+	private final long id;
+	private final long timestamp;
+
+	public EventId(long id, long timestamp) {
+		this.id = id;
+		this.timestamp = timestamp;
+	}
+
+	public long getId() {
+		return id;
+	}
+
+	public long getTimestamp() {
+		return timestamp;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		EventId eventId = (EventId) o;
+		return id == eventId.id &&
+			timestamp == eventId.timestamp;
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(id, timestamp);
+	}
+
+	@Override
+	public String toString() {
+		return "EventId{" +
+			"id=" + id +
+			", timestamp=" + timestamp +
+			'}';
+	}
+
+	/** {@link TypeSerializer} for {@link EventId}. */
+	public static class EventIdSerializer extends TypeSerializerSingleton<EventId> {
+
+		private static final long serialVersionUID = -5685733582601394497L;
+
+		private EventIdSerializer() {
+		}
+
+		public static final EventIdSerializer INSTANCE = new EventIdSerializer();
+
+		@Override
+		public boolean isImmutableType() {
+			return true;
+		}
+
+		@Override
+		public EventId createInstance() {
+			return null;
+		}
+
+		@Override
+		public EventId copy(EventId from) {
+			return new EventId(from.id, from.timestamp);
+		}
+
+		@Override
+		public EventId copy(EventId from, EventId reuse) {
+			return copy(from);
+		}
+
+		@Override
+		public int getLength() {
+			return 2 * LongSerializer.INSTANCE.getLength();
+		}
+
+		@Override
+		public void serialize(EventId record, DataOutputView target) throws IOException {
+			LongSerializer.INSTANCE.serialize(record.id, target);
+			LongSerializer.INSTANCE.serialize(record.timestamp, target);
+		}
+
+		@Override
+		public EventId deserialize(DataInputView source) throws IOException {
+			Long id = LongSerializer.INSTANCE.deserialize(source);
+			Long timestamp = LongSerializer.INSTANCE.deserialize(source);
+
+			return new EventId(id, timestamp);
+		}
+
+		@Override
+		public EventId deserialize(EventId reuse, DataInputView source) throws IOException {
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			LongSerializer.INSTANCE.copy(source, target);
+			LongSerializer.INSTANCE.copy(source, target);
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj.getClass().equals(EventIdSerializer.class);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
new file mode 100644
index 0000000..ca1ecae
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
@@ -0,0 +1,204 @@
+/*
+ * 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.nfa.sharedbuffer;
+
+import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Implements locking logic for incoming event and
+ * {@link SharedBufferNode} using a lock reference counter.
+ */
+public final class Lockable<T> {
+
+	private int refCounter;
+
+	private final T element;
+
+	public Lockable(T element, int refCounter) {
+		this.refCounter = refCounter;
+		this.element = element;
+	}
+
+	public void lock() {
+		refCounter += 1;
+	}
+
+	/**
+	 * Releases lock on this object. If no more locks are acquired on it, this method will return true.
+	 *
+	 * @return true if no more locks are acquired
+	 */
+	boolean release() {
+		if (refCounter <= 0) {
+			return true;
+		}
+
+		refCounter -= 1;
+		return refCounter == 0;
+	}
+
+	int getRefCounter() {
+		return refCounter;
+	}
+
+	public T getElement() {
+		return element;
+	}
+
+	@Override
+	public String toString() {
+		return "Lock{" +
+			"refCounter=" + refCounter +
+			'}';
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		Lockable<?> lockable = (Lockable<?>) o;
+		return refCounter == lockable.refCounter &&
+			Objects.equals(element, lockable.element);
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(refCounter, element);
+	}
+
+	/** Serializer for {@link Lockable}. */
+	public static class LockableTypeSerializer<E> extends TypeSerializer<Lockable<E>> {
+		private static final long serialVersionUID = 3298801058463337340L;
+		private final TypeSerializer<E> elementSerializer;
+
+		LockableTypeSerializer(TypeSerializer<E> elementSerializer) {
+			this.elementSerializer = elementSerializer;
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return false;
+		}
+
+		@Override
+		public TypeSerializer<Lockable<E>> duplicate() {
+			return new LockableTypeSerializer<>(elementSerializer);
+		}
+
+		@Override
+		public Lockable<E> createInstance() {
+			return null;
+		}
+
+		@Override
+		public Lockable<E> copy(Lockable<E> from) {
+			return new Lockable<E>(elementSerializer.copy(from.element), from.refCounter);
+		}
+
+		@Override
+		public Lockable<E> copy(
+			Lockable<E> from, Lockable<E> reuse) {
+			return copy(from);
+		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(Lockable<E> record, DataOutputView target) throws IOException {
+			IntSerializer.INSTANCE.serialize(record.refCounter, target);
+			elementSerializer.serialize(record.element, target);
+		}
+
+		@Override
+		public Lockable<E> deserialize(DataInputView source) throws IOException {
+			Integer refCount = IntSerializer.INSTANCE.deserialize(source);
+			E record = elementSerializer.deserialize(source);
+			return new Lockable<>(record, refCount);
+		}
+
+		@Override
+		public Lockable<E> deserialize(
+			Lockable<E> reuse, DataInputView source) throws IOException {
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			IntSerializer.INSTANCE.copy(source, target); // refCounter
+
+			E element = elementSerializer.deserialize(source);
+			elementSerializer.serialize(element, target);
+		}
+
+		@Override
+		public boolean equals(Object o) {
+			if (this == o) {
+				return true;
+			}
+			if (o == null || getClass() != o.getClass()) {
+				return false;
+			}
+			LockableTypeSerializer<?> that = (LockableTypeSerializer<?>) o;
+			return Objects.equals(elementSerializer, that.elementSerializer);
+		}
+
+		@Override
+		public int hashCode() {
+			return Objects.hash(elementSerializer);
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj.getClass().equals(LockableTypeSerializer.class);
+		}
+
+		@Override
+		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+			return elementSerializer.snapshotConfiguration();
+		}
+
+		@Override
+		public CompatibilityResult<Lockable<E>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+			CompatibilityResult<E> inputComaptibilityResult = elementSerializer.ensureCompatibility(configSnapshot);
+			if (inputComaptibilityResult.isRequiresMigration()) {
+				return CompatibilityResult.requiresMigration(new LockableTypeSerializer<>(
+					new TypeDeserializerAdapter<>(inputComaptibilityResult.getConvertDeserializer()))
+				);
+			} else {
+				return CompatibilityResult.compatible();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/NodeId.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/NodeId.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/NodeId.java
new file mode 100644
index 0000000..3a13184
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/NodeId.java
@@ -0,0 +1,155 @@
+/*
+ * 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.nfa.sharedbuffer;
+
+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.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * Unique identifier for {@link SharedBufferNode}.
+ */
+public class NodeId {
+
+	private final String pageName;
+	private final EventId eventId;
+
+	public NodeId(EventId eventId, String pageName) {
+		this.eventId = eventId;
+		this.pageName = pageName;
+	}
+
+	public EventId getEventId() {
+		return eventId;
+	}
+
+	public String getPageName() {
+		return pageName;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		NodeId nodeId = (NodeId) o;
+		return Objects.equals(eventId, nodeId.eventId) &&
+			Objects.equals(pageName, nodeId.pageName);
+	}
+
+	@Override
+	public int hashCode() {
+		return Objects.hash(eventId, pageName);
+	}
+
+	@Override
+	public String toString() {
+		return "NodeId{" +
+			"eventId=" + eventId +
+			", pageName='" + pageName + '\'' +
+			'}';
+	}
+
+	/** Serializer for {@link NodeId}. */
+	public static class NodeIdSerializer extends TypeSerializerSingleton<NodeId> {
+
+		private static final long serialVersionUID = 9209498028181378582L;
+
+		public static final NodeIdSerializer INSTANCE = new NodeIdSerializer();
+
+		private NodeIdSerializer() {
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return true;
+		}
+
+		@Override
+		public NodeId createInstance() {
+			return null;
+		}
+
+		@Override
+		public NodeId copy(NodeId from) {
+			return new NodeId(from.eventId, from.pageName);
+		}
+
+		@Override
+		public NodeId copy(NodeId from, NodeId reuse) {
+			return copy(from);
+		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(NodeId record, DataOutputView target) throws IOException {
+			if (record != null) {
+				target.writeByte(1);
+				EventId.EventIdSerializer.INSTANCE.serialize(record.eventId, target);
+				StringSerializer.INSTANCE.serialize(record.pageName, target);
+			} else {
+				target.writeByte(0);
+			}
+		}
+
+		@Override
+		public NodeId deserialize(DataInputView source) throws IOException {
+			byte b = source.readByte();
+			if (b == 0) {
+				return null;
+			}
+
+			EventId eventId = EventId.EventIdSerializer.INSTANCE.deserialize(source);
+			String pageName = StringSerializer.INSTANCE.deserialize(source);
+			return new NodeId(eventId, pageName);
+		}
+
+		@Override
+		public NodeId deserialize(NodeId reuse, DataInputView source) throws IOException {
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			target.writeByte(source.readByte());
+
+			LongSerializer.INSTANCE.copy(source, target); // eventId
+			LongSerializer.INSTANCE.copy(source, target); // timestamp
+			StringSerializer.INSTANCE.copy(source, target); // pageName
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj.getClass().equals(NodeIdSerializer.class);
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
new file mode 100644
index 0000000..50d997c
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBuffer.java
@@ -0,0 +1,395 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOVICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Vhe 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.nfa.sharedbuffer;
+
+import org.apache.flink.api.common.state.KeyedStateStore;
+import org.apache.flink.api.common.state.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.nfa.DeweyNumber;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;
+
+import org.apache.commons.lang3.StringUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.cep.nfa.compiler.NFAStateNameHandler.getOriginalNameFromInternal;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A shared buffer implementation which stores values under according state. Additionally, the values can be
+ * versioned such that it is possible to retrieve their predecessor element in the buffer.
+ *
+ * <p>The idea of the implementation is to have a buffer for incoming events with unique ids assigned to them. This way
+ * we do not need to deserialize events during processing and we store only one copy of the event.
+ *
+ * <p>The entries in {@link SharedBuffer} are {@link SharedBufferNode}. The shared buffer node allows to store
+ * relations between different entries. A dewey versioning scheme allows to discriminate between
+ * different relations (e.g. preceding element).
+ *
+ * <p>The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams".
+ *
+ * @param <V> Type of the values
+ * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">
+ * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
+ */
+public class SharedBuffer<V> {
+
+	private static final String entriesStateName = "sharedBuffer-entries";
+	private static final String eventsStateName = "sharedBuffer-events";
+	private static final String eventsCountStateName = "sharedBuffer-events-count";
+
+	/** The buffer holding the unique events seen so far. */
+	private MapState<EventId, Lockable<V>> eventsBuffer;
+
+	/** The number of events seen so far in the stream per timestamp. */
+	private MapState<Long, Long> eventsCount;
+	private MapState<NodeId, Lockable<SharedBufferNode>> pages;
+
+	public SharedBuffer(KeyedStateStore stateStore, TypeSerializer<V> valueSerializer) {
+		this.eventsBuffer = stateStore.getMapState(
+			new MapStateDescriptor<>(
+				eventsStateName,
+				EventId.EventIdSerializer.INSTANCE,
+				new Lockable.LockableTypeSerializer<>(valueSerializer)));
+
+		this.pages = stateStore.getMapState(
+			new MapStateDescriptor<>(
+				entriesStateName,
+				NodeId.NodeIdSerializer.INSTANCE,
+				new Lockable.LockableTypeSerializer<>(new SharedBufferNode.SharedBufferNodeSerializer())));
+
+		this.eventsCount = stateStore.getMapState(
+			new MapStateDescriptor<>(
+				eventsCountStateName,
+				LongSerializer.INSTANCE,
+				LongSerializer.INSTANCE));
+	}
+
+	/**
+	 * Adds another unique event to the shared buffer and assigns a unique id for it. It automatically creates a
+	 * lock on this event, so it won't be removed during processing of that event. Therefore the lock should be removed
+	 * after processing all {@link org.apache.flink.cep.nfa.ComputationState}s
+	 *
+	 * <p><b>NOTE:</b>Should be called only once for each unique event!
+	 *
+	 * @param value event to be registered
+	 * @return unique id of that event that should be used when putting entries to the buffer.
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public EventId registerEvent(V value, long timestamp) throws Exception {
+		Long id = eventsCount.get(timestamp);
+		if (id == null) {
+			id = 0L;
+		}
+
+		EventId eventId = new EventId(id, timestamp);
+		eventsBuffer.put(eventId, new Lockable<>(value, 1));
+		eventsCount.put(timestamp, id + 1L);
+		return eventId;
+	}
+
+	/**
+	 * Initializes underlying state with given map of events and entries. Should be used only in case of migration from
+	 * old state.
+	 *
+	 * @param events  map of events with assigned unique ids
+	 * @param entries map of SharedBufferNodes
+	 * @throws Exception Thrown if the system cannot access the state.
+	 * @deprecated Only for state migration!
+	 */
+	@Deprecated
+	public void init(
+			Map<EventId, Lockable<V>> events,
+			Map<NodeId, Lockable<SharedBufferNode>> entries) throws Exception {
+		eventsBuffer.putAll(events);
+		pages.putAll(entries);
+
+		Map<Long, Long> maxIds = events.keySet().stream().collect(Collectors.toMap(
+			EventId::getTimestamp,
+			EventId::getId,
+			Math::max
+		));
+		eventsCount.putAll(maxIds);
+	}
+
+	/**
+	 * Stores given value (value + timestamp) under the given state. It assigns no preceding element
+	 * relation to the entry.
+	 *
+	 * @param stateName name of the state that the event should be assigned to
+	 * @param eventId   unique id of event assigned by this SharedBuffer
+	 * @param version   Version of the previous relation
+	 * @return assigned id of this entry
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public NodeId put(
+			final String stateName,
+			final EventId eventId,
+			final DeweyNumber version) throws Exception {
+
+		return put(stateName, eventId, null, version);
+	}
+
+	/**
+	 * Stores given value (value + timestamp) under the given state. It assigns a preceding element
+	 * relation to the previous entry.
+	 *
+	 * @param stateName     name of the state that the event should be assigned to
+	 * @param eventId       unique id of event assigned by this SharedBuffer
+	 * @param previousNodeId id of previous entry
+	 * @param version       Version of the previous relation
+	 * @return assigned id of this element
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public NodeId put(
+			final String stateName,
+			final EventId eventId,
+			@Nullable final NodeId previousNodeId,
+			final DeweyNumber version) throws Exception {
+
+		if (previousNodeId != null) {
+			lockNode(previousNodeId);
+		}
+
+		NodeId currentNodeId = new NodeId(eventId, getOriginalNameFromInternal(stateName));
+		Lockable<SharedBufferNode> currentNode = pages.get(currentNodeId);
+		if (currentNode == null) {
+			currentNode = new Lockable<>(new SharedBufferNode(), 0);
+			lockEvent(eventId);
+		}
+
+		currentNode.getElement().addEdge(new SharedBufferEdge(
+			previousNodeId,
+			version));
+		pages.put(currentNodeId, currentNode);
+
+		return currentNodeId;
+	}
+
+	/**
+	 * Checks if there is no elements in the buffer.
+	 *
+	 * @return true if there is no elements in the buffer
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public boolean isEmpty() throws Exception {
+		return Iterables.isEmpty(eventsBuffer.keys());
+	}
+
+	/**
+	 * Returns all elements from the previous relation starting at the given entry.
+	 *
+	 * @param nodeId  id of the starting entry
+	 * @param version Version of the previous relation which shall be extracted
+	 * @return Collection of previous relations starting with the given value
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public List<Map<String, List<V>>> extractPatterns(
+			final NodeId nodeId,
+			final DeweyNumber version) throws Exception {
+
+		List<Map<String, List<V>>> result = new ArrayList<>();
+
+		// stack to remember the current extraction states
+		Stack<ExtractionState> extractionStates = new Stack<>();
+
+		// get the starting shared buffer entry for the previous relation
+		Lockable<SharedBufferNode> entryLock = pages.get(nodeId);
+
+		if (entryLock != null) {
+			SharedBufferNode entry = entryLock.getElement();
+			extractionStates.add(new ExtractionState(Tuple2.of(nodeId, entry), version, new Stack<>()));
+
+			// use a depth first search to reconstruct the previous relations
+			while (!extractionStates.isEmpty()) {
+				final ExtractionState extractionState = extractionStates.pop();
+				// current path of the depth first search
+				final Stack<Tuple2<NodeId, SharedBufferNode>> currentPath = extractionState.getPath();
+				final Tuple2<NodeId, SharedBufferNode> currentEntry = extractionState.getEntry();
+
+				// termination criterion
+				if (currentEntry == null) {
+					final Map<String, List<V>> completePath = new LinkedHashMap<>();
+
+					while (!currentPath.isEmpty()) {
+						final NodeId currentPathEntry = currentPath.pop().f0;
+
+						String page = currentPathEntry.getPageName();
+						List<V> values = completePath
+							.computeIfAbsent(page, k -> new ArrayList<>());
+						values.add(eventsBuffer.get(currentPathEntry.getEventId()).getElement());
+					}
+					result.add(completePath);
+				} else {
+
+					// append state to the path
+					currentPath.push(currentEntry);
+
+					boolean firstMatch = true;
+					for (SharedBufferEdge edge : currentEntry.f1.getEdges()) {
+						// we can only proceed if the current version is compatible to the version
+						// of this previous relation
+						final DeweyNumber currentVersion = extractionState.getVersion();
+						if (currentVersion.isCompatibleWith(edge.getDeweyNumber())) {
+							final NodeId target = edge.getTarget();
+							Stack<Tuple2<NodeId, SharedBufferNode>> newPath;
+
+							if (firstMatch) {
+								// for the first match we don't have to copy the current path
+								newPath = currentPath;
+								firstMatch = false;
+							} else {
+								newPath = new Stack<>();
+								newPath.addAll(currentPath);
+							}
+
+							extractionStates.push(new ExtractionState(
+								target != null ? Tuple2.of(target, pages.get(target).getElement()) : null,
+								edge.getDeweyNumber(),
+								newPath));
+						}
+					}
+				}
+
+			}
+		}
+		return result;
+	}
+
+	/**
+	 * Increases the reference counter for the given entry so that it is not
+	 * accidentally removed.
+	 *
+	 * @param node id of the entry
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public void lockNode(final NodeId node) throws Exception {
+		Lockable<SharedBufferNode> sharedBufferNode = pages.get(node);
+		if (sharedBufferNode != null) {
+			sharedBufferNode.lock();
+			pages.put(node, sharedBufferNode);
+		}
+	}
+
+	/**
+	 * Decreases the reference counter for the given entry so that it can be
+	 * removed once the reference counter reaches 0.
+	 *
+	 * @param node id of the entry
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public void releaseNode(final NodeId node) throws Exception {
+		Lockable<SharedBufferNode> sharedBufferNode = pages.get(node);
+		if (sharedBufferNode != null) {
+			if (sharedBufferNode.release()) {
+				removeNode(node, sharedBufferNode.getElement());
+			} else {
+				pages.put(node, sharedBufferNode);
+			}
+		}
+	}
+
+	private void removeNode(NodeId node, SharedBufferNode sharedBufferNode) throws Exception {
+		pages.remove(node);
+		EventId eventId = node.getEventId();
+		releaseEvent(eventId);
+
+		for (SharedBufferEdge sharedBufferEdge : sharedBufferNode.getEdges()) {
+			releaseNode(sharedBufferEdge.getTarget());
+		}
+	}
+
+	private void lockEvent(EventId eventId) throws Exception {
+		Lockable<V> eventWrapper = eventsBuffer.get(eventId);
+		checkState(
+			eventWrapper != null,
+			"Referring to non existent event with id %s",
+			eventId);
+		eventWrapper.lock();
+		eventsBuffer.put(eventId, eventWrapper);
+	}
+
+	/**
+	 * Decreases the reference counter for the given event so that it can be
+	 * removed once the reference counter reaches 0.
+	 *
+	 * @param eventId id of the event
+	 * @throws Exception Thrown if the system cannot access the state.
+	 */
+	public void releaseEvent(EventId eventId) throws Exception {
+		Lockable<V> eventWrapper = eventsBuffer.get(eventId);
+		if (eventWrapper != null) {
+			if (eventWrapper.release()) {
+				eventsBuffer.remove(eventId);
+			} else {
+				eventsBuffer.put(eventId, eventWrapper);
+			}
+		}
+	}
+
+	/**
+	 * Helper class to store the extraction state while extracting a sequence of values following
+	 * the versioned entry edges.
+	 */
+	private static class ExtractionState {
+
+		private final Tuple2<NodeId, SharedBufferNode> entry;
+		private final DeweyNumber version;
+		private final Stack<Tuple2<NodeId, SharedBufferNode>> path;
+
+		ExtractionState(
+				final Tuple2<NodeId, SharedBufferNode> entry,
+				final DeweyNumber version,
+				final Stack<Tuple2<NodeId, SharedBufferNode>> path) {
+			this.entry = entry;
+			this.version = version;
+			this.path = path;
+		}
+
+		public Tuple2<NodeId, SharedBufferNode> getEntry() {
+			return entry;
+		}
+
+		public Stack<Tuple2<NodeId, SharedBufferNode>> getPath() {
+			return path;
+		}
+
+		public DeweyNumber getVersion() {
+			return version;
+		}
+
+		@Override
+		public String toString() {
+			return "ExtractionState(" + entry + ", " + version + ", [" +
+				StringUtils.join(path, ", ") + "])";
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferEdge.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..c8d9021
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferEdge.java
@@ -0,0 +1,126 @@
+/*
+ * 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.nfa.sharedbuffer;
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.cep.nfa.DeweyNumber;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Versioned edge in {@link SharedBuffer} that allows retrieving predecessors.
+ */
+public class SharedBufferEdge {
+
+	private final NodeId target;
+	private final DeweyNumber deweyNumber;
+
+	/**
+	 * Creates versioned (with {@link DeweyNumber}) edge that points to the target entry.
+	 *
+	 * @param target      id of target entry
+	 * @param deweyNumber version for this edge
+	 */
+	public SharedBufferEdge(NodeId target, DeweyNumber deweyNumber) {
+		this.target = target;
+		this.deweyNumber = deweyNumber;
+	}
+
+	NodeId getTarget() {
+		return target;
+	}
+
+	DeweyNumber getDeweyNumber() {
+		return deweyNumber;
+	}
+
+	@Override
+	public String toString() {
+		return "SharedBufferEdge{" +
+			"target=" + target +
+			", deweyNumber=" + deweyNumber +
+			'}';
+	}
+
+	/** Serializer for {@link SharedBufferEdge}. */
+	public static class SharedBufferEdgeSerializer extends TypeSerializerSingleton<SharedBufferEdge> {
+
+		private static final long serialVersionUID = -5122474955050663979L;
+
+		static final SharedBufferEdgeSerializer INSTANCE = new SharedBufferEdgeSerializer();
+
+		private SharedBufferEdgeSerializer() {}
+
+		@Override
+		public boolean isImmutableType() {
+			return true;
+		}
+
+		@Override
+		public SharedBufferEdge createInstance() {
+			return null;
+		}
+
+		@Override
+		public SharedBufferEdge copy(SharedBufferEdge from) {
+			return new SharedBufferEdge(from.target, from.deweyNumber);
+		}
+
+		@Override
+		public SharedBufferEdge copy(SharedBufferEdge from, SharedBufferEdge reuse) {
+			return copy(from);
+		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(SharedBufferEdge record, DataOutputView target) throws IOException {
+			NodeId.NodeIdSerializer.INSTANCE.serialize(record.target, target);
+			DeweyNumber.DeweyNumberSerializer.INSTANCE.serialize(record.deweyNumber, target);
+		}
+
+		@Override
+		public SharedBufferEdge deserialize(DataInputView source) throws IOException {
+			NodeId target = NodeId.NodeIdSerializer.INSTANCE.deserialize(source);
+			DeweyNumber deweyNumber = DeweyNumber.DeweyNumberSerializer.INSTANCE.deserialize(source);
+			return new SharedBufferEdge(target, deweyNumber);
+		}
+
+		@Override
+		public SharedBufferEdge deserialize(SharedBufferEdge reuse, DataInputView source) throws IOException {
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			NodeId.NodeIdSerializer.INSTANCE.copy(source, target);
+			DeweyNumber.DeweyNumberSerializer.INSTANCE.copy(source, target);
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj.getClass().equals(SharedBufferEdgeSerializer.class);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferNode.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b613625
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferNode.java
@@ -0,0 +1,120 @@
+/*
+ * 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.nfa.sharedbuffer;
+
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferEdge.SharedBufferEdgeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An entry in {@link SharedBuffer} that allows to store relations between different entries.
+ */
+public class SharedBufferNode {
+
+	private final List<SharedBufferEdge> edges;
+
+	public SharedBufferNode() {
+		edges = new ArrayList<>();
+	}
+
+	private SharedBufferNode(List<SharedBufferEdge> edges) {
+		this.edges = edges;
+	}
+
+	public List<SharedBufferEdge> getEdges() {
+		return edges;
+	}
+
+	public void addEdge(SharedBufferEdge edge) {
+		edges.add(edge);
+	}
+
+	@Override
+	public String toString() {
+		return "SharedBufferNode{" +
+			"edges=" + edges +
+			'}';
+	}
+
+	/** Serializer for {@link SharedBufferNode}. */
+	public static class SharedBufferNodeSerializer extends TypeSerializerSingleton<SharedBufferNode> {
+
+		private static final long serialVersionUID = -6687780732295439832L;
+
+		private final ListSerializer<SharedBufferEdge> edgesSerializer =
+			new ListSerializer<>(SharedBufferEdgeSerializer.INSTANCE);
+
+		@Override
+		public boolean isImmutableType() {
+			return false;
+		}
+
+		@Override
+		public SharedBufferNode createInstance() {
+			return new SharedBufferNode(new ArrayList<>());
+		}
+
+		@Override
+		public SharedBufferNode copy(SharedBufferNode from) {
+			return new SharedBufferNode(edgesSerializer.copy(from.edges));
+		}
+
+		@Override
+		public SharedBufferNode copy(SharedBufferNode from, SharedBufferNode reuse) {
+			return copy(from);
+		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(SharedBufferNode record, DataOutputView target) throws IOException {
+			edgesSerializer.serialize(record.edges, target);
+		}
+
+		@Override
+		public SharedBufferNode deserialize(DataInputView source) throws IOException {
+			List<SharedBufferEdge> edges = edgesSerializer.deserialize(source);
+			return new SharedBufferNode(edges);
+		}
+
+		@Override
+		public SharedBufferNode deserialize(SharedBufferNode reuse, DataInputView source) throws IOException {
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			edgesSerializer.copy(source, target);
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj.getClass().equals(SharedBufferNodeSerializer.class);
+		}
+	}
+}


[10/10] flink git commit: [FLINK-9538] Make KeyedStateFunction an interface

Posted by dw...@apache.org.
[FLINK-9538] Make KeyedStateFunction an interface

This closes #6134


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

Branch: refs/heads/master
Commit: 975f9b1b8c4ebb4c96eebc57aafefa8d10c8a689
Parents: 05ee3ce
Author: yanghua <ya...@gmail.com>
Authored: Thu Jun 7 15:27:25 2018 +0800
Committer: Dawid Wysakowicz <dw...@apache.org>
Committed: Wed Jun 13 15:02:10 2018 +0200

----------------------------------------------------------------------
 .../flink/runtime/state/KeyedStateFunction.java |  5 ++--
 .../runtime/state/StateBackendTestBase.java     | 27 ++++++++++++++++++++
 2 files changed, 30 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/975f9b1b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
index b125de9..8bb352a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateFunction.java
@@ -26,7 +26,8 @@ import org.apache.flink.api.common.state.State;
  * @param <K> The type of key.
  * @param <S> The type of state.
  */
-public abstract class KeyedStateFunction<K, S extends State> {
+@FunctionalInterface
+public interface KeyedStateFunction<K, S extends State> {
 
 	/**
 	 * The actual method to be applied on each of the states.
@@ -34,5 +35,5 @@ public abstract class KeyedStateFunction<K, S extends State> {
 	 * @param key the key whose state is being processed.
 	 * @param state the state associated with the aforementioned key.
 	 */
-	public abstract void process(K key, S state) throws Exception;
+	void process(K key, S state) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/975f9b1b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index b809d84..845b751 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -3748,6 +3748,33 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	}
 
 	@Test
+	public void testApplyToAllKeysLambdaFunction() throws Exception {
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+
+		try {
+			ListStateDescriptor<String> listStateDescriptor =
+				new ListStateDescriptor<>("foo", StringSerializer.INSTANCE);
+
+			ListState<String> listState =
+				backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, listStateDescriptor);
+
+			for (int i = 0; i < 100; ++i) {
+				backend.setCurrentKey(i);
+				listState.add("Hello" + i);
+			}
+
+			// valid state value via applyToAllKeys().
+			backend.applyToAllKeys(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, listStateDescriptor,
+				(Integer key, ListState<String> state) -> assertEquals("Hello" + key, state.get().iterator().next())
+			);
+		}
+		finally {
+			IOUtils.closeQuietly(backend);
+			backend.dispose();
+		}
+	}
+
+	@Test
 	public void testAsyncSnapshotCancellation() throws Exception {
 		OneShotLatch blocker = new OneShotLatch();
 		OneShotLatch waiter = new OneShotLatch();


[02/10] flink git commit: [FLINK-8725] Separate state from NFA in CEP library

Posted by dw...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
index 37ad006..3879699 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAStatusChangeITCase.java
@@ -36,8 +36,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 /**
- * Tests if the {@link NFA} status ({@link NFA#computationStates} or {@link NFA#eventSharedBuffer})
- * is changed after processing events.
+ * Tests if the {@link NFAState} status is changed after processing events.
  */
 public class NFAStatusChangeITCase {
 
@@ -76,45 +75,47 @@ public class NFAStatusChangeITCase {
 		NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true);
 		NFA<Event> nfa = nfaFactory.createNFA();
 
-		nfa.process(new Event(1, "b", 1.0), 1L);
-		assertFalse("NFA status should not change as the event does not match the take condition of the 'start' state", nfa.isNFAChanged());
+		NFAState<Event> nfaState = nfa.createNFAState();
 
-		nfa.resetNFAChanged();
-		nfa.process(new Event(2, "a", 1.0), 2L);
-		assertTrue("NFA status should change as the event matches the take condition of the 'start' state", nfa.isNFAChanged());
+		nfa.process(nfaState, new Event(1, "b", 1.0), 1L);
+		assertFalse("NFA status should not change as the event does not match the take condition of the 'start' state", nfaState.isStateChanged());
+
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(2, "a", 1.0), 2L);
+		assertTrue("NFA status should change as the event matches the take condition of the 'start' state", nfaState.isStateChanged());
 
 		// the status of the queue of ComputationStatus changed,
 		// more than one ComputationStatus is generated by the event from some ComputationStatus
-		nfa.resetNFAChanged();
-		nfa.process(new Event(3, "f", 1.0), 3L);
-		assertTrue("NFA status should change as the event matches the ignore condition and proceed condition of the 'middle:1' state", nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(3, "f", 1.0), 3L);
+		assertTrue("NFA status should change as the event matches the ignore condition and proceed condition of the 'middle:1' state", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have not changed
-		nfa.resetNFAChanged();
-		nfa.process(new Event(4, "f", 1.0), 4L);
-		assertFalse("NFA status should not change as the event only matches the ignore condition of the 'middle:2' state and the target state is still 'middle:2'", nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(4, "f", 1.0), 4L);
+		assertFalse("NFA status should not change as the event only matches the ignore condition of the 'middle:2' state and the target state is still 'middle:2'", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have changed
-		nfa.resetNFAChanged();
-		nfa.process(new Event(5, "b", 1.0), 5L);
-		assertTrue("NFA status should change as the event matches the take condition of 'middle:2' state", nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(5, "b", 1.0), 5L);
+		assertTrue("NFA status should change as the event matches the take condition of 'middle:2' state", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have changed
-		nfa.resetNFAChanged();
-		nfa.process(new Event(6, "d", 1.0), 6L);
-		assertTrue("NFA status should change as the event matches the take condition of 'middle2' state", nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(6, "d", 1.0), 6L);
+		assertTrue("NFA status should change as the event matches the take condition of 'middle2' state", nfaState.isStateChanged());
 
 		// both the queue of ComputationStatus and eventSharedBuffer have not changed
 		// as the timestamp is within the window
-		nfa.resetNFAChanged();
-		nfa.process(null, 8L);
-		assertFalse("NFA status should not change as the timestamp is within the window", nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, null, 8L);
+		assertFalse("NFA status should not change as the timestamp is within the window", nfaState.isStateChanged());
 
 		// timeout ComputationStatus will be removed from the queue of ComputationStatus and timeout event will
 		// be removed from eventSharedBuffer as the timeout happens
-		nfa.resetNFAChanged();
-		Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.process(null, 12L).f1;
-		assertTrue("NFA status should change as timeout happens", nfa.isNFAChanged() && !timeoutResults.isEmpty());
+		nfaState.resetStateChanged();
+		Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutResults = nfa.process(nfaState, null, 12L).f1;
+		assertTrue("NFA status should change as timeout happens", nfaState.isStateChanged() && !timeoutResults.isEmpty());
 	}
 
 	@Test
@@ -143,12 +144,14 @@ public class NFAStatusChangeITCase {
 		NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true);
 		NFA<Event> nfa = nfaFactory.createNFA();
 
-		nfa.resetNFAChanged();
-		nfa.process(new Event(6, "start", 1.0), 6L);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(6, "start", 1.0), 6L);
 
-		nfa.resetNFAChanged();
-		nfa.process(new Event(6, "a", 1.0), 7L);
-		assertTrue(nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(6, "a", 1.0), 7L);
+		assertTrue(nfaState.isStateChanged());
 	}
 
 	@Test
@@ -170,11 +173,13 @@ public class NFAStatusChangeITCase {
 		NFACompiler.NFAFactory<Event> nfaFactory = NFACompiler.compileFactory(pattern, Event.createTypeSerializer(), true);
 		NFA<Event> nfa = nfaFactory.createNFA();
 
-		nfa.resetNFAChanged();
-		nfa.process(new Event(6, "start", 1.0), 6L);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(6, "start", 1.0), 6L);
 
-		nfa.resetNFAChanged();
-		nfa.process(new Event(6, "end", 1.0), 17L);
-		assertTrue(nfa.isNFAChanged());
+		nfaState.resetStateChanged();
+		nfa.process(nfaState, new Event(6, "end", 1.0), 17L);
+		assertTrue(nfaState.isStateChanged());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
index 7721653..5d43111 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATest.java
@@ -52,7 +52,6 @@ import static org.junit.Assert.assertEquals;
 public class NFATest extends TestLogger {
 	@Test
 	public void testSimpleNFA() {
-		NFA<Event> nfa = new NFA<>(Event.createTypeSerializer(), 0, false);
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		streamEvents.add(new StreamRecord<>(new Event(1, "start", 1.0), 1L));
@@ -86,9 +85,12 @@ public class NFATest extends TestLogger {
 			});
 		endState.addIgnore(BooleanConditions.<Event>trueFunction());
 
-		nfa.addState(startState);
-		nfa.addState(endState);
-		nfa.addState(endingState);
+		List<State<Event>> states = new ArrayList<>();
+		states.add(startState);
+		states.add(endState);
+		states.add(endingState);
+
+		NFA<Event> nfa = new NFA<>(Event.createTypeSerializer(), 0, false, states);
 
 		Set<Map<String, List<Event>>> expectedPatterns = new HashSet<>();
 
@@ -103,7 +105,7 @@ public class NFATest extends TestLogger {
 		expectedPatterns.add(firstPattern);
 		expectedPatterns.add(secondPattern);
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
@@ -126,7 +128,7 @@ public class NFATest extends TestLogger {
 
 		expectedPatterns.add(secondPattern);
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
@@ -145,7 +147,7 @@ public class NFATest extends TestLogger {
 
 		Set<Map<String, List<Event>>> expectedPatterns = Collections.emptySet();
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
@@ -172,7 +174,7 @@ public class NFATest extends TestLogger {
 
 		expectedPatterns.add(secondPattern);
 
-		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, streamEvents);
+		Collection<Map<String, List<Event>>> actualPatterns = runNFA(nfa, nfa.createNFAState(), streamEvents);
 
 		assertEquals(expectedPatterns, actualPatterns);
 	}
@@ -187,21 +189,25 @@ public class NFATest extends TestLogger {
 		streamEvents.add(new StreamRecord<>(new Event(3, "loop", 3.0), 103L));
 		streamEvents.add(new StreamRecord<>(new Event(4, "loop", 4.0), 104L));
 		streamEvents.add(new StreamRecord<>(new Event(5, "loop", 5.0), 105L));
-		runNFA(nfa, streamEvents);
 
-		NFA.NFASerializer<Event> serializer = new NFA.NFASerializer<>(Event.createTypeSerializer());
+		NFAState<Event> nfaState = nfa.createNFAState();
+		runNFA(nfa, nfaState, streamEvents);
+
+		NFAStateSerializer<Event> serializer = new NFAStateSerializer<>(Event.createTypeSerializer());
 
 		//serialize
 		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		serializer.serialize(nfa, new DataOutputViewStreamWrapper(baos));
+		serializer.serialize(nfaState, new DataOutputViewStreamWrapper(baos));
 		baos.close();
 	}
 
-	public <T> Collection<Map<String, List<T>>> runNFA(NFA<T> nfa, List<StreamRecord<T>> inputs) {
+	public <T> Collection<Map<String, List<T>>> runNFA(
+		NFA<T> nfa, NFAState<T> nfaState, List<StreamRecord<T>> inputs) {
 		Set<Map<String, List<T>>> actualPatterns = new HashSet<>();
 
 		for (StreamRecord<T> streamEvent : inputs) {
 			Collection<Map<String, List<T>>> matchedPatterns = nfa.process(
+				nfaState,
 				streamEvent.getValue(),
 				streamEvent.getTimestamp()).f0;
 
@@ -311,24 +317,26 @@ public class NFATest extends TestLogger {
 			Event b3 = new Event(41, "b", 5.0);
 			Event d = new Event(43, "d", 4.0);
 
-			nfa.process(a, 1);
-			nfa.process(b, 2);
-			nfa.process(c, 3);
-			nfa.process(b1, 4);
-			nfa.process(b2, 5);
-			nfa.process(b3, 6);
-			nfa.process(d, 7);
-			nfa.process(a, 8);
+			NFAState<Event> nfaState = nfa.createNFAState();
+
+			nfa.process(nfaState, a, 1);
+			nfa.process(nfaState, b, 2);
+			nfa.process(nfaState, c, 3);
+			nfa.process(nfaState, b1, 4);
+			nfa.process(nfaState, b2, 5);
+			nfa.process(nfaState, b3, 6);
+			nfa.process(nfaState, d, 7);
+			nfa.process(nfaState, a, 8);
 
-			NFA.NFASerializer<Event> serializer = new NFA.NFASerializer<>(Event.createTypeSerializer());
+			NFAStateSerializer<Event> serializer = new NFAStateSerializer<>(Event.createTypeSerializer());
 
 			//serialize
 			ByteArrayOutputStream baos = new ByteArrayOutputStream();
-			serializer.serialize(nfa, new DataOutputViewStreamWrapper(baos));
+			serializer.serialize(nfaState, new DataOutputViewStreamWrapper(baos));
 			baos.close();
 
 			// copy
-			NFA.NFASerializer<Event> copySerializer = new NFA.NFASerializer<>(Event.createTypeSerializer());
+			NFAStateSerializer<Event> copySerializer = new NFAStateSerializer<>(Event.createTypeSerializer());
 			ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray());
 			ByteArrayOutputStream out = new ByteArrayOutputStream();
 			copySerializer.duplicate().copy(new DataInputViewStreamWrapper(in), new DataOutputViewStreamWrapper(out));
@@ -337,15 +345,14 @@ public class NFATest extends TestLogger {
 
 			// deserialize
 			ByteArrayInputStream bais = new ByteArrayInputStream(out.toByteArray());
-			NFA<Event> copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais));
+			NFAState<Event> copy = serializer.duplicate().deserialize(new DataInputViewStreamWrapper(bais));
 			bais.close();
 
-			assertEquals(nfa, copy);
+			assertEquals(nfaState, copy);
 		}
 	}
 
 	private NFA<Event> createStartEndNFA(long windowLength) {
-		NFA<Event> nfa = new NFA<>(Event.createTypeSerializer(), windowLength, false);
 
 		State<Event> startState = new State<>("start", State.StateType.Start);
 		State<Event> endState = new State<>("end", State.StateType.Normal);
@@ -373,11 +380,12 @@ public class NFATest extends TestLogger {
 			});
 		endState.addIgnore(BooleanConditions.<Event>trueFunction());
 
-		nfa.addState(startState);
-		nfa.addState(endState);
-		nfa.addState(endingState);
+		List<State<Event>> states = new ArrayList<>();
+		states.add(startState);
+		states.add(endState);
+		states.add(endingState);
 
-		return nfa;
+		return new NFA<>(Event.createTypeSerializer(), windowLength, false, states);
 	}
 
 	private NFA<Event> createLoopingNFA(long windowLength) {

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
index a9e1795..b5312cb 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java
@@ -35,16 +35,36 @@ import java.util.Map;
  */
 public class NFATestUtilities {
 
-	public static List<List<Event>> feedNFA(List<StreamRecord<Event>> inputEvents, NFA<Event> nfa) {
-		return feedNFA(inputEvents, nfa, AfterMatchSkipStrategy.noSkip());
+	public static List<List<Event>> feedNFA(
+		List<StreamRecord<Event>> inputEvents,
+		NFA<Event> nfa) {
+		return feedNFA(inputEvents, nfa, nfa.createNFAState(), AfterMatchSkipStrategy.noSkip());
 	}
 
-	public static List<List<Event>> feedNFA(List<StreamRecord<Event>> inputEvents, NFA<Event> nfa,
-											AfterMatchSkipStrategy afterMatchSkipStrategy) {
+	public static List<List<Event>> feedNFA(
+			List<StreamRecord<Event>> inputEvents,
+			NFA<Event> nfa,
+			NFAState<Event> nfaState) {
+		return feedNFA(inputEvents, nfa, nfaState, AfterMatchSkipStrategy.noSkip());
+	}
+
+	public static List<List<Event>> feedNFA(
+		List<StreamRecord<Event>> inputEvents,
+		NFA<Event> nfa,
+		AfterMatchSkipStrategy afterMatchSkipStrategy) {
+		return feedNFA(inputEvents, nfa, nfa.createNFAState(), afterMatchSkipStrategy);
+	}
+
+	public static List<List<Event>> feedNFA(
+			List<StreamRecord<Event>> inputEvents,
+			NFA<Event> nfa,
+			NFAState<Event> nfaState,
+			AfterMatchSkipStrategy afterMatchSkipStrategy) {
 		List<List<Event>> resultingPatterns = new ArrayList<>();
 
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
+				nfaState,
 				inputEvent.getValue(),
 				inputEvent.getTimestamp(),
 				afterMatchSkipStrategy).f0;

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
index 357107f..c94d739 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java
@@ -140,11 +140,13 @@ public void testClearingBuffer() throws Exception {
 
 	NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-	List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+	NFAState<Event> nfaState = nfa.createNFAState();
+
+	List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 	compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 		Lists.newArrayList(a1, b1, c1, d)
 	));
-	assertTrue(nfa.isEmpty());
+	assertTrue(nfaState.isEmpty());
 }
 
 @Test
@@ -182,13 +184,15 @@ public void testClearingBufferWithUntilAtTheEnd() throws Exception {
 
 	NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-	List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+	NFAState<Event> nfaState = nfa.createNFAState();
+
+	List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 	compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 		Lists.newArrayList(a1, d1, d2, d3),
 		Lists.newArrayList(a1, d1, d2),
 		Lists.newArrayList(a1, d1)
 	));
-	assertTrue(nfa.isEmpty());
+	assertTrue(nfaState.isEmpty());
 }
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
index f88e5b2..b603174 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java
@@ -91,13 +91,15 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -137,7 +139,9 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, breaking),
@@ -145,7 +149,7 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent3, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -185,13 +189,15 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -233,12 +239,14 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -278,14 +286,16 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking),
 			Lists.newArrayList(startEvent, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -325,7 +335,9 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3, breaking),
@@ -334,7 +346,7 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1, breaking),
 			Lists.newArrayList(startEvent, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -374,14 +386,16 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, breaking),
 			Lists.newArrayList(startEvent, middleEvent1, breaking),
 			Lists.newArrayList(startEvent, breaking)
 		));
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -502,7 +516,9 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3),
@@ -510,7 +526,7 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1)
 		));
 
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -553,7 +569,9 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3),
@@ -561,7 +579,7 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent, middleEvent1)
 		));
 
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 	@Test
@@ -604,7 +622,9 @@ public class UntilConditionITCase {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(startEvent, middleEvent1, middleEvent2, middleEvent3),
@@ -613,6 +633,6 @@ public class UntilConditionITCase {
 			Lists.newArrayList(startEvent)
 		));
 
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
index ec2cf47..d1b6d59 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java
@@ -40,6 +40,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -132,7 +133,7 @@ public class NFACompilerTest extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, serializer, false);
 
-		Set<State<Event>> states = nfa.getStates();
+		Collection<State<Event>> states = nfa.getStates();
 		assertEquals(4, states.size());
 
 		Map<String, State<Event>> stateMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
index 2fda47f..3151498 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
@@ -71,7 +71,7 @@ public class CEPMigrationTest {
 
 	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
 	public static Collection<MigrationVersion> parameters () {
-		return Arrays.asList(MigrationVersion.v1_3, MigrationVersion.v1_4);
+		return Arrays.asList(MigrationVersion.v1_5);
 	}
 
 	public CEPMigrationTest(MigrationVersion migrateVersion) {

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
index 98cb468..5a98445 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
@@ -462,9 +462,9 @@ public class CEPOperatorTest extends TestLogger {
 		try {
 			harness.open();
 
-			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "nfaOperatorState");
+			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "nfaValueState");
 			final ValueState nfaOperatorStateSpy = Mockito.spy(nfaOperatorState);
-			Whitebox.setInternalState(operator, "nfaOperatorState", nfaOperatorStateSpy);
+			Whitebox.setInternalState(operator, "nfaValueState", nfaOperatorStateSpy);
 
 			Event startEvent = new Event(42, "c", 1.0);
 			SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0);
@@ -507,9 +507,9 @@ public class CEPOperatorTest extends TestLogger {
 
 			harness.open();
 
-			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "nfaOperatorState");
+			final ValueState nfaOperatorState = (ValueState) Whitebox.<ValueState>getInternalState(operator, "nfaValueState");
 			final ValueState nfaOperatorStateSpy = Mockito.spy(nfaOperatorState);
-			Whitebox.setInternalState(operator, "nfaOperatorState", nfaOperatorStateSpy);
+			Whitebox.setInternalState(operator, "nfaValueState", nfaOperatorStateSpy);
 
 			Event startEvent = new Event(42, "c", 1.0);
 			SubEvent middleEvent = new SubEvent(42, "a", 1.0, 10.0);
@@ -568,8 +568,8 @@ public class CEPOperatorTest extends TestLogger {
 			assertEquals(2L, harness.numEventTimeTimers());
 			assertEquals(4L, operator.getPQSize(42));
 			assertEquals(1L, operator.getPQSize(43));
-			assertTrue(!operator.hasNonEmptyNFA(42));
-			assertTrue(!operator.hasNonEmptyNFA(43));
+			assertTrue(!operator.hasNonEmptyNFAState(42));
+			assertTrue(!operator.hasNonEmptyNFAState(43));
 
 			harness.processWatermark(new Watermark(2L));
 
@@ -581,9 +581,9 @@ public class CEPOperatorTest extends TestLogger {
 			// for 43 the element entered the NFA and the PQ is empty
 
 			assertEquals(2L, harness.numEventTimeTimers());
-			assertTrue(operator.hasNonEmptyNFA(42));
+			assertTrue(operator.hasNonEmptyNFAState(42));
 			assertEquals(1L, operator.getPQSize(42));
-			assertTrue(operator.hasNonEmptyNFA(43));
+			assertTrue(operator.hasNonEmptyNFAState(43));
 			assertTrue(!operator.hasNonEmptyPQ(43));
 
 			harness.processElement(new StreamRecord<>(startEvent2, 4L));
@@ -605,9 +605,9 @@ public class CEPOperatorTest extends TestLogger {
 			// now we have 1 key because the 43 expired and was removed.
 			// 42 is still there due to startEvent2
 			assertEquals(1L, harness.numEventTimeTimers());
-			assertTrue(operator2.hasNonEmptyNFA(42));
+			assertTrue(operator2.hasNonEmptyNFAState(42));
 			assertTrue(!operator2.hasNonEmptyPQ(42));
-			assertTrue(!operator2.hasNonEmptyNFA(43));
+			assertTrue(!operator2.hasNonEmptyNFAState(43));
 			assertTrue(!operator2.hasNonEmptyPQ(43));
 
 			verifyPattern(harness.getOutput().poll(), startEvent1, middleEvent1, endEvent1);
@@ -622,7 +622,7 @@ public class CEPOperatorTest extends TestLogger {
 			harness.processWatermark(20L);
 			harness.processWatermark(21L);
 
-			assertTrue(!operator2.hasNonEmptyNFA(42));
+			assertTrue(!operator2.hasNonEmptyNFAState(42));
 			assertTrue(!operator2.hasNonEmptyPQ(42));
 			assertEquals(0L, harness.numEventTimeTimers());
 
@@ -665,7 +665,7 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertEquals(1L, harness.numEventTimeTimers());
 			assertEquals(7L, operator.getPQSize(41));
-			assertTrue(!operator.hasNonEmptyNFA(41));
+			assertTrue(!operator.hasNonEmptyNFAState(41));
 
 			harness.processWatermark(new Watermark(2L));
 
@@ -674,7 +674,7 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertEquals(1L, harness.numEventTimeTimers());
 			assertEquals(6L, operator.getPQSize(41));
-			assertTrue(operator.hasNonEmptyNFA(41)); // processed the first element
+			assertTrue(operator.hasNonEmptyNFAState(41)); // processed the first element
 
 			harness.processWatermark(new Watermark(8L));
 
@@ -714,12 +714,12 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertEquals(1L, harness.numEventTimeTimers());
 			assertEquals(0L, operator.getPQSize(41));
-			assertTrue(operator.hasNonEmptyNFA(41));
+			assertTrue(operator.hasNonEmptyNFAState(41));
 
 			harness.processWatermark(new Watermark(17L));
 			verifyWatermark(harness.getOutput().poll(), 17L);
 
-			assertTrue(!operator.hasNonEmptyNFA(41));
+			assertTrue(!operator.hasNonEmptyNFAState(41));
 			assertTrue(!operator.hasNonEmptyPQ(41));
 			assertEquals(0L, harness.numEventTimeTimers());
 		} finally {
@@ -800,8 +800,8 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertTrue(!operator.hasNonEmptyPQ(42));
 			assertTrue(!operator.hasNonEmptyPQ(43));
-			assertTrue(operator.hasNonEmptyNFA(42));
-			assertTrue(operator.hasNonEmptyNFA(43));
+			assertTrue(operator.hasNonEmptyNFAState(42));
+			assertTrue(operator.hasNonEmptyNFAState(43));
 
 			harness.setProcessingTime(3L);
 
@@ -834,10 +834,10 @@ public class CEPOperatorTest extends TestLogger {
 
 			harness.setProcessingTime(21L);
 
-			assertTrue(operator2.hasNonEmptyNFA(42));
+			assertTrue(operator2.hasNonEmptyNFAState(42));
 
 			harness.processElement(new StreamRecord<>(startEvent1, 21L));
-			assertTrue(operator2.hasNonEmptyNFA(42));
+			assertTrue(operator2.hasNonEmptyNFAState(42));
 
 			harness.setProcessingTime(49L);
 
@@ -845,7 +845,7 @@ public class CEPOperatorTest extends TestLogger {
 			harness.processElement(new StreamRecord<>(new Event(42, "foobar", 1.0), 2L));
 
 			// the pattern expired
-			assertTrue(!operator2.hasNonEmptyNFA(42));
+			assertTrue(!operator2.hasNonEmptyNFAState(42));
 
 			assertEquals(0L, harness.numEventTimeTimers());
 			assertTrue(!operator2.hasNonEmptyPQ(42));
@@ -988,12 +988,12 @@ public class CEPOperatorTest extends TestLogger {
 			harness
 				.processElement(new StreamRecord<>(new SubEvent(42, "barfoo", 1.0, 5.0), 0L));
 
-			assertTrue(!operator.hasNonEmptyNFA(42));
-			assertTrue(!operator.hasNonEmptyNFA(43));
+			assertTrue(!operator.hasNonEmptyNFAState(42));
+			assertTrue(!operator.hasNonEmptyNFAState(43));
 
 			harness.setProcessingTime(3L);
-			assertTrue(operator.hasNonEmptyNFA(42));
-			assertTrue(operator.hasNonEmptyNFA(43));
+			assertTrue(operator.hasNonEmptyNFAState(42));
+			assertTrue(operator.hasNonEmptyNFAState(43));
 
 			harness.processElement(new StreamRecord<>(middleEvent2, 3L));
 			harness.processElement(new StreamRecord<>(middleEvent1, 3L));
@@ -1047,14 +1047,14 @@ public class CEPOperatorTest extends TestLogger {
 
 			assertTrue(operator.hasNonEmptyPQ(42));
 			assertTrue(operator.hasNonEmptyPQ(43));
-			assertTrue(!operator.hasNonEmptyNFA(42));
-			assertTrue(!operator.hasNonEmptyNFA(43));
+			assertTrue(!operator.hasNonEmptyNFAState(42));
+			assertTrue(!operator.hasNonEmptyNFAState(43));
 
 			harness.processWatermark(3L);
 			assertTrue(!operator.hasNonEmptyPQ(42));
 			assertTrue(!operator.hasNonEmptyPQ(43));
-			assertTrue(operator.hasNonEmptyNFA(42));
-			assertTrue(operator.hasNonEmptyNFA(43));
+			assertTrue(operator.hasNonEmptyNFAState(42));
+			assertTrue(operator.hasNonEmptyNFAState(43));
 
 			harness.processElement(new StreamRecord<>(startEvent2, 4L));
 			harness.processElement(new StreamRecord<Event>(middleEvent2, 5L));

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot
deleted file mode 100644
index 34c5110..0000000
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.4-snapshot and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot
new file mode 100644
index 0000000..73934e5
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot
deleted file mode 100644
index d4d3405..0000000
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.4-snapshot and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot
new file mode 100644
index 0000000..3126ff4
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-conditions-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot
deleted file mode 100644
index c87f307..0000000
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.4-snapshot and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot
new file mode 100644
index 0000000..63b1b49
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.5-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot
deleted file mode 100644
index 660a21a..0000000
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.4-snapshot and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot
new file mode 100644
index 0000000..1b1fa75
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.5-snapshot differ


[06/10] flink git commit: [FLINK-9418] Migrate SharedBuffer to use MapState

Posted by dw...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index 7cdf732..73a9200 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -31,9 +31,11 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.EventComparator;
 import org.apache.flink.cep.nfa.AfterMatchSkipStrategy;
 import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.nfa.NFA.MigratedNFA;
 import org.apache.flink.cep.nfa.NFAState;
 import org.apache.flink.cep.nfa.NFAStateSerializer;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
 import org.apache.flink.runtime.state.KeyedStateFunction;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.VoidNamespace;
@@ -54,7 +56,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
 import java.util.stream.Stream;
-import java.util.stream.StreamSupport;
 
 /**
  * Abstract CEP pattern operator for a keyed input stream. For each key, the operator creates
@@ -68,8 +69,8 @@ import java.util.stream.StreamSupport;
  * @param <OUT> Type of the output elements
  */
 public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Function>
-	extends AbstractUdfStreamOperator<OUT, F>
-	implements OneInputStreamOperator<IN, OUT>, Triggerable<KEY, VoidNamespace> {
+		extends AbstractUdfStreamOperator<OUT, F>
+		implements OneInputStreamOperator<IN, OUT>, Triggerable<KEY, VoidNamespace> {
 
 	private static final long serialVersionUID = -4166778210774160757L;
 
@@ -82,11 +83,12 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	private static final String NFA_STATE_NAME = "nfaStateName";
 	private static final String EVENT_QUEUE_STATE_NAME = "eventQueuesStateName";
 
-	private transient ValueState<NFAState<IN>> nfaValueState;
-	private transient MapState<Long, List<IN>> elementQueueState;
-
 	private final NFACompiler.NFAFactory<IN> nfaFactory;
 
+	private transient ValueState<NFAState> computationStates;
+	private transient MapState<Long, List<IN>> elementQueueState;
+	private transient SharedBuffer<IN> partialMatches;
+
 	private transient InternalTimerService<VoidNamespace> timerService;
 
 	private transient NFA<IN> nfa;
@@ -134,22 +136,19 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	public void initializeState(StateInitializationContext context) throws Exception {
 		super.initializeState(context);
 
-		if (nfaValueState == null) {
-			nfaValueState = getRuntimeContext().getState(
+		// initializeState through the provided context
+		computationStates = context.getKeyedStateStore().getState(
 				new ValueStateDescriptor<>(
 						NFA_STATE_NAME,
-						new NFAStateSerializer<>(inputSerializer)));
-		}
+						NFAStateSerializer.INSTANCE));
 
-		if (elementQueueState == null) {
-			elementQueueState = getRuntimeContext().getMapState(
-					new MapStateDescriptor<>(
-							EVENT_QUEUE_STATE_NAME,
-							LongSerializer.INSTANCE,
-							new ListSerializer<>(inputSerializer)
-					)
-			);
-		}
+		partialMatches = new SharedBuffer<>(context.getKeyedStateStore(), inputSerializer);
+
+		elementQueueState = context.getKeyedStateStore().getMapState(
+				new MapStateDescriptor<>(
+						EVENT_QUEUE_STATE_NAME,
+						LongSerializer.INSTANCE,
+						new ListSerializer<>(inputSerializer)));
 
 		migrateOldState();
 	}
@@ -162,15 +161,14 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 				"nfaOperatorStateName",
 				new NFA.NFASerializer<>(inputSerializer)
 			),
-			new KeyedStateFunction<Object, ValueState<NFA<IN>>>() {
+			new KeyedStateFunction<Object, ValueState<MigratedNFA<IN>>>() {
 				@Override
-				public void process(Object key, ValueState<NFA<IN>> state) throws Exception {
-					NFA<IN> oldState = state.value();
-					if (oldState instanceof NFA.NFASerializer.DummyNFA) {
-						NFA.NFASerializer.DummyNFA<IN> dummyNFA = (NFA.NFASerializer.DummyNFA<IN>) oldState;
-						nfaValueState.update(new NFAState<>(dummyNFA.getComputationStates(), dummyNFA.getSharedBuffer(), false));
-						state.clear();
-					}
+				public void process(Object key, ValueState<MigratedNFA<IN>> state) throws Exception {
+					MigratedNFA<IN> oldState = state.value();
+					computationStates.update(new NFAState(oldState.getComputationStates()));
+					org.apache.flink.cep.nfa.SharedBuffer<IN> sharedBuffer = oldState.getSharedBuffer();
+					partialMatches.init(sharedBuffer.getEventsBuffer(), sharedBuffer.getPages());
+					state.clear();
 				}
 			}
 		);
@@ -193,7 +191,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		if (isProcessingTime) {
 			if (comparator == null) {
 				// there can be no out of order elements in processing time
-				NFAState<IN> nfaState = getNFAState();
+				NFAState nfaState = getNFAState();
 				processEvent(nfaState, element.getValue(), getProcessingTimeService().getCurrentProcessingTime());
 				updateNFA(nfaState);
 			} else {
@@ -269,14 +267,22 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 
 		// STEP 1
 		PriorityQueue<Long> sortedTimestamps = getSortedTimestamps();
-		NFAState<IN> nfaState = getNFAState();
+		NFAState nfaState = getNFAState();
 
 		// STEP 2
 		while (!sortedTimestamps.isEmpty() && sortedTimestamps.peek() <= timerService.currentWatermark()) {
 			long timestamp = sortedTimestamps.poll();
-			sort(elementQueueState.get(timestamp)).forEachOrdered(
-				event -> processEvent(nfaState, event, timestamp)
-			);
+			try (Stream<IN> elements = sort(elementQueueState.get(timestamp))) {
+				elements.forEachOrdered(
+						event -> {
+							try {
+								processEvent(nfaState, event, timestamp);
+							} catch (Exception e) {
+								throw new RuntimeException(e);
+							}
+						}
+				);
+			}
 			elementQueueState.remove(timestamp);
 		}
 
@@ -289,7 +295,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		}
 		updateNFA(nfaState);
 
-		if (!sortedTimestamps.isEmpty() || !nfaState.isEmpty()) {
+		if (!sortedTimestamps.isEmpty() || !partialMatches.isEmpty()) {
 			saveRegisterWatermarkTimer();
 		}
 
@@ -307,14 +313,22 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 
 		// STEP 1
 		PriorityQueue<Long> sortedTimestamps = getSortedTimestamps();
-		NFAState<IN> nfa = getNFAState();
+		NFAState nfa = getNFAState();
 
 		// STEP 2
 		while (!sortedTimestamps.isEmpty()) {
 			long timestamp = sortedTimestamps.poll();
-			sort(elementQueueState.get(timestamp)).forEachOrdered(
-				event -> processEvent(nfa, event, timestamp)
-			);
+			try (Stream<IN> elements = sort(elementQueueState.get(timestamp))) {
+				elements.forEachOrdered(
+						event -> {
+							try {
+								processEvent(nfa, event, timestamp);
+							} catch (Exception e) {
+								throw new RuntimeException(e);
+							}
+						}
+				);
+			}
 			elementQueueState.remove(timestamp);
 		}
 
@@ -325,38 +339,30 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		updateNFA(nfa);
 	}
 
-	private Stream<IN> sort(Iterable<IN> iter) {
-		Stream<IN> stream = StreamSupport.stream(iter.spliterator(), false);
-		if (comparator == null) {
-			return stream;
-		} else {
-			return stream.sorted(comparator);
-		}
+	private Stream<IN> sort(Collection<IN> elements) {
+		Stream<IN> stream = elements.stream();
+		return (comparator == null) ? stream : stream.sorted(comparator);
 	}
 
 	private void updateLastSeenWatermark(long timestamp) {
 		this.lastWatermark = timestamp;
 	}
 
-	private NFAState<IN> getNFAState() throws IOException {
-		NFAState<IN> nfaState = nfaValueState.value();
-		return nfaState != null ? nfaState : nfa.createNFAState();
+	private NFAState getNFAState() throws IOException {
+		NFAState nfaState = computationStates.value();
+		return nfaState != null ? nfaState : nfa.createInitialNFAState();
 	}
 
-	private void updateNFA(NFAState<IN> nfaState) throws IOException {
+	private void updateNFA(NFAState nfaState) throws IOException {
 		if (nfaState.isStateChanged()) {
-			if (nfaState.isEmpty()) {
-				nfaValueState.clear();
-			} else {
-				nfaState.resetStateChanged();
-				nfaValueState.update(nfaState);
-			}
+			nfaState.resetStateChanged();
+			computationStates.update(nfaState);
 		}
 	}
 
 	private PriorityQueue<Long> getSortedTimestamps() throws Exception {
 		PriorityQueue<Long> sortedTimestamps = new PriorityQueue<>();
-		for (Long timestamp: elementQueueState.keys()) {
+		for (Long timestamp : elementQueueState.keys()) {
 			sortedTimestamps.offer(timestamp);
 		}
 		return sortedTimestamps;
@@ -370,24 +376,18 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	 * @param event The current event to be processed
 	 * @param timestamp The timestamp of the event
 	 */
-	private void processEvent(NFAState<IN> nfaState, IN event, long timestamp)  {
+	private void processEvent(NFAState nfaState, IN event, long timestamp) throws Exception {
 		Tuple2<Collection<Map<String, List<IN>>>, Collection<Tuple2<Map<String, List<IN>>, Long>>> patterns =
-			nfa.process(nfaState, event, timestamp, afterMatchSkipStrategy);
-
-		try {
-			processMatchedSequences(patterns.f0, timestamp);
-			processTimedOutSequences(patterns.f1, timestamp);
-		} catch (Exception e) {
-			//rethrow as Runtime, to be able to use processEvent in Stream.
-			throw new RuntimeException(e);
-		}
+				nfa.process(partialMatches, nfaState, event, timestamp, afterMatchSkipStrategy);
+		processMatchedSequences(patterns.f0, timestamp);
+		processTimedOutSequences(patterns.f1, timestamp);
 	}
 
 	/**
 	 * Advances the time for the given NFA to the given timestamp. This can lead to pruning and
 	 * timeouts.
 	 */
-	private void advanceTime(NFAState<IN> nfaState, long timestamp) throws Exception {
+	private void advanceTime(NFAState nfaState, long timestamp) throws Exception {
 		processEvent(nfaState, null, timestamp);
 	}
 
@@ -401,9 +401,9 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	//////////////////////			Testing Methods			//////////////////////
 
 	@VisibleForTesting
-	public boolean hasNonEmptyNFAState(KEY key) throws IOException {
+	public boolean hasNonEmptySharedBuffer(KEY key) throws Exception {
 		setCurrentKey(key);
-		return nfaValueState.value() != null;
+		return !partialMatches.isEmpty();
 	}
 
 	@VisibleForTesting

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java
index a3b98f5..bc5e2b1 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java
@@ -270,7 +270,7 @@ public class CEPOperatorUtils {
 		final boolean isProcessingTime = inputStream.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
 
 		// compile our pattern into a NFAFactory to instantiate NFAs later on
-		final NFACompiler.NFAFactory<IN> nfaFactory = NFACompiler.compileFactory(pattern, inputSerializer, timeoutHandling);
+		final NFACompiler.NFAFactory<IN> nfaFactory = NFACompiler.compileFactory(pattern, timeoutHandling);
 
 		final SingleOutputStreamOperator<OUT> patternStream;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
index 0386d0e..f716ec4 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
@@ -93,6 +93,6 @@ public abstract class IterativeCondition<T> implements Function, Serializable {
 		 *
 		 * @param name The name of the pattern.
 		 */
-		Iterable<T> getEventsForPattern(String name);
+		Iterable<T> getEventsForPattern(String name) throws Exception;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
index f767d92..e6efd5e 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -34,6 +33,7 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 
 /**
  * IT tests covering {@link AfterMatchSkipStrategy}.
@@ -41,7 +41,7 @@ import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
 public class AfterMatchSkipITCase extends TestLogger{
 
 	@Test
-	public void testSkipToNext() {
+	public void testSkipToNext() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event a1 = new Event(1, "a", 0.0);
@@ -67,7 +67,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				}
 			}).times(3);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -80,7 +80,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipPastLast() {
+	public void testSkipPastLast() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event a1 = new Event(1, "a", 0.0);
@@ -106,7 +106,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				}
 			}).times(3);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -117,7 +117,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipToFirst() {
+	public void testSkipToFirst() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event ab1 = new Event(1, "ab", 0.0);
@@ -150,7 +150,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				}
 			}).times(2);
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -161,7 +161,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipToLast() {
+	public void testSkipToLast() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event ab1 = new Event(1, "ab", 0.0);
@@ -193,7 +193,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				return value.getName().contains("b");
 			}
 		}).times(2);
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -204,7 +204,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipPastLast2() {
+	public void testSkipPastLast2() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event a1 = new Event(1, "a1", 0.0);
@@ -251,7 +251,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 					return value.getName().contains("d");
 				}
 		});
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -268,7 +268,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipPastLast3() {
+	public void testSkipPastLast3() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event a1 = new Event(1, "a1", 0.0);
@@ -297,7 +297,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				}
 			}
 		);
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -307,7 +307,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipToFirstWithOptionalMatch() {
+	public void testSkipToFirstWithOptionalMatch() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event ab1 = new Event(1, "ab1", 0.0);
@@ -341,7 +341,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				return value.getName().contains("c");
 			}
 		});
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -352,7 +352,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipToFirstAtStartPosition() {
+	public void testSkipToFirstAtStartPosition() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event ab1 = new Event(1, "ab1", 0.0);
@@ -380,7 +380,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				return value.getName().contains("c");
 			}
 		});
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -391,7 +391,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipToFirstWithOneOrMore() {
+	public void testSkipToFirstWithOneOrMore() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event a1 = new Event(1, "a1", 0.0);
@@ -425,7 +425,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				return value.getName().contains("b");
 			}
 		}).oneOrMore().consecutive();
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 
@@ -437,7 +437,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 	}
 
 	@Test
-	public void testSkipToLastWithOneOrMore() {
+	public void testSkipToLastWithOneOrMore() throws Exception {
 		List<StreamRecord<Event>> streamEvents = new ArrayList<>();
 
 		Event a1 = new Event(1, "a1", 0.0);
@@ -471,7 +471,7 @@ public class AfterMatchSkipITCase extends TestLogger{
 				return value.getName().contains("b");
 			}
 		}).oneOrMore().consecutive();
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GreedyITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GreedyITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GreedyITCase.java
index 2c7f23c..9e00130 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GreedyITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GreedyITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -34,6 +33,7 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 
 /**
  * IT tests covering {@link Pattern#greedy()}.
@@ -41,7 +41,7 @@ import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
 public class GreedyITCase extends TestLogger {
 
 	@Test
-	public void testGreedyZeroOrMore() {
+	public void testGreedyZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -80,7 +80,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -90,7 +90,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyZeroOrMoreInBetween() {
+	public void testGreedyZeroOrMoreInBetween() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -132,7 +132,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -142,7 +142,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyZeroOrMoreWithDummyEventsAfterQuantifier() {
+	public void testGreedyZeroOrMoreWithDummyEventsAfterQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -180,7 +180,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -190,7 +190,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyZeroOrMoreWithDummyEventsBeforeQuantifier() {
+	public void testGreedyZeroOrMoreWithDummyEventsBeforeQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -224,7 +224,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -234,7 +234,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyUntilZeroOrMoreWithDummyEventsAfterQuantifier() {
+	public void testGreedyUntilZeroOrMoreWithDummyEventsAfterQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -281,7 +281,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -291,7 +291,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyUntilWithDummyEventsBeforeQuantifier() {
+	public void testGreedyUntilWithDummyEventsBeforeQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -338,7 +338,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -348,7 +348,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyOneOrMore() {
+	public void testGreedyOneOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -387,7 +387,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -397,7 +397,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyOneOrMoreInBetween() {
+	public void testGreedyOneOrMoreInBetween() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -439,7 +439,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -449,7 +449,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyOneOrMoreWithDummyEventsAfterQuantifier() {
+	public void testGreedyOneOrMoreWithDummyEventsAfterQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -487,7 +487,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -497,7 +497,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyOneOrMoreWithDummyEventsBeforeQuantifier() {
+	public void testGreedyOneOrMoreWithDummyEventsBeforeQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -531,7 +531,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -539,7 +539,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyUntilOneOrMoreWithDummyEventsAfterQuantifier() {
+	public void testGreedyUntilOneOrMoreWithDummyEventsAfterQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -586,7 +586,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -596,7 +596,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyUntilOneOrMoreWithDummyEventsBeforeQuantifier() {
+	public void testGreedyUntilOneOrMoreWithDummyEventsBeforeQuantifier() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -643,7 +643,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -651,7 +651,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyZeroOrMoreBeforeGroupPattern() {
+	public void testGreedyZeroOrMoreBeforeGroupPattern() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -713,7 +713,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -723,7 +723,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEndWithZeroOrMoreGreedy() {
+	public void testEndWithZeroOrMoreGreedy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -754,7 +754,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		}).oneOrMore().optional().greedy();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -767,7 +767,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEndWithZeroOrMoreConsecutiveGreedy() {
+	public void testEndWithZeroOrMoreConsecutiveGreedy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -798,7 +798,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		}).oneOrMore().optional().consecutive().greedy();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -810,7 +810,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testEndWithGreedyTimesRange() {
+	public void testEndWithGreedyTimesRange() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -843,7 +843,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		}).times(2, 5).greedy();
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -855,7 +855,7 @@ public class GreedyITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGreedyTimesRange() {
+	public void testGreedyTimesRange() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -896,7 +896,7 @@ public class GreedyITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
index f9b9aa2..a992f46 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
@@ -19,7 +19,6 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.GroupPattern;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
@@ -35,7 +34,8 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
-import static org.junit.Assert.assertTrue;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
+import static org.junit.Assert.assertEquals;
 
 /**
  * IT tests covering {@link GroupPattern}.
@@ -44,7 +44,7 @@ import static org.junit.Assert.assertTrue;
 public class GroupITCase extends TestLogger {
 
 	@Test
-	public void testGroupFollowedByTimes() {
+	public void testGroupFollowedByTimes() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -92,7 +92,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -102,7 +102,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByOptional() {
+	public void testGroupFollowedByOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -148,7 +148,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -159,7 +159,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testFollowedByGroupTimesOptional() {
+	public void testFollowedByGroupTimesOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -201,7 +201,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -211,7 +211,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByOneOrMore() {
+	public void testGroupFollowedByOneOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -259,7 +259,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -270,7 +270,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByZeroOrMore() {
+	public void testGroupFollowedByZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -318,7 +318,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -330,7 +330,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByAnyTimesCombinations() {
+	public void testGroupFollowedByAnyTimesCombinations() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -382,7 +382,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -394,7 +394,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByAnyTimesOptional() {
+	public void testGroupFollowedByAnyTimesOptional() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -446,7 +446,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -458,7 +458,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByAnyOneOrMore() {
+	public void testGroupFollowedByAnyOneOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -510,7 +510,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -525,7 +525,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupNextZeroOrMore() {
+	public void testGroupNextZeroOrMore() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -577,7 +577,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -590,7 +590,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupNotFollowedBy() {
+	public void testGroupNotFollowedBy() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -647,7 +647,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -657,7 +657,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupNotNext() {
+	public void testGroupNotNext() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -714,7 +714,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -724,7 +724,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupNest() {
+	public void testGroupNest() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event d = new Event(40, "d", 1.0);
@@ -781,7 +781,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -794,7 +794,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupNestTimes() {
+	public void testGroupNestTimes() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event d = new Event(40, "d", 1.0);
@@ -869,7 +869,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -882,7 +882,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupNestTimesConsecutive() {
+	public void testGroupNestTimesConsecutive() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event d = new Event(40, "d", 1.0);
@@ -958,7 +958,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -969,7 +969,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupBegin() {
+	public void testGroupBegin() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event a1 = new Event(41, "a", 2.0);
@@ -1008,7 +1008,7 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -1020,7 +1020,7 @@ public class GroupITCase extends TestLogger {
 	}
 
 	@Test
-	public void testGroupFollowedByOneOrMoreWithUntilCondition() {
+	public void testGroupFollowedByOneOrMoreWithUntilCondition() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		Event c = new Event(40, "c", 1.0);
@@ -1073,9 +1073,9 @@ public class GroupITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
-		NFAState<Event> nfaState = nfa.createNFAState();
+		NFAState nfaState = nfa.createInitialNFAState();
 
 		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
@@ -1084,7 +1084,8 @@ public class GroupITCase extends TestLogger {
 			Lists.newArrayList(c, a1, b1, a2, b2, d)
 		));
 
-		assertTrue(nfaState.isEmpty());
+		assertEquals(1, nfaState.getComputationStates().size());
+		assertEquals("start", nfaState.getComputationStates().peek().getCurrentStateName());
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java
index 80754b7..9012379 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java
@@ -20,7 +20,6 @@ package org.apache.flink.cep.nfa;
 
 import org.apache.flink.cep.Event;
 import org.apache.flink.cep.SubEvent;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.pattern.Pattern;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.cep.pattern.conditions.SimpleCondition;
@@ -36,6 +35,7 @@ import java.util.List;
 
 import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps;
 import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA;
+import static org.apache.flink.cep.utils.NFAUtils.compile;
 
 /**
  * IT tests covering {@link IterativeCondition} usage.
@@ -57,7 +57,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	private final Event endEvent = new Event(46, "end", 1.0);
 
 	@Test
-	public void testIterativeWithBranchingPatternEager() {
+	public void testIterativeWithBranchingPatternEager() throws Exception {
 		List<List<Event>> actual = testIterativeWithBranchingPattern(true);
 
 		compareMaps(actual,
@@ -72,7 +72,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIterativeWithBranchingPatternCombinations() {
+	public void testIterativeWithBranchingPatternCombinations() throws Exception {
 		List<List<Event>> actual = testIterativeWithBranchingPattern(false);
 
 		compareMaps(actual,
@@ -88,7 +88,7 @@ public class IterativeConditionsITCase extends TestLogger {
 		);
 	}
 
-	private List<List<Event>> testIterativeWithBranchingPattern(boolean eager) {
+	private List<List<Event>> testIterativeWithBranchingPattern(boolean eager) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(startEvent1, 1));
@@ -136,7 +136,7 @@ public class IterativeConditionsITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
@@ -161,7 +161,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIterativeWithLoopingStartingEager() {
+	public void testIterativeWithLoopingStartingEager() throws Exception {
 		List<List<Event>> actual = testIterativeWithLoopingStarting(true);
 
 		compareMaps(actual,
@@ -176,7 +176,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIterativeWithLoopingStartingCombination() {
+	public void testIterativeWithLoopingStartingCombination() throws Exception {
 		List<List<Event>> actual = testIterativeWithLoopingStarting(false);
 
 		compareMaps(actual,
@@ -191,7 +191,7 @@ public class IterativeConditionsITCase extends TestLogger {
 		);
 	}
 
-	private List<List<Event>> testIterativeWithLoopingStarting(boolean eager) {
+	private List<List<Event>> testIterativeWithLoopingStarting(boolean eager) throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(startEvent1, 1L));
@@ -223,7 +223,7 @@ public class IterativeConditionsITCase extends TestLogger {
 				}
 			});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		return feedNFA(inputEvents, nfa);
 	}
@@ -248,7 +248,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIterativeWithPrevPatternDependency() {
+	public void testIterativeWithPrevPatternDependency() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(startEvent1, 1L));
@@ -279,7 +279,7 @@ public class IterativeConditionsITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -292,7 +292,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIterativeWithABACPattern() {
+	public void testIterativeWithABACPattern() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(startEvent1, 1L)); //1
@@ -346,7 +346,7 @@ public class IterativeConditionsITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 
@@ -362,7 +362,7 @@ public class IterativeConditionsITCase extends TestLogger {
 	}
 
 	@Test
-	public void testIterativeWithPrevPatternDependencyAfterBranching() {
+	public void testIterativeWithPrevPatternDependencyAfterBranching() throws Exception {
 		List<StreamRecord<Event>> inputEvents = new ArrayList<>();
 
 		inputEvents.add(new StreamRecord<>(startEvent1, 1L));
@@ -403,7 +403,7 @@ public class IterativeConditionsITCase extends TestLogger {
 			}
 		});
 
-		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
+		NFA<Event> nfa = compile(pattern, false);
 
 		List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
 


[08/10] flink git commit: [FLINK-9418] Migrate SharedBuffer to use MapState

Posted by dw...@apache.org.
[FLINK-9418] Migrate SharedBuffer to use MapState


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

Branch: refs/heads/master
Commit: 9218df82bc5748cf7fd4717b5b5e2f7dba0e01be
Parents: 45d5442
Author: Dawid Wysakowicz <dw...@apache.org>
Authored: Fri Jun 1 16:26:10 2018 +0200
Committer: Dawid Wysakowicz <dw...@apache.org>
Committed: Wed Jun 13 14:59:59 2018 +0200

----------------------------------------------------------------------
 .../apache/flink/cep/nfa/ComputationState.java  |  96 +-
 .../org/apache/flink/cep/nfa/DeweyNumber.java   |   4 +
 .../apache/flink/cep/nfa/MigrationUtils.java    | 131 +++
 .../main/java/org/apache/flink/cep/nfa/NFA.java | 584 +++++-------
 .../flink/cep/nfa/NFASerializationUtils.java    |  70 --
 .../java/org/apache/flink/cep/nfa/NFAState.java | 101 +-
 .../flink/cep/nfa/NFAStateSerializer.java       | 162 +---
 .../nfa/NFAStateSerializerConfigSnapshot.java   |  45 -
 .../org/apache/flink/cep/nfa/SharedBuffer.java  | 939 +++----------------
 .../flink/cep/nfa/compiler/NFACompiler.java     |  37 +-
 .../flink/cep/nfa/sharedbuffer/EventId.java     | 141 +++
 .../flink/cep/nfa/sharedbuffer/Lockable.java    | 204 ++++
 .../flink/cep/nfa/sharedbuffer/NodeId.java      | 155 +++
 .../cep/nfa/sharedbuffer/SharedBuffer.java      | 395 ++++++++
 .../cep/nfa/sharedbuffer/SharedBufferEdge.java  | 126 +++
 .../cep/nfa/sharedbuffer/SharedBufferNode.java  | 120 +++
 .../AbstractKeyedCEPPatternOperator.java        | 136 +--
 .../flink/cep/operator/CEPOperatorUtils.java    |   2 +-
 .../pattern/conditions/IterativeCondition.java  |   2 +-
 .../flink/cep/nfa/AfterMatchSkipITCase.java     |  42 +-
 .../org/apache/flink/cep/nfa/GreedyITCase.java  |  70 +-
 .../org/apache/flink/cep/nfa/GroupITCase.java   |  73 +-
 .../cep/nfa/IterativeConditionsITCase.java      |  30 +-
 .../org/apache/flink/cep/nfa/NFAITCase.java     | 320 ++++---
 .../flink/cep/nfa/NFAStateAccessTest.java       | 197 ++++
 .../flink/cep/nfa/NFAStatusChangeITCase.java    |  57 +-
 .../java/org/apache/flink/cep/nfa/NFATest.java  | 153 ++-
 .../apache/flink/cep/nfa/NFATestUtilities.java  |  18 +-
 .../apache/flink/cep/nfa/NotPatternITCase.java  |  90 +-
 .../apache/flink/cep/nfa/SameElementITCase.java |  34 +-
 .../apache/flink/cep/nfa/SharedBufferTest.java  | 239 -----
 .../apache/flink/cep/nfa/TimesOrMoreITCase.java |  46 +-
 .../apache/flink/cep/nfa/TimesRangeITCase.java  |  50 +-
 .../flink/cep/nfa/UntilConditionITCase.java     |  79 +-
 .../flink/cep/nfa/compiler/NFACompilerTest.java |  15 +-
 .../cep/nfa/sharedbuffer/SharedBufferTest.java  | 216 +++++
 .../flink/cep/operator/CEPMigrationTest.java    |   8 +-
 .../flink/cep/operator/CEPOperatorTest.java     |  73 +-
 .../flink/cep/operator/CEPRescalingTest.java    |  31 +-
 .../cep/operator/CepOperatorTestUtilities.java  |   2 +
 .../org/apache/flink/cep/utils/NFAUtils.java    |  47 +
 .../flink/cep/utils/TestSharedBuffer.java       | 251 +++++
 42 files changed, 3126 insertions(+), 2465 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
index c9aef27..65715a7 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
@@ -18,78 +18,51 @@
 
 package org.apache.flink.cep.nfa;
 
+import org.apache.flink.cep.nfa.sharedbuffer.NodeId;
+
 import javax.annotation.Nullable;
 
 import java.util.Objects;
 
 /**
- * Helper class which encapsulates the state of the NFA computation. It points to the current state,
- * the last taken event, its occurrence timestamp, the current version and the starting timestamp
+ * Helper class which encapsulates the currentStateName of the NFA computation. It points to the current currentStateName,
+ * the previous entry of the pattern, the current version and the starting timestamp
  * of the overall pattern.
- *
- * @param <T> Type of the input events
  */
-public class ComputationState<T> {
-	// pointer to the NFA state of the computation
-	private final String state;
-
-	// the last taken event
-	private final T event;
-
-	private final int counter;
+public class ComputationState {
+	// pointer to the NFA currentStateName of the computation
+	private final String currentStateName;
 
-	// timestamp of the last taken event
-	private final long timestamp;
-
-	// The current version of the state to discriminate the valid pattern paths in the SharedBuffer
+	// The current version of the currentStateName to discriminate the valid pattern paths in the SharedBuffer
 	private final DeweyNumber version;
 
 	// Timestamp of the first element in the pattern
 	private final long startTimestamp;
 
 	@Nullable
-	private final String previousState;
+	private final NodeId previousBufferEntry;
 
 	private ComputationState(
 			final String currentState,
-			@Nullable final String previousState,
-			final T event,
-			final int counter,
-			final long timestamp,
+			@Nullable final NodeId previousBufferEntry,
 			final DeweyNumber version,
 			final long startTimestamp) {
-		this.state = currentState;
-		this.event = event;
-		this.counter = counter;
-		this.timestamp = timestamp;
+		this.currentStateName = currentState;
 		this.version = version;
 		this.startTimestamp = startTimestamp;
-		this.previousState = previousState;
-	}
-
-	public int getCounter() {
-		return counter;
+		this.previousBufferEntry = previousBufferEntry;
 	}
 
-	public long getTimestamp() {
-		return timestamp;
+	public NodeId getPreviousBufferEntry() {
+		return previousBufferEntry;
 	}
 
 	public long getStartTimestamp() {
 		return startTimestamp;
 	}
 
-	public String getState() {
-		return state;
-	}
-
-	@Nullable
-	public String getPreviousState() {
-		return previousState;
-	}
-
-	public T getEvent() {
-		return event;
+	public String getCurrentStateName() {
+		return currentStateName;
 	}
 
 	public DeweyNumber getVersion() {
@@ -100,40 +73,43 @@ public class ComputationState<T> {
 	public boolean equals(Object obj) {
 		if (obj instanceof ComputationState) {
 			ComputationState other = (ComputationState) obj;
-			return Objects.equals(state, other.state) &&
-				Objects.equals(event, other.event) &&
-				counter == other.counter &&
-				timestamp == other.timestamp &&
+			return Objects.equals(currentStateName, other.currentStateName) &&
 				Objects.equals(version, other.version) &&
 				startTimestamp == other.startTimestamp &&
-				Objects.equals(previousState, other.previousState);
-
+				Objects.equals(previousBufferEntry, other.previousBufferEntry);
 		} else {
 			return false;
 		}
 	}
 
 	@Override
+	public String toString() {
+		return "ComputationState{" +
+			"currentStateName='" + currentStateName + '\'' +
+			", version=" + version +
+			", startTimestamp=" + startTimestamp +
+			", previousBufferEntry=" + previousBufferEntry +
+			'}';
+	}
+
+	@Override
 	public int hashCode() {
-		return Objects.hash(state, event, counter, timestamp, version, startTimestamp, previousState);
+		return Objects.hash(currentStateName, version, startTimestamp, previousBufferEntry);
 	}
 
-	public static <T> ComputationState<T> createStartState(final String state) {
-		return new ComputationState<>(state, null, null, 0, -1L, new DeweyNumber(1), -1L);
+	public static ComputationState createStartState(final String state) {
+		return createStartState(state, new DeweyNumber(1));
 	}
 
-	public static <T> ComputationState<T> createStartState(final String state, final DeweyNumber version) {
-		return new ComputationState<T>(state, null, null, 0, -1L, version, -1L);
+	public static ComputationState createStartState(final String state, final DeweyNumber version) {
+		return createState(state, null, version, -1L);
 	}
 
-	public static <T> ComputationState<T> createState(
+	public static ComputationState createState(
 			final String currentState,
-			final String previousState,
-			final T event,
-			final int counter,
-			final long timestamp,
+			final NodeId previousEntry,
 			final DeweyNumber version,
 			final long startTimestamp) {
-		return new ComputationState<>(currentState, previousState, event, counter, timestamp, version, startTimestamp);
+		return new ComputationState(currentState, previousEntry, version, startTimestamp);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
index f066141..34897fa 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/DeweyNumber.java
@@ -192,6 +192,10 @@ public class DeweyNumber implements Serializable {
 
 		private final IntSerializer elemSerializer = IntSerializer.INSTANCE;
 
+		public static final DeweyNumberSerializer INSTANCE = new DeweyNumberSerializer();
+
+		private DeweyNumberSerializer() {}
+
 		@Override
 		public boolean isImmutableType() {
 			return false;

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/MigrationUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/MigrationUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/MigrationUtils.java
new file mode 100644
index 0000000..f1656a1
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/MigrationUtils.java
@@ -0,0 +1,131 @@
+/*
+ * 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.nfa;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.EnumSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.cep.nfa.sharedbuffer.NodeId;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.util.Preconditions;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.LinkedList;
+import java.util.Queue;
+
+/**
+ * Methods for deserialization of old format NFA.
+ */
+class MigrationUtils {
+
+	/**
+	 * Skips bytes corresponding to serialized states. In flink 1.6+ the states are no longer kept in state.
+	 */
+	static <T> void skipSerializedStates(DataInputView in) throws IOException {
+		TypeSerializer<String> nameSerializer = StringSerializer.INSTANCE;
+		TypeSerializer<State.StateType> stateTypeSerializer = new EnumSerializer<>(State.StateType.class);
+		TypeSerializer<StateTransitionAction> actionSerializer = new EnumSerializer<>(StateTransitionAction.class);
+
+		final int noOfStates = in.readInt();
+
+		for (int i = 0; i < noOfStates; i++) {
+			nameSerializer.deserialize(in);
+			stateTypeSerializer.deserialize(in);
+		}
+
+		for (int i = 0; i < noOfStates; i++) {
+			String srcName = nameSerializer.deserialize(in);
+
+			int noOfTransitions = in.readInt();
+			for (int j = 0; j < noOfTransitions; j++) {
+				String src = nameSerializer.deserialize(in);
+				Preconditions.checkState(src.equals(srcName),
+					"Source Edge names do not match (" + srcName + " - " + src + ").");
+
+				nameSerializer.deserialize(in);
+				actionSerializer.deserialize(in);
+
+				try {
+					skipCondition(in);
+				} catch (ClassNotFoundException e) {
+					e.printStackTrace();
+				}
+			}
+		}
+	}
+
+	private static <T> void skipCondition(DataInputView in) throws IOException, ClassNotFoundException {
+		boolean hasCondition = in.readBoolean();
+		if (hasCondition) {
+			int length = in.readInt();
+
+			byte[] serCondition = new byte[length];
+			in.read(serCondition);
+
+			ByteArrayInputStream bais = new ByteArrayInputStream(serCondition);
+			ObjectInputStream ois = new ObjectInputStream(bais);
+
+			ois.readObject();
+			ois.close();
+			bais.close();
+		}
+	}
+
+	static <T> Queue<ComputationState> deserializeComputationStates(
+			org.apache.flink.cep.nfa.SharedBuffer<T> sharedBuffer,
+			TypeSerializer<T> eventSerializer,
+			DataInputView source) throws IOException {
+
+		Queue<ComputationState> computationStates = new LinkedList<>();
+		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
+		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
+		DeweyNumber.DeweyNumberSerializer versionSerializer = DeweyNumber.DeweyNumberSerializer.INSTANCE;
+
+		int computationStateNo = source.readInt();
+		for (int i = 0; i < computationStateNo; i++) {
+			String state = stateNameSerializer.deserialize(source);
+			String prevState = stateNameSerializer.deserialize(source);
+			long timestamp = timestampSerializer.deserialize(source);
+			DeweyNumber version = versionSerializer.deserialize(source);
+			long startTimestamp = timestampSerializer.deserialize(source);
+			int counter = source.readInt();
+
+			T event = null;
+			if (source.readBoolean()) {
+				event = eventSerializer.deserialize(source);
+			}
+
+			NodeId nodeId;
+			if (prevState != null) {
+				nodeId = sharedBuffer.getNodeId(prevState, timestamp, counter, event);
+			} else {
+				nodeId = null;
+			}
+
+			computationStates.add(ComputationState.createState(state, nodeId, version, startTimestamp));
+		}
+		return computationStates;
+	}
+
+	private MigrationUtils() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index bc28cc5..227a34d 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -26,21 +26,21 @@ import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.common.typeutils.base.EnumSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
-import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler;
+import org.apache.flink.cep.nfa.sharedbuffer.EventId;
+import org.apache.flink.cep.nfa.sharedbuffer.NodeId;
+import org.apache.flink.cep.nfa.sharedbuffer.SharedBuffer;
 import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.Preconditions;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -48,13 +48,14 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.Stack;
 
-import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.cep.nfa.MigrationUtils.deserializeComputationStates;
 
 /**
  * Non-deterministic finite automaton implementation.
@@ -101,21 +102,22 @@ public class NFA<T> {
 	 */
 	private final boolean handleTimeout;
 
-	private final TypeSerializer<T> eventSerializer;
-
 	public NFA(
-			final TypeSerializer<T> eventSerializer,
+			final Collection<State<T>> validStates,
 			final long windowTime,
-			final boolean handleTimeout,
-			final Collection<State<T>> states) {
-		this.eventSerializer = checkNotNull(eventSerializer);
+			final boolean handleTimeout
+	) {
 		this.windowTime = windowTime;
 		this.handleTimeout = handleTimeout;
+		this.states = loadStates(validStates);
+	}
 
-		this.states = new HashMap<>();
-		for (State<T> state : states) {
-			this.states.put(state.getName(), state);
+	private Map<String, State<T>> loadStates(final Collection<State<T>> validStates) {
+		Map<String, State<T>> tmp = new HashMap<>(4);
+		for (State<T> state : validStates) {
+			tmp.put(state.getName(), state);
 		}
+		return Collections.unmodifiableMap(tmp);
 	}
 
 	@VisibleForTesting
@@ -123,46 +125,45 @@ public class NFA<T> {
 		return states.values();
 	}
 
-	public NFAState<T> createNFAState() {
-		List<ComputationState<T>> startingStates = new ArrayList<>();
+	public NFAState createInitialNFAState() {
+		Queue<ComputationState> startingStates = new LinkedList<>();
 		for (State<T> state : states.values()) {
 			if (state.isStart()) {
 				startingStates.add(ComputationState.createStartState(state.getName()));
 			}
 		}
-		return new NFAState<>(startingStates);
-	}
-
-	private State<T> getState(String state) {
-		return states.get(state);
+		return new NFAState(startingStates);
 	}
 
-	private State<T> getState(ComputationState<T> state) {
-		return states.get(state.getState());
+	private State<T> getState(ComputationState state) {
+		return states.get(state.getCurrentStateName());
 	}
 
-	private boolean isStartState(ComputationState<T> state) {
+	private boolean isStartState(ComputationState state) {
 		State<T> stateObject = getState(state);
 		if (stateObject == null) {
-			throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values());
+			throw new FlinkRuntimeException("State " + state.getCurrentStateName() + " does not exist in the NFA. NFA has states "
+				+ states.values());
 		}
 
 		return stateObject.isStart();
 	}
 
-	private boolean isStopState(ComputationState<T> state) {
+	private boolean isStopState(ComputationState state) {
 		State<T> stateObject = getState(state);
 		if (stateObject == null) {
-			throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values());
+			throw new FlinkRuntimeException("State " + state.getCurrentStateName() + " does not exist in the NFA. NFA has states "
+				+ states.values());
 		}
 
 		return stateObject.isStop();
 	}
 
-	private boolean isFinalState(ComputationState<T> state) {
+	private boolean isFinalState(ComputationState state) {
 		State<T> stateObject = getState(state);
 		if (stateObject == null) {
-			throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values());
+			throw new FlinkRuntimeException("State " + state.getCurrentStateName() + " does not exist in the NFA. NFA has states "
+				+ states.values());
 		}
 
 		return stateObject.isFinal();
@@ -177,6 +178,7 @@ public class NFA<T> {
 	 * <p>If computations reach a stop state, the path forward is discarded and currently constructed path is returned
 	 * with the element that resulted in the stop state.
 	 *
+	 * @param sharedBuffer the SharedBuffer object that we need to work upon while processing
 	 * @param nfaState The NFAState object that we need to affect while processing
 	 * @param event The current event to be processed or null if only pruning shall be done
 	 * @param timestamp The timestamp of the current event
@@ -185,8 +187,11 @@ public class NFA<T> {
 	 * activated)
 	 */
 	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(
-		NFAState<T> nfaState, final T event, final long timestamp) {
-		return process(nfaState, event, timestamp, AfterMatchSkipStrategy.noSkip());
+			final SharedBuffer<T> sharedBuffer,
+			final NFAState nfaState,
+			final T event,
+			final long timestamp) throws Exception {
+		return process(sharedBuffer, nfaState, event, timestamp, AfterMatchSkipStrategy.noSkip());
 	}
 
 	/**
@@ -197,6 +202,7 @@ public class NFA<T> {
 	 * <p>If computations reach a stop state, the path forward is discarded and currently constructed path is returned
 	 * with the element that resulted in the stop state.
 	 *
+	 * @param sharedBuffer the SharedBuffer object that we need to work upon while processing
 	 * @param nfaState The NFAState object that we need to affect while processing
 	 * @param event The current event to be processed or null if only pruning shall be done
 	 * @param timestamp The timestamp of the current event
@@ -206,13 +212,24 @@ public class NFA<T> {
 	 * activated)
 	 */
 	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(
-			NFAState<T> nfaState,
+			final SharedBuffer<T> sharedBuffer,
+			final NFAState nfaState,
 			final T event,
 			final long timestamp,
-			AfterMatchSkipStrategy afterMatchSkipStrategy) {
+			final AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
+
+		try (EventWrapper eventWrapper = new EventWrapper(event, timestamp, sharedBuffer)) {
+			return doProcess(sharedBuffer, nfaState, eventWrapper, afterMatchSkipStrategy);
+		}
+	}
+
+	private Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> doProcess(
+			final SharedBuffer<T> sharedBuffer,
+			final NFAState nfaState,
+			final EventWrapper event,
+			final AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
 
-		Queue<ComputationState<T>> computationStates = nfaState.getComputationStates();
-		SharedBuffer<String, T> eventSharedBuffer = nfaState.getEventSharedBuffer();
+		Queue<ComputationState> computationStates = nfaState.getComputationStates();
 
 		final int numberComputationStates = computationStates.size();
 		final Collection<Map<String, List<T>>> result = new ArrayList<>();
@@ -220,67 +237,53 @@ public class NFA<T> {
 
 		// iterate over all current computations
 		for (int i = 0; i < numberComputationStates; i++) {
-			ComputationState<T> computationState = computationStates.poll();
+			ComputationState computationState = computationStates.poll();
 
-			final Collection<ComputationState<T>> newComputationStates;
+			final Collection<ComputationState> newComputationStates;
 
 			if (!isStartState(computationState) &&
 				windowTime > 0L &&
-				timestamp - computationState.getStartTimestamp() >= windowTime) {
+				event.getTimestamp() - computationState.getStartTimestamp() >= windowTime) {
 
 				if (handleTimeout) {
 					// extract the timed out event pattern
-					Map<String, List<T>> timedOutPattern = extractCurrentMatches(eventSharedBuffer, computationState);
-					timeoutResult.add(Tuple2.of(timedOutPattern, timestamp));
+					Map<String, List<T>> timedOutPattern = extractCurrentMatches(sharedBuffer, computationState);
+					timeoutResult.add(Tuple2.of(timedOutPattern, event.getTimestamp()));
 				}
 
-				eventSharedBuffer.release(
-						NFAStateNameHandler.getOriginalNameFromInternal(computationState.getPreviousState()),
-						computationState.getEvent(),
-						computationState.getTimestamp(),
-						computationState.getCounter());
+				sharedBuffer.releaseNode(computationState.getPreviousBufferEntry());
 
 				newComputationStates = Collections.emptyList();
-				nfaState.setStateChanged(true);
-			} else if (event != null) {
-				newComputationStates = computeNextStates(eventSharedBuffer, computationState, event, timestamp);
+				nfaState.setStateChanged();
+			} else if (event.getEvent() != null) {
+				newComputationStates = computeNextStates(sharedBuffer, computationState, event, event.getTimestamp());
 
 				if (newComputationStates.size() != 1) {
-					nfaState.setStateChanged(true);
+					nfaState.setStateChanged();
 				} else if (!newComputationStates.iterator().next().equals(computationState)) {
-					nfaState.setStateChanged(true);
+					nfaState.setStateChanged();
 				}
 			} else {
 				newComputationStates = Collections.singleton(computationState);
 			}
 
 			//delay adding new computation states in case a stop state is reached and we discard the path.
-			final Collection<ComputationState<T>> statesToRetain = new ArrayList<>();
+			final Collection<ComputationState> statesToRetain = new ArrayList<>();
 			//if stop state reached in this path
 			boolean shouldDiscardPath = false;
-			for (final ComputationState<T> newComputationState: newComputationStates) {
+			for (final ComputationState newComputationState : newComputationStates) {
 
 				if (isFinalState(newComputationState)) {
 					// we've reached a final state and can thus retrieve the matching event sequence
-					Map<String, List<T>> matchedPattern = extractCurrentMatches(eventSharedBuffer, newComputationState);
+					Map<String, List<T>> matchedPattern = extractCurrentMatches(sharedBuffer, newComputationState);
 					result.add(matchedPattern);
 
 					// remove found patterns because they are no longer needed
-					eventSharedBuffer.release(
-							NFAStateNameHandler.getOriginalNameFromInternal(
-									newComputationState.getPreviousState()),
-							newComputationState.getEvent(),
-							newComputationState.getTimestamp(),
-							newComputationState.getCounter());
+					sharedBuffer.releaseNode(newComputationState.getPreviousBufferEntry());
 				} else if (isStopState(newComputationState)) {
 					//reached stop state. release entry for the stop state
 					shouldDiscardPath = true;
-					eventSharedBuffer.release(
-							NFAStateNameHandler.getOriginalNameFromInternal(
-									newComputationState.getPreviousState()),
-							newComputationState.getEvent(),
-							newComputationState.getTimestamp(),
-							newComputationState.getCounter());
+					sharedBuffer.releaseNode(newComputationState.getPreviousBufferEntry());
 				} else {
 					// add new computation state; it will be processed once the next event arrives
 					statesToRetain.add(newComputationState);
@@ -290,13 +293,8 @@ public class NFA<T> {
 			if (shouldDiscardPath) {
 				// a stop state was reached in this branch. release branch which results in removing previous event from
 				// the buffer
-				for (final ComputationState<T> state : statesToRetain) {
-					eventSharedBuffer.release(
-							NFAStateNameHandler.getOriginalNameFromInternal(
-									state.getPreviousState()),
-							state.getEvent(),
-							state.getTimestamp(),
-							state.getCounter());
+				for (final ComputationState state : statesToRetain) {
+					sharedBuffer.releaseNode(state.getPreviousBufferEntry());
 				}
 			} else {
 				computationStates.addAll(statesToRetain);
@@ -305,30 +303,17 @@ public class NFA<T> {
 		}
 
 		discardComputationStatesAccordingToStrategy(
-			eventSharedBuffer, computationStates, result, afterMatchSkipStrategy);
-
-		// prune shared buffer based on window length
-		if (windowTime > 0L) {
-			long pruningTimestamp = timestamp - windowTime;
-
-			if (pruningTimestamp < timestamp) {
-				// the check is to guard against underflows
-
-				// remove all elements which are expired
-				// with respect to the window length
-				if (eventSharedBuffer.prune(pruningTimestamp)) {
-					nfaState.setStateChanged(true);
-				}
-			}
-		}
+			sharedBuffer, computationStates, result, afterMatchSkipStrategy);
 
 		return Tuple2.of(result, timeoutResult);
 	}
 
 	private void discardComputationStatesAccordingToStrategy(
-		SharedBuffer<String, T> eventSharedBuffer,
-		Queue<ComputationState<T>> computationStates,
-		Collection<Map<String, List<T>>> matchedResult, AfterMatchSkipStrategy afterMatchSkipStrategy) {
+			final SharedBuffer<T> sharedBuffer,
+			final Queue<ComputationState> computationStates,
+			final Collection<Map<String, List<T>>> matchedResult,
+			final AfterMatchSkipStrategy afterMatchSkipStrategy) throws Exception {
+
 		Set<T> discardEvents = new HashSet<>();
 		switch(afterMatchSkipStrategy.getStrategy()) {
 			case SKIP_TO_LAST:
@@ -363,24 +348,25 @@ public class NFA<T> {
 				break;
 		}
 		if (!discardEvents.isEmpty()) {
-			List<ComputationState<T>> discardStates = new ArrayList<>();
-			for (ComputationState<T> computationState : computationStates) {
-				Map<String, List<T>> partialMatch = extractCurrentMatches(eventSharedBuffer, computationState);
+			List<ComputationState> discardStates = new ArrayList<>();
+			for (ComputationState computationState : computationStates) {
+				boolean discard = false;
+				Map<String, List<T>> partialMatch = extractCurrentMatches(sharedBuffer, computationState);
 				for (List<T> list: partialMatch.values()) {
 					for (T e: list) {
 						if (discardEvents.contains(e)) {
 							// discard the computation state.
-							eventSharedBuffer.release(
-								NFAStateNameHandler.getOriginalNameFromInternal(
-									computationState.getState()),
-								computationState.getEvent(),
-								computationState.getTimestamp(),
-								computationState.getCounter()
-							);
-							discardStates.add(computationState);
+							discard = true;
 							break;
 						}
 					}
+					if (discard) {
+						break;
+					}
+				}
+				if (discard) {
+					sharedBuffer.releaseNode(computationState.getPreviousBufferEntry());
+					discardStates.add(computationState);
 				}
 			}
 			computationStates.removeAll(discardStates);
@@ -439,6 +425,50 @@ public class NFA<T> {
 	}
 
 	/**
+	 * Helper class that ensures event is registered only once throughout the life of this object and released on close
+	 * of this object. This allows to wrap whole processing of the event with try-with-resources block.
+	 */
+	private class EventWrapper implements AutoCloseable {
+
+		private final T event;
+
+		private long timestamp;
+
+		private final SharedBuffer<T> sharedBuffer;
+
+		private EventId eventId;
+
+		EventWrapper(T event, long timestamp, SharedBuffer<T> sharedBuffer) {
+			this.event = event;
+			this.timestamp = timestamp;
+			this.sharedBuffer = sharedBuffer;
+		}
+
+		EventId getEventId() throws Exception {
+			if (eventId == null) {
+				this.eventId = sharedBuffer.registerEvent(event, timestamp);
+			}
+
+			return eventId;
+		}
+
+		T getEvent() {
+			return event;
+		}
+
+		public long getTimestamp() {
+			return timestamp;
+		}
+
+		@Override
+		public void close() throws Exception {
+			if (eventId != null) {
+				sharedBuffer.releaseEvent(eventId);
+			}
+		}
+	}
+
+	/**
 	 * Computes the next computation states based on the given computation state, the current event,
 	 * its timestamp and the internal state machine. The algorithm is:
 	 *<ol>
@@ -465,19 +495,19 @@ public class NFA<T> {
 	 *     <li>Release the corresponding entries in {@link SharedBuffer}.</li>
 	 *</ol>
 	 *
-	 * @param eventSharedBuffer The shared buffer that we need to change
+	 * @param sharedBuffer The shared buffer that we need to change
 	 * @param computationState Current computation state
 	 * @param event Current event which is processed
 	 * @param timestamp Timestamp of the current event
 	 * @return Collection of computation states which result from the current one
 	 */
-	private Collection<ComputationState<T>> computeNextStates(
-			final SharedBuffer<String, T> eventSharedBuffer,
-			final ComputationState<T> computationState,
-			final T event,
-			final long timestamp) {
+	private Collection<ComputationState> computeNextStates(
+			final SharedBuffer<T> sharedBuffer,
+			final ComputationState computationState,
+			final EventWrapper event,
+			final long timestamp) throws Exception {
 
-		final OutgoingEdges<T> outgoingEdges = createDecisionGraph(eventSharedBuffer, computationState, event);
+		final OutgoingEdges<T> outgoingEdges = createDecisionGraph(sharedBuffer, computationState, event.getEvent());
 
 		// Create the computing version based on the previously computed edges
 		// We need to defer the creation of computation states until we know how many edges start
@@ -487,7 +517,7 @@ public class NFA<T> {
 		int ignoreBranchesToVisit = outgoingEdges.getTotalIgnoreBranches();
 		int totalTakeToSkip = Math.max(0, outgoingEdges.getTotalTakeBranches() - 1);
 
-		final List<ComputationState<T>> resultingComputationStates = new ArrayList<>();
+		final List<ComputationState> resultingComputationStates = new ArrayList<>();
 		for (StateTransition<T> edge : edges) {
 			switch (edge.getAction()) {
 				case IGNORE: {
@@ -508,13 +538,10 @@ public class NFA<T> {
 						}
 
 						addComputationState(
-								eventSharedBuffer,
+								sharedBuffer,
 								resultingComputationStates,
 								edge.getTargetState(),
-								getState(computationState.getPreviousState()),
-								computationState.getEvent(),
-								computationState.getCounter(),
-								computationState.getTimestamp(),
+								computationState.getPreviousBufferEntry(),
 								version,
 								computationState.getStartTimestamp()
 						);
@@ -524,61 +551,47 @@ public class NFA<T> {
 				case TAKE:
 					final State<T> nextState = edge.getTargetState();
 					final State<T> currentState = edge.getSourceState();
-					final State<T> previousState = getState(computationState.getPreviousState());
 
-					final T previousEvent = computationState.getEvent();
+					final NodeId previousEntry = computationState.getPreviousBufferEntry();
 
 					final DeweyNumber currentVersion = computationState.getVersion().increase(takeBranchesToVisit);
 					final DeweyNumber nextVersion = new DeweyNumber(currentVersion).addStage();
 					takeBranchesToVisit--;
 
-					final int counter;
+					final NodeId newEntry;
 					final long startTimestamp;
 					if (isStartState(computationState)) {
 						startTimestamp = timestamp;
-						counter = eventSharedBuffer.put(
-							NFAStateNameHandler.getOriginalNameFromInternal(
-									currentState.getName()),
-							event,
-							timestamp,
+						newEntry = sharedBuffer.put(
+							currentState.getName(),
+							event.getEventId(),
+							previousEntry,
 							currentVersion);
 					} else {
 						startTimestamp = computationState.getStartTimestamp();
-						counter = eventSharedBuffer.put(
-							NFAStateNameHandler.getOriginalNameFromInternal(
-									currentState.getName()),
-							event,
-							timestamp,
-							NFAStateNameHandler.getOriginalNameFromInternal(
-									previousState.getName()),
-							previousEvent,
-							computationState.getTimestamp(),
-							computationState.getCounter(),
+						newEntry = sharedBuffer.put(
+							currentState.getName(),
+							event.getEventId(),
+							previousEntry,
 							currentVersion);
 					}
 
 					addComputationState(
-							eventSharedBuffer,
+							sharedBuffer,
 							resultingComputationStates,
 							nextState,
-							currentState,
-							event,
-							counter,
-							timestamp,
+							newEntry,
 							nextVersion,
 							startTimestamp);
 
 					//check if newly created state is optional (have a PROCEED path to Final state)
-					final State<T> finalState = findFinalStateAfterProceed(eventSharedBuffer, nextState, event, computationState);
+					final State<T> finalState = findFinalStateAfterProceed(sharedBuffer, nextState, event.getEvent(), computationState);
 					if (finalState != null) {
 						addComputationState(
-								eventSharedBuffer,
+								sharedBuffer,
 								resultingComputationStates,
 								finalState,
-								currentState,
-								event,
-								counter,
-								timestamp,
+								newEntry,
 								nextVersion,
 								startTimestamp);
 					}
@@ -592,45 +605,34 @@ public class NFA<T> {
 					outgoingEdges.getTotalTakeBranches());
 
 			DeweyNumber startVersion = computationState.getVersion().increase(totalBranches);
-			ComputationState<T> startState = ComputationState.createStartState(computationState.getState(), startVersion);
+			ComputationState startState = ComputationState.createStartState(computationState.getCurrentStateName(), startVersion);
 			resultingComputationStates.add(startState);
 		}
 
-		if (computationState.getEvent() != null) {
+		if (computationState.getPreviousBufferEntry() != null) {
 			// release the shared entry referenced by the current computation state.
-			eventSharedBuffer.release(
-					NFAStateNameHandler.getOriginalNameFromInternal(
-							computationState.getPreviousState()),
-					computationState.getEvent(),
-					computationState.getTimestamp(),
-					computationState.getCounter());
+			sharedBuffer.releaseNode(computationState.getPreviousBufferEntry());
 		}
 
 		return resultingComputationStates;
 	}
 
 	private void addComputationState(
-			SharedBuffer<String, T> eventSharedBuffer,
-			List<ComputationState<T>> computationStates,
+			SharedBuffer<T> sharedBuffer,
+			List<ComputationState> computationStates,
 			State<T> currentState,
-			State<T> previousState,
-			T event,
-			int counter,
-			long timestamp,
+			NodeId previousEntry,
 			DeweyNumber version,
-			long startTimestamp) {
-		ComputationState<T> computationState = ComputationState.createState(
-				currentState.getName(), previousState.getName(), event, counter, timestamp, version, startTimestamp);
+			long startTimestamp) throws Exception {
+		ComputationState computationState = ComputationState.createState(
+				currentState.getName(), previousEntry, version, startTimestamp);
 		computationStates.add(computationState);
 
-		String originalStateName = NFAStateNameHandler.getOriginalNameFromInternal(previousState.getName());
-		eventSharedBuffer.lock(originalStateName, event, timestamp, counter);
+		sharedBuffer.lockNode(previousEntry);
 	}
 
-	private State<T> findFinalStateAfterProceed(
-		SharedBuffer<String, T> eventSharedBuffer,
-		State<T> state, T event,
-		ComputationState<T> computationState) {
+	private State<T> findFinalStateAfterProceed(SharedBuffer<T> sharedBuffer, State<T> state, T event,
+			ComputationState computationState) {
 		final Stack<State<T>> statesToCheck = new Stack<>();
 		statesToCheck.push(state);
 
@@ -639,7 +641,7 @@ public class NFA<T> {
 				final State<T> currentState = statesToCheck.pop();
 				for (StateTransition<T> transition : currentState.getStateTransitions()) {
 					if (transition.getAction() == StateTransitionAction.PROCEED &&
-							checkFilterCondition(eventSharedBuffer, computationState, transition.getCondition(), event)) {
+							checkFilterCondition(sharedBuffer, computationState, transition.getCondition(), event)) {
 						if (transition.getTargetState().isFinal()) {
 							return transition.getTargetState();
 						} else {
@@ -649,7 +651,7 @@ public class NFA<T> {
 				}
 			}
 		} catch (Exception e) {
-			throw new RuntimeException("Failure happened in filter function.", e);
+			throw new FlinkRuntimeException("Failure happened in filter function.", e);
 		}
 
 		return null;
@@ -659,9 +661,8 @@ public class NFA<T> {
 		return takeBranches == 0 && ignoreBranches == 0 ? 0 : ignoreBranches + Math.max(1, takeBranches);
 	}
 
-	private OutgoingEdges<T> createDecisionGraph(
-		SharedBuffer<String, T> eventSharedBuffer,
-		ComputationState<T> computationState, T event) {
+	private OutgoingEdges<T> createDecisionGraph(SharedBuffer<T> sharedBuffer, ComputationState computationState,
+			T event) {
 		State<T> state = getState(computationState);
 		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(state);
 
@@ -676,7 +677,7 @@ public class NFA<T> {
 			// check all state transitions for each state
 			for (StateTransition<T> stateTransition : stateTransitions) {
 				try {
-					if (checkFilterCondition(eventSharedBuffer, computationState, stateTransition.getCondition(), event)) {
+					if (checkFilterCondition(sharedBuffer, computationState, stateTransition.getCondition(), event)) {
 						// filter condition is true
 						switch (stateTransition.getAction()) {
 							case PROCEED:
@@ -691,19 +692,16 @@ public class NFA<T> {
 						}
 					}
 				} catch (Exception e) {
-					throw new RuntimeException("Failure happened in filter function.", e);
+					throw new FlinkRuntimeException("Failure happened in filter function.", e);
 				}
 			}
 		}
 		return outgoingEdges;
 	}
 
-	private boolean checkFilterCondition(
-		SharedBuffer<String, T> eventSharedBuffer,
-		ComputationState<T> computationState,
-		IterativeCondition<T> condition,
-		T event) throws Exception {
-		return condition == null || condition.filter(event, new ConditionContext<>(this, eventSharedBuffer, computationState));
+	private boolean checkFilterCondition(SharedBuffer<T> sharedBuffer, ComputationState computationState,
+			IterativeCondition<T> condition, T event) throws Exception {
+		return condition == null || condition.filter(event, new ConditionContext<>(this, sharedBuffer, computationState));
 	}
 
 	/**
@@ -711,22 +709,18 @@ public class NFA<T> {
 	 * sequence is returned as a map which contains the events and the names of the states to which
 	 * the events were mapped.
 	 *
-	 * @param eventSharedBuffer The {@link SharedBuffer} from which to extract the matches
+	 * @param sharedBuffer The {@link SharedBuffer} from which to extract the matches
 	 * @param computationState The end computation state of the extracted event sequences
 	 * @return Collection of event sequences which end in the given computation state
 	 */
-	Map<String, List<T>> extractCurrentMatches(
-		SharedBuffer<String, T> eventSharedBuffer, final ComputationState<T> computationState) {
-		if (computationState.getPreviousState() == null) {
+	private Map<String, List<T>> extractCurrentMatches(final SharedBuffer<T> sharedBuffer,
+			final ComputationState computationState) throws Exception {
+		if (computationState.getPreviousBufferEntry() == null) {
 			return new HashMap<>();
 		}
 
-		List<Map<String, List<T>>> paths = eventSharedBuffer.extractPatterns(
-				NFAStateNameHandler.getOriginalNameFromInternal(
-						computationState.getPreviousState()),
-				computationState.getEvent(),
-				computationState.getTimestamp(),
-				computationState.getCounter(),
+		List<Map<String, List<T>>> paths = sharedBuffer.extractPatterns(
+				computationState.getPreviousBufferEntry(),
 				computationState.getVersion());
 
 		if (paths.isEmpty()) {
@@ -740,16 +734,8 @@ public class NFA<T> {
 		for (String key: path.keySet()) {
 			List<T> events = path.get(key);
 
-			List<T> values = result.get(key);
-			if (values == null) {
-				values = new ArrayList<>(events.size());
-				result.put(key, values);
-			}
-
-			for (T event: events) {
-				// copy the element so that the user can change it
-				values.add(eventSerializer.isImmutableType() ? event : eventSerializer.copy(event));
-			}
+			List<T> values = result.computeIfAbsent(key, k -> new ArrayList<>(events.size()));
+			values.addAll(events);
 		}
 		return result;
 	}
@@ -757,7 +743,7 @@ public class NFA<T> {
 	/**
 	 * The context used when evaluating this computation state.
 	 */
-	public static class ConditionContext<T> implements IterativeCondition.Context<T> {
+	private static class ConditionContext<T> implements IterativeCondition.Context<T> {
 
 		/**
 		 * A flag indicating if we should recompute the matching pattern, so that
@@ -766,7 +752,7 @@ public class NFA<T> {
 		private boolean shouldUpdate;
 
 		/** The current computation state. */
-		private ComputationState<T> computationState;
+		private ComputationState computationState;
 
 		/**
 		 * The matched pattern so far. A condition will be evaluated over this
@@ -777,27 +763,27 @@ public class NFA<T> {
 
 		private NFA<T> nfa;
 
-		private SharedBuffer<String, T> eventSharedBuffer;
+		private SharedBuffer<T> sharedBuffer;
 
-		public ConditionContext(
-				NFA<T> nfa,
-				SharedBuffer<String, T> eventSharedBuffer,
-				ComputationState<T> computationState) {
+		ConditionContext(
+				final NFA<T> nfa,
+				final SharedBuffer<T> sharedBuffer,
+				final ComputationState computationState) {
 			this.computationState = computationState;
 			this.nfa = nfa;
-			this.eventSharedBuffer = eventSharedBuffer;
+			this.sharedBuffer = sharedBuffer;
 			this.shouldUpdate = true;
 		}
 
 		@Override
-		public Iterable<T> getEventsForPattern(final String key) {
+		public Iterable<T> getEventsForPattern(final String key) throws Exception {
 			Preconditions.checkNotNull(key);
 
 			// the (partially) matched pattern is computed lazily when this method is called.
 			// this is to avoid any overheads when using a simple, non-iterative condition.
 
 			if (shouldUpdate) {
-				this.matchedEvents = nfa.extractCurrentMatches(eventSharedBuffer, computationState);
+				this.matchedEvents = nfa.extractCurrentMatches(sharedBuffer, computationState);
 				shouldUpdate = false;
 			}
 
@@ -813,6 +799,33 @@ public class NFA<T> {
 		}
 	}
 
+	////////////////////				DEPRECATED/MIGRATION UTILS
+
+	/**
+	 * Wrapper for migrated state.
+	 */
+	public static class MigratedNFA<T> {
+
+		private final Queue<ComputationState> computationStates;
+		private final org.apache.flink.cep.nfa.SharedBuffer<T> sharedBuffer;
+
+		public org.apache.flink.cep.nfa.SharedBuffer<T> getSharedBuffer() {
+			return sharedBuffer;
+		}
+
+		public Queue<ComputationState> getComputationStates() {
+			return computationStates;
+		}
+
+		MigratedNFA(
+				final Queue<ComputationState> computationStates,
+				final org.apache.flink.cep.nfa.SharedBuffer<T> sharedBuffer
+		) {
+			this.sharedBuffer = sharedBuffer;
+			this.computationStates = computationStates;
+		}
+	}
+
 	/**
 	 * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state.
 	 */
@@ -826,7 +839,7 @@ public class NFA<T> {
 
 		public NFASerializerConfigSnapshot(
 			TypeSerializer<T> eventSerializer,
-			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
+			TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufferSerializer) {
 
 			super(eventSerializer, sharedBufferSerializer);
 		}
@@ -838,24 +851,27 @@ public class NFA<T> {
 	}
 
 	/**
-	 * Only for backward compatibility with <1.5.
+	 * Only for backward compatibility with <=1.5.
 	 */
 	@Deprecated
-	public static class NFASerializer<T> extends TypeSerializer<NFA<T>> {
+	public static class NFASerializer<T> extends TypeSerializer<MigratedNFA<T>> {
 
 		private static final long serialVersionUID = 2098282423980597010L;
 
-		private final TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer;
+		private final TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufferSerializer;
 
 		private final TypeSerializer<T> eventSerializer;
 
 		public NFASerializer(TypeSerializer<T> typeSerializer) {
-			this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer));
+			this(typeSerializer,
+				new org.apache.flink.cep.nfa.SharedBuffer.SharedBufferSerializer<>(
+					StringSerializer.INSTANCE,
+					typeSerializer));
 		}
 
 		NFASerializer(
-			TypeSerializer<T> typeSerializer,
-			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
+				TypeSerializer<T> typeSerializer,
+				TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufferSerializer) {
 			this.eventSerializer = typeSerializer;
 			this.sharedBufferSerializer = sharedBufferSerializer;
 		}
@@ -871,17 +887,17 @@ public class NFA<T> {
 		}
 
 		@Override
-		public NFA<T> createInstance() {
+		public MigratedNFA<T> createInstance() {
 			return null;
 		}
 
 		@Override
-		public NFA<T> copy(NFA<T> from) {
+		public MigratedNFA<T> copy(MigratedNFA<T> from) {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public NFA<T> copy(NFA<T> from, NFA<T> reuse) {
+		public MigratedNFA<T> copy(MigratedNFA<T> from, MigratedNFA<T> reuse) {
 			return copy(from);
 		}
 
@@ -891,52 +907,31 @@ public class NFA<T> {
 		}
 
 		@Override
-		public void serialize(NFA<T> record, DataOutputView target) throws IOException {
+		public void serialize(MigratedNFA<T> record, DataOutputView target) {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public NFA<T> deserialize(DataInputView source) throws IOException {
-			deserializeStates(source);
+		public MigratedNFA<T> deserialize(DataInputView source) throws IOException {
+			MigrationUtils.skipSerializedStates(source);
 			source.readLong();
 			source.readBoolean();
 
-			SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
-			Queue<ComputationState<T>> computationStates = NFASerializationUtils.deserializeComputationStates(eventSerializer, source);
+			org.apache.flink.cep.nfa.SharedBuffer<T> sharedBuffer = sharedBufferSerializer.deserialize(source);
+			Queue<ComputationState> computationStates = deserializeComputationStates(sharedBuffer, eventSerializer, source);
 
-			return new DummyNFA<>(eventSerializer, computationStates, sharedBuffer);
-		}
-
-		/**
-		 * Dummy nfa just for backwards compatibility.
-		 */
-		public static class DummyNFA<T> extends NFA<T> {
-
-			Queue<ComputationState<T>> computationStates;
-			SharedBuffer<String, T> sharedBuffer;
-
-			public SharedBuffer<String, T> getSharedBuffer() {
-				return sharedBuffer;
-			}
-
-			public Queue<ComputationState<T>> getComputationStates() {
-				return computationStates;
-			}
-
-			DummyNFA(TypeSerializer<T> eventSerializer, Queue<ComputationState<T>> computationStates, SharedBuffer<String, T> sharedBuffer) {
-				super(eventSerializer, 0, false, Collections.emptyList());
-				this.sharedBuffer = sharedBuffer;
-				this.computationStates = computationStates;
-			}
+			return new MigratedNFA<>(computationStates, sharedBuffer);
 		}
 
 		@Override
-		public NFA<T> deserialize(NFA<T> reuse, DataInputView source) throws IOException {
+		public MigratedNFA<T> deserialize(
+				MigratedNFA<T> reuse,
+				DataInputView source) throws IOException {
 			return deserialize(source);
 		}
 
 		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
+		public void copy(DataInputView source, DataOutputView target) {
 			throw new UnsupportedOperationException();
 		}
 
@@ -964,23 +959,23 @@ public class NFA<T> {
 		}
 
 		@Override
-		public CompatibilityResult<NFA<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public CompatibilityResult<MigratedNFA<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
 			if (configSnapshot instanceof NFASerializerConfigSnapshot) {
 				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
 					((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
 
 				CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						serializersAndConfigs.get(0).f0,
+					serializersAndConfigs.get(0).f0,
+					UnloadableDummyTypeSerializer.class,
+					serializersAndConfigs.get(0).f1,
+					eventSerializer);
+
+				CompatibilityResult<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufCompatResult =
+					CompatibilityUtil.resolveCompatibilityResult(
+						serializersAndConfigs.get(1).f0,
 						UnloadableDummyTypeSerializer.class,
-						serializersAndConfigs.get(0).f1,
-						eventSerializer);
-
-				CompatibilityResult<SharedBuffer<String, T>> sharedBufCompatResult =
-						CompatibilityUtil.resolveCompatibilityResult(
-								serializersAndConfigs.get(1).f0,
-								UnloadableDummyTypeSerializer.class,
-								serializersAndConfigs.get(1).f1,
-								sharedBufferSerializer);
+						serializersAndConfigs.get(1).f1,
+						sharedBufferSerializer);
 
 				if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
 					return CompatibilityResult.compatible();
@@ -997,70 +992,5 @@ public class NFA<T> {
 
 			return CompatibilityResult.requiresMigration();
 		}
-
-		private Set<State<T>> deserializeStates(DataInputView in) throws IOException {
-			TypeSerializer<String> nameSerializer = StringSerializer.INSTANCE;
-			TypeSerializer<State.StateType> stateTypeSerializer = new EnumSerializer<>(State.StateType.class);
-			TypeSerializer<StateTransitionAction> actionSerializer = new EnumSerializer<>(StateTransitionAction.class);
-
-			final int noOfStates = in.readInt();
-			Map<String, State<T>> states = new HashMap<>(noOfStates);
-
-			for (int i = 0; i < noOfStates; i++) {
-				String stateName = nameSerializer.deserialize(in);
-				State.StateType stateType = stateTypeSerializer.deserialize(in);
-
-				State<T> state = new State<>(stateName, stateType);
-				states.put(stateName, state);
-			}
-
-			for (int i = 0; i < noOfStates; i++) {
-				String srcName = nameSerializer.deserialize(in);
-
-				int noOfTransitions = in.readInt();
-				for (int j = 0; j < noOfTransitions; j++) {
-					String src = nameSerializer.deserialize(in);
-					Preconditions.checkState(src.equals(srcName),
-							"Source Edge names do not match (" + srcName + " - " + src + ").");
-
-					String trgt = nameSerializer.deserialize(in);
-					StateTransitionAction action = actionSerializer.deserialize(in);
-
-					IterativeCondition<T> condition = null;
-					try {
-						condition = deserializeCondition(in);
-					} catch (ClassNotFoundException e) {
-						e.printStackTrace();
-					}
-
-					State<T> srcState = states.get(src);
-					State<T> trgtState = states.get(trgt);
-					srcState.addStateTransition(action, trgtState, condition);
-				}
-
-			}
-			return new HashSet<>(states.values());
-		}
-
-		private IterativeCondition<T> deserializeCondition(DataInputView in) throws IOException, ClassNotFoundException {
-			boolean hasCondition = in.readBoolean();
-			if (hasCondition) {
-				int length = in.readInt();
-
-				byte[] serCondition = new byte[length];
-				in.read(serCondition);
-
-				ByteArrayInputStream bais = new ByteArrayInputStream(serCondition);
-				ObjectInputStream ois = new ObjectInputStream(bais);
-
-				IterativeCondition<T> condition = (IterativeCondition<T>) ois.readObject();
-				ois.close();
-				bais.close();
-
-				return condition;
-			}
-			return null;
-		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java
deleted file mode 100644
index 612eb25..0000000
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFASerializationUtils.java
+++ /dev/null
@@ -1,70 +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.nfa;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.core.memory.DataInputView;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.Queue;
-
-class NFASerializationUtils {
-
-	/**
-	 * Deserializes {@link Queue} of {@link ComputationState}s. The queue is represented as count of states then follows
-	 * n instances of the computational state.
-	 *
-	 * @param eventSerializer event serializer for deserializing accepted events
-	 * @param source          view on the serialized data
-	 * @param <T>             type of processed events
-	 * @return queue of computation states
-	 */
-	static <T> Queue<ComputationState<T>> deserializeComputationStates(TypeSerializer<T> eventSerializer,
-			DataInputView source) throws IOException {
-		Queue<ComputationState<T>> computationStates = new LinkedList<>();
-		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
-		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
-
-		int computationStateNo = source.readInt();
-		for (int i = 0; i < computationStateNo; i++) {
-			String state = stateNameSerializer.deserialize(source);
-			String prevState = stateNameSerializer.deserialize(source);
-			long timestamp = timestampSerializer.deserialize(source);
-			DeweyNumber version = versionSerializer.deserialize(source);
-			long startTimestamp = timestampSerializer.deserialize(source);
-			int counter = source.readInt();
-
-			T event = null;
-			if (source.readBoolean()) {
-				event = eventSerializer.deserialize(source);
-			}
-
-			computationStates.add(ComputationState.createState(
-				state, prevState, event, counter, timestamp, version, startTimestamp));
-		}
-		return computationStates;
-	}
-
-	private NFASerializationUtils() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
index e9414c3..a89c72e 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
@@ -18,83 +18,29 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator;
-
-import java.util.LinkedList;
+import java.util.Objects;
 import java.util.Queue;
 
 /**
  * State kept for a {@link NFA}.
- *
- * <p>The {@link AbstractKeyedCEPPatternOperator CEP operator}
- * keeps one NFA per key, for keyed input streams, and a single global NFA for non-keyed ones.
- * When an event gets processed, it updates the NFA's internal state machine.
- *
- * <p>An event that belongs to a partially matched sequence is kept in an internal
- * {@link SharedBuffer buffer}, which is a memory-optimized data-structure exactly for
- * this purpose. Events in the buffer are removed when all the matched sequences that
- * contain them are:
- * <ol>
- *  <li>emitted (success)</li>
- *  <li>discarded (patterns containing NOT)</li>
- *  <li>timed-out (windowed patterns)</li>
- * </ol>
- *
- * <p>The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams".
- *
- * @param <T> Type of the processed events
- * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">
- * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
  */
-public class NFAState<T> {
+public class NFAState {
 
 	/**
 	 * Current set of {@link ComputationState computation states} within the state machine.
 	 * These are the "active" intermediate states that are waiting for new matching
 	 * events to transition to new valid states.
 	 */
-	private Queue<ComputationState<T>> computationStates;
-
-	/**
-	 * Buffer used to store the matched events.
-	 */
-	private SharedBuffer<String, T> eventSharedBuffer;
+	private final Queue<ComputationState> computationStates;
 
 	/**
 	 * Flag indicating whether the matching status of the state machine has changed.
 	 */
 	private boolean stateChanged;
 
-	public NFAState(
-			Queue<ComputationState<T>> computationStates,
-			SharedBuffer<String, T> eventSharedBuffer,
-			boolean stateChanged) {
+	public NFAState(Queue<ComputationState> computationStates) {
 		this.computationStates = computationStates;
-		this.eventSharedBuffer = eventSharedBuffer;
-		this.stateChanged = stateChanged;
-	}
-
-	public NFAState() {
-		this(new LinkedList<>(), new SharedBuffer<>(), false);
-	}
-
-	public NFAState(Iterable<ComputationState<T>> startingStates) {
-		this();
-
-		for (ComputationState<T> startingState : startingStates) {
-			computationStates.add(startingState);
-		}
-	}
-
-	/**
-	 * Check if the NFA has finished processing all incoming data so far. That is
-	 * when the buffer keeping the matches is empty.
-	 *
-	 * @return {@code true} if there are no elements in the {@link SharedBuffer},
-	 * {@code false} otherwise.
-	 */
-	public boolean isEmpty() {
-		return eventSharedBuffer.isEmpty();
+		this.stateChanged = false;
 	}
 
 	/**
@@ -113,32 +59,39 @@ public class NFAState<T> {
 		this.stateChanged = false;
 	}
 
-	public void setStateChanged(boolean stateChanged) {
-		this.stateChanged = stateChanged;
+	/**
+	 * Set the changed bit checked via {@link #isStateChanged()} to {@code true}.
+	 */
+	public void setStateChanged() {
+		this.stateChanged = true;
 	}
 
-	public Queue<ComputationState<T>> getComputationStates() {
+	public Queue<ComputationState> getComputationStates() {
 		return computationStates;
 	}
 
-	public SharedBuffer<String, T> getEventSharedBuffer() {
-		return eventSharedBuffer;
-	}
-
 	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof NFAState) {
-			@SuppressWarnings("unchecked")
-			NFAState<T> other = (NFAState<T>) obj;
-
-			return eventSharedBuffer.equals(other.eventSharedBuffer);
-		} else {
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
 			return false;
 		}
+		NFAState nfaState = (NFAState) o;
+		return 	Objects.equals(computationStates, nfaState.computationStates);
 	}
 
 	@Override
 	public int hashCode() {
-		return eventSharedBuffer.hashCode();
+		return Objects.hash(computationStates, stateChanged);
+	}
+
+	@Override
+	public String toString() {
+		return "NFAState{" +
+			"computationStates=" + computationStates +
+			", stateChanged=" + stateChanged +
+			'}';
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
----------------------------------------------------------------------
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 5b057e1..bac144d 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
@@ -18,15 +18,11 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.cep.nfa.sharedbuffer.NodeId;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
@@ -35,30 +31,20 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.util.List;
+import java.util.LinkedList;
 import java.util.Queue;
 
 /**
  * A {@link TypeSerializer} for {@link NFAState} that uses Java Serialization.
  */
-public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
+public class NFAStateSerializer extends TypeSerializerSingleton<NFAState> {
 
 	private static final long serialVersionUID = 2098282423980597010L;
 
-	private final TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer;
-
-	private final TypeSerializer<T> eventSerializer;
-
-	public NFAStateSerializer(TypeSerializer<T> typeSerializer) {
-		this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer));
+	private NFAStateSerializer() {
 	}
 
-	public NFAStateSerializer(
-			TypeSerializer<T> typeSerializer,
-			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
-		this.eventSerializer = typeSerializer;
-		this.sharedBufferSerializer = sharedBufferSerializer;
-	}
+	public static final NFAStateSerializer INSTANCE = new NFAStateSerializer();
 
 	@Override
 	public boolean isImmutableType() {
@@ -66,17 +52,17 @@ public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
 	}
 
 	@Override
-	public NFAStateSerializer<T> duplicate() {
-		return new NFAStateSerializer<>(eventSerializer.duplicate());
+	public NFAStateSerializer duplicate() {
+		return new NFAStateSerializer();
 	}
 
 	@Override
-	public NFAState<T> createInstance() {
+	public NFAState createInstance() {
 		return null;
 	}
 
 	@Override
-	public NFAState<T> copy(NFAState<T> from) {
+	public NFAState copy(NFAState from) {
 		try {
 			ByteArrayOutputStream baos = new ByteArrayOutputStream();
 			serialize(from, new DataOutputViewStreamWrapper(baos));
@@ -85,7 +71,7 @@ public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
 			byte[] data = baos.toByteArray();
 
 			ByteArrayInputStream bais = new ByteArrayInputStream(data);
-			NFAState<T> copy = deserialize(new DataInputViewStreamWrapper(bais));
+			NFAState copy = deserialize(new DataInputViewStreamWrapper(bais));
 			bais.close();
 			return copy;
 		} catch (IOException e) {
@@ -94,7 +80,7 @@ public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
 	}
 
 	@Override
-	public NFAState<T> copy(NFAState<T> from, NFAState<T> reuse) {
+	public NFAState copy(NFAState from, NFAState reuse) {
 		return copy(from);
 	}
 
@@ -104,141 +90,57 @@ public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
 	}
 
 	@Override
-	public void serialize(NFAState<T> record, DataOutputView target) throws IOException {
-		sharedBufferSerializer.serialize(record.getEventSharedBuffer(), target);
+	public void serialize(NFAState record, DataOutputView target) throws IOException {
 
 		target.writeInt(record.getComputationStates().size());
 
 		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
 		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+		DeweyNumber.DeweyNumberSerializer versionSerializer = DeweyNumber.DeweyNumberSerializer.INSTANCE;
+		NodeId.NodeIdSerializer nodeIdSerializer = NodeId.NodeIdSerializer.INSTANCE;
 
-		for (ComputationState<T> computationState: record.getComputationStates()) {
-			stateNameSerializer.serialize(computationState.getState(), target);
-			stateNameSerializer.serialize(computationState.getPreviousState(), target);
+		for (ComputationState computationState : record.getComputationStates()) {
+			stateNameSerializer.serialize(computationState.getCurrentStateName(), target);
+			nodeIdSerializer.serialize(computationState.getPreviousBufferEntry(), target);
 
-			timestampSerializer.serialize(computationState.getTimestamp(), target);
 			versionSerializer.serialize(computationState.getVersion(), target);
 			timestampSerializer.serialize(computationState.getStartTimestamp(), target);
-			target.writeInt(computationState.getCounter());
-
-			if (computationState.getEvent() == null) {
-				target.writeBoolean(false);
-			} else {
-				target.writeBoolean(true);
-				eventSerializer.serialize(computationState.getEvent(), target);
-			}
 		}
 	}
 
 	@Override
-	public NFAState<T> deserialize(DataInputView source) throws IOException {
-		SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
-		Queue<ComputationState<T>> computationStates = NFASerializationUtils.deserializeComputationStates(
-			eventSerializer, source);
-		return new NFAState<>(computationStates, sharedBuffer, false);
-	}
-
-	@Override
-	public NFAState<T> deserialize(NFAState<T> reuse, DataInputView source) throws IOException {
-		return deserialize(source);
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
-		sharedBufferSerializer.serialize(sharedBuffer, target);
-
+	public NFAState deserialize(DataInputView source) throws IOException {
+		Queue<ComputationState> computationStates = new LinkedList<>();
 		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
 		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+		DeweyNumber.DeweyNumberSerializer versionSerializer = DeweyNumber.DeweyNumberSerializer.INSTANCE;
+		NodeId.NodeIdSerializer nodeIdSerializer = NodeId.NodeIdSerializer.INSTANCE;
 
 		int computationStateNo = source.readInt();
-		target.writeInt(computationStateNo);
-
 		for (int i = 0; i < computationStateNo; i++) {
-			String stateName = stateNameSerializer.deserialize(source);
-			stateNameSerializer.serialize(stateName, target);
-
-			String prevStateName = stateNameSerializer.deserialize(source);
-			stateNameSerializer.serialize(prevStateName, target);
-
-			long timestamp = timestampSerializer.deserialize(source);
-			timestampSerializer.serialize(timestamp, target);
-
+			String state = stateNameSerializer.deserialize(source);
+			NodeId prevState = nodeIdSerializer.deserialize(source);
 			DeweyNumber version = versionSerializer.deserialize(source);
-			versionSerializer.serialize(version, target);
-
 			long startTimestamp = timestampSerializer.deserialize(source);
-			timestampSerializer.serialize(startTimestamp, target);
-
-			int counter = source.readInt();
-			target.writeInt(counter);
 
-			boolean hasEvent = source.readBoolean();
-			target.writeBoolean(hasEvent);
-			if (hasEvent) {
-				T event = eventSerializer.deserialize(source);
-				eventSerializer.serialize(event, target);
-			}
+			computationStates.add(ComputationState.createState(state, prevState, version, startTimestamp));
 		}
+		return new NFAState(computationStates);
 	}
 
 	@Override
-	public boolean equals(Object obj) {
-		return obj == this ||
-				(obj != null && obj.getClass().equals(getClass()) &&
-						sharedBufferSerializer.equals(((NFAStateSerializer) obj).sharedBufferSerializer) &&
-						eventSerializer.equals(((NFAStateSerializer) obj).eventSerializer));
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return true;
+	public NFAState deserialize(NFAState reuse, DataInputView source) throws IOException {
+		return deserialize(source);
 	}
 
 	@Override
-	public int hashCode() {
-		return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode();
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		serialize(deserialize(source), target);
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
-		return new NFAStateSerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer);
+	public boolean canEqual(Object obj) {
+		return true;
 	}
 
-	@Override
-	public CompatibilityResult<NFAState<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof NFAStateSerializerConfigSnapshot) {
-			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
-				((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-			CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-				serializersAndConfigs.get(0).f0,
-				UnloadableDummyTypeSerializer.class,
-				serializersAndConfigs.get(0).f1,
-				eventSerializer);
-
-			CompatibilityResult<SharedBuffer<String, T>> sharedBufCompatResult =
-				CompatibilityUtil.resolveCompatibilityResult(
-					serializersAndConfigs.get(1).f0,
-					UnloadableDummyTypeSerializer.class,
-					serializersAndConfigs.get(1).f1,
-					sharedBufferSerializer);
-
-			if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else {
-				if (eventCompatResult.getConvertDeserializer() != null &&
-					sharedBufCompatResult.getConvertDeserializer() != null) {
-					return CompatibilityResult.requiresMigration(
-						new NFAStateSerializer<>(
-							new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()),
-							new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer())));
-				}
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9218df82/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java
deleted file mode 100644
index ab72348..0000000
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java
+++ /dev/null
@@ -1,45 +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.nfa;
-
-import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * {@link CompositeTypeSerializerConfigSnapshot} for {@link NFAStateSerializer}.
- */
-public final class NFAStateSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
-
-	private static final int VERSION = 1;
-
-	/** This empty constructor is required for deserializing the configuration. */
-	public NFAStateSerializerConfigSnapshot() {}
-
-	public NFAStateSerializerConfigSnapshot(
-			TypeSerializer<T> eventSerializer,
-			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
-
-		super(eventSerializer, sharedBufferSerializer);
-	}
-
-	@Override
-	public int getVersion() {
-		return VERSION;
-	}
-}


[03/10] flink git commit: [FLINK-8725] Separate state from NFA in CEP library

Posted by dw...@apache.org.
[FLINK-8725] Separate state from NFA in CEP library

This also changes the serialization of state to not include the static
NFA parts and to also not include any user code.


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

Branch: refs/heads/master
Commit: 55cd059a3bc03af36586d9328e6b821c047015b1
Parents: a00a398
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Feb 7 13:55:11 2018 +0100
Committer: Dawid Wysakowicz <dw...@apache.org>
Committed: Wed Jun 13 14:59:58 2018 +0200

----------------------------------------------------------------------
 .../apache/flink/cep/nfa/ComputationState.java  | 114 +---
 .../main/java/org/apache/flink/cep/nfa/NFA.java | 679 +++++--------------
 .../java/org/apache/flink/cep/nfa/NFAState.java | 144 ++++
 .../flink/cep/nfa/NFAStateSerializer.java       | 267 ++++++++
 .../nfa/NFAStateSerializerConfigSnapshot.java   |  45 ++
 .../org/apache/flink/cep/nfa/SharedBuffer.java  |   9 +-
 .../flink/cep/nfa/compiler/NFACompiler.java     |   7 +-
 .../AbstractKeyedCEPPatternOperator.java        |  67 +-
 .../pattern/conditions/IterativeCondition.java  |   2 +-
 .../org/apache/flink/cep/nfa/GroupITCase.java   |   6 +-
 .../org/apache/flink/cep/nfa/NFAITCase.java     |  66 +-
 .../flink/cep/nfa/NFAStatusChangeITCase.java    |  75 +-
 .../java/org/apache/flink/cep/nfa/NFATest.java  |  68 +-
 .../apache/flink/cep/nfa/NFATestUtilities.java  |  28 +-
 .../apache/flink/cep/nfa/SameElementITCase.java |  12 +-
 .../flink/cep/nfa/UntilConditionITCase.java     |  60 +-
 .../flink/cep/nfa/compiler/NFACompilerTest.java |   3 +-
 .../flink/cep/operator/CEPMigrationTest.java    |   2 +-
 .../flink/cep/operator/CEPOperatorTest.java     |  56 +-
 ...-migration-after-branching-flink1.4-snapshot | Bin 19058 -> 0 bytes
 ...-migration-after-branching-flink1.5-snapshot | Bin 0 -> 17433 bytes
 .../cep-migration-conditions-flink1.4-snapshot  | Bin 19503 -> 0 bytes
 .../cep-migration-conditions-flink1.5-snapshot  | Bin 0 -> 17117 bytes
 ...-single-pattern-afterwards-flink1.4-snapshot | Bin 16848 -> 0 bytes
 ...-single-pattern-afterwards-flink1.5-snapshot | Bin 0 -> 16850 bytes
 ...ation-starting-new-pattern-flink1.4-snapshot | Bin 18866 -> 0 bytes
 ...ation-starting-new-pattern-flink1.5-snapshot | Bin 0 -> 17241 bytes
 27 files changed, 906 insertions(+), 804 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
index 88ef3d3..c9aef27 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/ComputationState.java
@@ -18,13 +18,8 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.cep.pattern.conditions.IterativeCondition;
-import org.apache.flink.util.Preconditions;
+import javax.annotation.Nullable;
 
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
 import java.util.Objects;
 
 /**
@@ -36,7 +31,7 @@ import java.util.Objects;
  */
 public class ComputationState<T> {
 	// pointer to the NFA state of the computation
-	private final State<T> state;
+	private final String state;
 
 	// the last taken event
 	private final T event;
@@ -52,14 +47,12 @@ public class ComputationState<T> {
 	// Timestamp of the first element in the pattern
 	private final long startTimestamp;
 
-	private final State<T> previousState;
-
-	private final ConditionContext conditionContext;
+	@Nullable
+	private final String previousState;
 
 	private ComputationState(
-			final NFA<T> nfa,
-			final State<T> currentState,
-			final State<T> previousState,
+			final String currentState,
+			@Nullable final String previousState,
 			final T event,
 			final int counter,
 			final long timestamp,
@@ -72,25 +65,12 @@ public class ComputationState<T> {
 		this.version = version;
 		this.startTimestamp = startTimestamp;
 		this.previousState = previousState;
-		this.conditionContext = new ConditionContext(nfa, this);
 	}
 
 	public int getCounter() {
 		return counter;
 	}
 
-	public ConditionContext getConditionContext() {
-		return conditionContext;
-	}
-
-	public boolean isFinalState() {
-		return state.isFinal();
-	}
-
-	public boolean isStartState() {
-		return state.isStart() && event == null;
-	}
-
 	public long getTimestamp() {
 		return timestamp;
 	}
@@ -99,11 +79,12 @@ public class ComputationState<T> {
 		return startTimestamp;
 	}
 
-	public State<T> getState() {
+	public String getState() {
 		return state;
 	}
 
-	public State<T> getPreviousState() {
+	@Nullable
+	public String getPreviousState() {
 		return previousState;
 	}
 
@@ -137,85 +118,22 @@ public class ComputationState<T> {
 		return Objects.hash(state, event, counter, timestamp, version, startTimestamp, previousState);
 	}
 
-	public static <T> ComputationState<T> createStartState(final NFA<T> nfa, final State<T> state) {
-		Preconditions.checkArgument(state.isStart());
-		return new ComputationState<>(nfa, state, null, null, 0, -1L, new DeweyNumber(1), -1L);
+	public static <T> ComputationState<T> createStartState(final String state) {
+		return new ComputationState<>(state, null, null, 0, -1L, new DeweyNumber(1), -1L);
 	}
 
-	public static <T> ComputationState<T> createStartState(final NFA<T> nfa, final State<T> state, final DeweyNumber version) {
-		Preconditions.checkArgument(state.isStart());
-		return new ComputationState<>(nfa, state, null, null, 0, -1L, version, -1L);
+	public static <T> ComputationState<T> createStartState(final String state, final DeweyNumber version) {
+		return new ComputationState<T>(state, null, null, 0, -1L, version, -1L);
 	}
 
 	public static <T> ComputationState<T> createState(
-			final NFA<T> nfa,
-			final State<T> currentState,
-			final State<T> previousState,
+			final String currentState,
+			final String previousState,
 			final T event,
 			final int counter,
 			final long timestamp,
 			final DeweyNumber version,
 			final long startTimestamp) {
-		return new ComputationState<>(nfa, currentState, previousState, event, counter, timestamp, version, startTimestamp);
-	}
-
-	public boolean isStopState() {
-		return state.isStop();
-	}
-
-	/**
-	 * The context used when evaluating this computation state.
-	 */
-	public class ConditionContext implements IterativeCondition.Context<T> {
-
-		private static final long serialVersionUID = -6733978464782277795L;
-
-		/**
-		 * A flag indicating if we should recompute the matching pattern, so that
-		 * the {@link IterativeCondition iterative condition} can be evaluated.
-		 */
-		private boolean shouldUpdate;
-
-		/** The current computation state. */
-		private transient ComputationState<T> computationState;
-
-		/** The owning {@link NFA} of this computation state. */
-		private final NFA<T> nfa;
-
-		/**
-		 * The matched pattern so far. A condition will be evaluated over this
-		 * pattern. This is evaluated <b>only once</b>, as this is an expensive
-		 * operation that traverses a path in the {@link SharedBuffer}.
-		 */
-		private transient Map<String, List<T>> matchedEvents;
-
-		public ConditionContext(NFA<T> nfa, ComputationState<T> computationState) {
-			this.nfa = nfa;
-			this.computationState = computationState;
-			this.shouldUpdate = true;
-		}
-
-		@Override
-		public Iterable<T> getEventsForPattern(final String key) {
-			Preconditions.checkNotNull(key);
-
-			// the (partially) matched pattern is computed lazily when this method is called.
-			// this is to avoid any overheads when using a simple, non-iterative condition.
-
-			if (shouldUpdate) {
-				this.matchedEvents = nfa.extractCurrentMatches(computationState);
-				shouldUpdate = false;
-			}
-
-			return new Iterable<T>() {
-				@Override
-				public Iterator<T> iterator() {
-					List<T> elements = matchedEvents.get(key);
-					return elements == null
-							? Collections.EMPTY_LIST.<T>iterator()
-							: elements.iterator();
-				}
-			};
-		}
+		return new ComputationState<>(currentState, previousState, event, counter, timestamp, version, startTimestamp);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 5624db9..7324db3 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -18,44 +18,25 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.common.typeutils.base.EnumSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler;
 import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator;
 import org.apache.flink.cep.pattern.conditions.IterativeCondition;
-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.streaming.api.windowing.time.Time;
 import org.apache.flink.util.Preconditions;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Queue;
 import java.util.Set;
 import java.util.Stack;
@@ -85,16 +66,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">
  * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
  */
-public class NFA<T> implements Serializable {
-
-	private static final long serialVersionUID = 2957674889294717265L;
+public class NFA<T> {
 
 	/**
 	 * A set of all the valid NFA states, as returned by the
 	 * {@link NFACompiler NFACompiler}.
 	 * These are directly derived from the user-specified pattern.
 	 */
-	private Set<State<T>> states;
+	private final Map<String, State<T>> states;
 
 	/**
 	 * The length of a windowed pattern, as specified using the
@@ -109,82 +88,76 @@ public class NFA<T> implements Serializable {
 	 */
 	private final boolean handleTimeout;
 
-	/**
-	 * Current set of {@link ComputationState computation states} within the state machine.
-	 * These are the "active" intermediate states that are waiting for new matching
-	 * events to transition to new valid states.
-	 */
-	private transient Queue<ComputationState<T>> computationStates;
-
-	/**
-	 * Buffer used to store the matched events.
-	 */
-	private SharedBuffer<String, T> eventSharedBuffer;
+	private final TypeSerializer<T> eventSerializer;
 
-	private TypeSerializer<T> eventSerializer;
-
-	/**
-	 * Flag indicating whether the matching status of the state machine has changed.
-	 */
-	private boolean nfaChanged;
-
-	public NFA(final TypeSerializer<T> eventSerializer,
+	public NFA(
+			final TypeSerializer<T> eventSerializer,
 			final long windowTime,
-			final boolean handleTimeout) {
+			final boolean handleTimeout,
+			final Collection<State<T>> states) {
 		this.eventSerializer = checkNotNull(eventSerializer);
 		this.windowTime = windowTime;
 		this.handleTimeout = handleTimeout;
-		this.eventSharedBuffer = new SharedBuffer<>();
-		this.computationStates = new LinkedList<>();
-		this.states = new HashSet<>();
-		this.nfaChanged = false;
+
+		this.states = new HashMap<>();
+		for (State<T> state : states) {
+			this.states.put(state.getName(), state);
+		}
 	}
 
-	public Set<State<T>> getStates() {
-		return states;
+	@VisibleForTesting
+	public Collection<State<T>> getStates() {
+		return states.values();
 	}
 
-	public void addStates(final Collection<State<T>> newStates) {
-		for (State<T> state: newStates) {
-			addState(state);
+	public NFAState<T> createNFAState() {
+		List<ComputationState<T>> startingStates = new ArrayList<>();
+		for (State<T> state : states.values()) {
+			if (state.isStart()) {
+				startingStates.add(ComputationState.createStartState(state.getName()));
+			}
 		}
+		return new NFAState<>(startingStates);
 	}
 
-	public void addState(final State<T> state) {
-		states.add(state);
+	private State<T> getState(String state) {
+		State<T> result = states.get(state);
+		return result;
+	}
 
-		if (state.isStart()) {
-			computationStates.add(ComputationState.createStartState(this, state));
-		}
+	private State<T> getState(ComputationState<T> state) {
+		State<T> result = states.get(state.getState());
+		return result;
 	}
 
-	/**
-	 * Check if the NFA has finished processing all incoming data so far. That is
-	 * when the buffer keeping the matches is empty.
-	 *
-	 * @return {@code true} if there are no elements in the {@link SharedBuffer},
-	 * {@code false} otherwise.
-	 */
-	public boolean isEmpty() {
-		return eventSharedBuffer.isEmpty();
+	private boolean isStartState(ComputationState<T> state) {
+		State<T> stateObject = getState(state);
+		if (stateObject == null) {
+			throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values());
+		}
+
+		return stateObject.isStart();
 	}
 
-	/**
-	 * Check if the matching status of the NFA has changed so far.
-	 *
-	 * @return {@code true} if matching status has changed, {@code false} otherwise
-	 */
-	public boolean isNFAChanged() {
-		return nfaChanged;
+	private boolean isStopState(ComputationState<T> state) {
+		State<T> stateObject = getState(state);
+		if (stateObject == null) {
+			throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values());
+		}
+
+		return stateObject.isStop();
 	}
 
-	/**
-	 * Reset {@link #nfaChanged} to {@code false}.
-	 */
-	public void resetNFAChanged() {
-		this.nfaChanged = false;
+	private boolean isFinalState(ComputationState<T> state) {
+		State<T> stateObject = getState(state);
+		if (stateObject == null) {
+			throw new RuntimeException("State " + state.getState() + " does not exist in the NFA. NFA has states " + states.values());
+		}
+
+		return stateObject.isFinal();
 	}
 
+
 	/**
 	 * Processes the next input event. If some of the computations reach a final state then the
 	 * resulting event sequences are returned. If computations time out and timeout handling is
@@ -193,14 +166,16 @@ public class NFA<T> implements Serializable {
 	 * <p>If computations reach a stop state, the path forward is discarded and currently constructed path is returned
 	 * with the element that resulted in the stop state.
 	 *
+	 * @param nfaState The NFAState object that we need to affect while processing
 	 * @param event The current event to be processed or null if only pruning shall be done
 	 * @param timestamp The timestamp of the current event
 	 * @return Tuple of the collection of matched patterns (e.g. the result of computations which have
 	 * reached a final state) and the collection of timed out patterns (if timeout handling is
 	 * activated)
 	 */
-	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(final T event, final long timestamp) {
-		return process(event, timestamp, AfterMatchSkipStrategy.noSkip());
+	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(
+		NFAState<T> nfaState, final T event, final long timestamp) {
+		return process(nfaState, event, timestamp, AfterMatchSkipStrategy.noSkip());
 	}
 
 	/**
@@ -211,6 +186,7 @@ public class NFA<T> implements Serializable {
 	 * <p>If computations reach a stop state, the path forward is discarded and currently constructed path is returned
 	 * with the element that resulted in the stop state.
 	 *
+	 * @param nfaState The NFAState object that we need to affect while processing
 	 * @param event The current event to be processed or null if only pruning shall be done
 	 * @param timestamp The timestamp of the current event
 	 * @param afterMatchSkipStrategy The skip strategy to use after per match
@@ -218,8 +194,15 @@ public class NFA<T> implements Serializable {
 	 * reached a final state) and the collection of timed out patterns (if timeout handling is
 	 * activated)
 	 */
-	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(final T event,
-		final long timestamp, AfterMatchSkipStrategy afterMatchSkipStrategy) {
+	public Tuple2<Collection<Map<String, List<T>>>, Collection<Tuple2<Map<String, List<T>>, Long>>> process(
+			NFAState<T> nfaState,
+			final T event,
+			final long timestamp,
+			AfterMatchSkipStrategy afterMatchSkipStrategy) {
+
+		Queue<ComputationState<T>> computationStates = nfaState.getComputationStates();
+		SharedBuffer<String, T> eventSharedBuffer = nfaState.getEventSharedBuffer();
+
 		final int numberComputationStates = computationStates.size();
 		final Collection<Map<String, List<T>>> result = new ArrayList<>();
 		final Collection<Tuple2<Map<String, List<T>>, Long>> timeoutResult = new ArrayList<>();
@@ -230,31 +213,31 @@ public class NFA<T> implements Serializable {
 
 			final Collection<ComputationState<T>> newComputationStates;
 
-			if (!computationState.isStartState() &&
+			if (!isStartState(computationState) &&
 				windowTime > 0L &&
 				timestamp - computationState.getStartTimestamp() >= windowTime) {
 
 				if (handleTimeout) {
 					// extract the timed out event pattern
-					Map<String, List<T>> timedOutPattern = extractCurrentMatches(computationState);
+					Map<String, List<T>> timedOutPattern = extractCurrentMatches(eventSharedBuffer, computationState);
 					timeoutResult.add(Tuple2.of(timedOutPattern, timestamp));
 				}
 
 				eventSharedBuffer.release(
-						NFAStateNameHandler.getOriginalNameFromInternal(computationState.getPreviousState().getName()),
+						NFAStateNameHandler.getOriginalNameFromInternal(computationState.getPreviousState()),
 						computationState.getEvent(),
 						computationState.getTimestamp(),
 						computationState.getCounter());
 
 				newComputationStates = Collections.emptyList();
-				nfaChanged = true;
+				nfaState.setStateChanged(true);
 			} else if (event != null) {
-				newComputationStates = computeNextStates(computationState, event, timestamp);
+				newComputationStates = computeNextStates(eventSharedBuffer, computationState, event, timestamp);
 
 				if (newComputationStates.size() != 1) {
-					nfaChanged = true;
+					nfaState.setStateChanged(true);
 				} else if (!newComputationStates.iterator().next().equals(computationState)) {
-					nfaChanged = true;
+					nfaState.setStateChanged(true);
 				}
 			} else {
 				newComputationStates = Collections.singleton(computationState);
@@ -266,24 +249,24 @@ public class NFA<T> implements Serializable {
 			boolean shouldDiscardPath = false;
 			for (final ComputationState<T> newComputationState: newComputationStates) {
 
-				if (newComputationState.isFinalState()) {
+				if (isFinalState(newComputationState)) {
 					// we've reached a final state and can thus retrieve the matching event sequence
-					Map<String, List<T>> matchedPattern = extractCurrentMatches(newComputationState);
+					Map<String, List<T>> matchedPattern = extractCurrentMatches(eventSharedBuffer, newComputationState);
 					result.add(matchedPattern);
 
 					// remove found patterns because they are no longer needed
 					eventSharedBuffer.release(
 							NFAStateNameHandler.getOriginalNameFromInternal(
-									newComputationState.getPreviousState().getName()),
+									newComputationState.getPreviousState()),
 							newComputationState.getEvent(),
 							newComputationState.getTimestamp(),
 							newComputationState.getCounter());
-				} else if (newComputationState.isStopState()) {
+				} else if (isStopState(newComputationState)) {
 					//reached stop state. release entry for the stop state
 					shouldDiscardPath = true;
 					eventSharedBuffer.release(
 							NFAStateNameHandler.getOriginalNameFromInternal(
-									newComputationState.getPreviousState().getName()),
+									newComputationState.getPreviousState()),
 							newComputationState.getEvent(),
 							newComputationState.getTimestamp(),
 							newComputationState.getCounter());
@@ -299,7 +282,7 @@ public class NFA<T> implements Serializable {
 				for (final ComputationState<T> state : statesToRetain) {
 					eventSharedBuffer.release(
 							NFAStateNameHandler.getOriginalNameFromInternal(
-									state.getPreviousState().getName()),
+									state.getPreviousState()),
 							state.getEvent(),
 							state.getTimestamp(),
 							state.getCounter());
@@ -310,7 +293,8 @@ public class NFA<T> implements Serializable {
 
 		}
 
-		discardComputationStatesAccordingToStrategy(computationStates, result, afterMatchSkipStrategy);
+		discardComputationStatesAccordingToStrategy(
+			eventSharedBuffer, computationStates, result, afterMatchSkipStrategy);
 
 		// prune shared buffer based on window length
 		if (windowTime > 0L) {
@@ -322,7 +306,7 @@ public class NFA<T> implements Serializable {
 				// remove all elements which are expired
 				// with respect to the window length
 				if (eventSharedBuffer.prune(pruningTimestamp)) {
-					nfaChanged = true;
+					nfaState.setStateChanged(true);
 				}
 			}
 		}
@@ -330,7 +314,9 @@ public class NFA<T> implements Serializable {
 		return Tuple2.of(result, timeoutResult);
 	}
 
-	private void discardComputationStatesAccordingToStrategy(Queue<ComputationState<T>> computationStates,
+	private void discardComputationStatesAccordingToStrategy(
+		SharedBuffer<String, T> eventSharedBuffer,
+		Queue<ComputationState<T>> computationStates,
 		Collection<Map<String, List<T>>> matchedResult, AfterMatchSkipStrategy afterMatchSkipStrategy) {
 		Set<T> discardEvents = new HashSet<>();
 		switch(afterMatchSkipStrategy.getStrategy()) {
@@ -368,14 +354,14 @@ public class NFA<T> implements Serializable {
 		if (!discardEvents.isEmpty()) {
 			List<ComputationState<T>> discardStates = new ArrayList<>();
 			for (ComputationState<T> computationState : computationStates) {
-				Map<String, List<T>> partialMatch = extractCurrentMatches(computationState);
+				Map<String, List<T>> partialMatch = extractCurrentMatches(eventSharedBuffer, computationState);
 				for (List<T> list: partialMatch.values()) {
 					for (T e: list) {
 						if (discardEvents.contains(e)) {
 							// discard the computation state.
 							eventSharedBuffer.release(
 								NFAStateNameHandler.getOriginalNameFromInternal(
-									computationState.getState().getName()),
+									computationState.getState()),
 								computationState.getEvent(),
 								computationState.getTimestamp(),
 								computationState.getCounter()
@@ -390,25 +376,6 @@ public class NFA<T> implements Serializable {
 		}
 	}
 
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof NFA) {
-			@SuppressWarnings("unchecked")
-			NFA<T> other = (NFA<T>) obj;
-
-			return eventSharedBuffer.equals(other.eventSharedBuffer) &&
-				states.equals(other.states) &&
-				windowTime == other.windowTime;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return Objects.hash(eventSharedBuffer, states, windowTime);
-	}
-
 	private static <T> boolean isEquivalentState(final State<T> s1, final State<T> s2) {
 		return s1.getName().equals(s2.getName());
 	}
@@ -487,17 +454,19 @@ public class NFA<T> implements Serializable {
 	 *     <li>Release the corresponding entries in {@link SharedBuffer}.</li>
 	 *</ol>
 	 *
+	 * @param eventSharedBuffer The shared buffer that we need to change
 	 * @param computationState Current computation state
 	 * @param event Current event which is processed
 	 * @param timestamp Timestamp of the current event
 	 * @return Collection of computation states which result from the current one
 	 */
 	private Collection<ComputationState<T>> computeNextStates(
+			final SharedBuffer<String, T> eventSharedBuffer,
 			final ComputationState<T> computationState,
 			final T event,
 			final long timestamp) {
 
-		final OutgoingEdges<T> outgoingEdges = createDecisionGraph(computationState, event);
+		final OutgoingEdges<T> outgoingEdges = createDecisionGraph(eventSharedBuffer, computationState, event);
 
 		// Create the computing version based on the previously computed edges
 		// We need to defer the creation of computation states until we know how many edges start
@@ -511,9 +480,9 @@ public class NFA<T> implements Serializable {
 		for (StateTransition<T> edge : edges) {
 			switch (edge.getAction()) {
 				case IGNORE: {
-					if (!computationState.isStartState()) {
+					if (!isStartState(computationState)) {
 						final DeweyNumber version;
-						if (isEquivalentState(edge.getTargetState(), computationState.getState())) {
+						if (isEquivalentState(edge.getTargetState(), getState(computationState))) {
 							//Stay in the same state (it can be either looping one or singleton)
 							final int toIncrease = calculateIncreasingSelfState(
 								outgoingEdges.getTotalIgnoreBranches(),
@@ -528,9 +497,10 @@ public class NFA<T> implements Serializable {
 						}
 
 						addComputationState(
+								eventSharedBuffer,
 								resultingComputationStates,
 								edge.getTargetState(),
-								computationState.getPreviousState(),
+								getState(computationState.getPreviousState()),
 								computationState.getEvent(),
 								computationState.getCounter(),
 								computationState.getTimestamp(),
@@ -543,7 +513,7 @@ public class NFA<T> implements Serializable {
 				case TAKE:
 					final State<T> nextState = edge.getTargetState();
 					final State<T> currentState = edge.getSourceState();
-					final State<T> previousState = computationState.getPreviousState();
+					final State<T> previousState = getState(computationState.getPreviousState());
 
 					final T previousEvent = computationState.getEvent();
 
@@ -553,7 +523,7 @@ public class NFA<T> implements Serializable {
 
 					final int counter;
 					final long startTimestamp;
-					if (computationState.isStartState()) {
+					if (isStartState(computationState)) {
 						startTimestamp = timestamp;
 						counter = eventSharedBuffer.put(
 							NFAStateNameHandler.getOriginalNameFromInternal(
@@ -577,6 +547,7 @@ public class NFA<T> implements Serializable {
 					}
 
 					addComputationState(
+							eventSharedBuffer,
 							resultingComputationStates,
 							nextState,
 							currentState,
@@ -587,9 +558,10 @@ public class NFA<T> implements Serializable {
 							startTimestamp);
 
 					//check if newly created state is optional (have a PROCEED path to Final state)
-					final State<T> finalState = findFinalStateAfterProceed(nextState, event, computationState);
+					final State<T> finalState = findFinalStateAfterProceed(eventSharedBuffer, nextState, event, computationState);
 					if (finalState != null) {
 						addComputationState(
+								eventSharedBuffer,
 								resultingComputationStates,
 								finalState,
 								currentState,
@@ -603,13 +575,13 @@ public class NFA<T> implements Serializable {
 			}
 		}
 
-		if (computationState.isStartState()) {
+		if (isStartState(computationState)) {
 			int totalBranches = calculateIncreasingSelfState(
 					outgoingEdges.getTotalIgnoreBranches(),
 					outgoingEdges.getTotalTakeBranches());
 
 			DeweyNumber startVersion = computationState.getVersion().increase(totalBranches);
-			ComputationState<T> startState = ComputationState.createStartState(this, computationState.getState(), startVersion);
+			ComputationState<T> startState = ComputationState.createStartState(computationState.getState(), startVersion);
 			resultingComputationStates.add(startState);
 		}
 
@@ -617,7 +589,7 @@ public class NFA<T> implements Serializable {
 			// release the shared entry referenced by the current computation state.
 			eventSharedBuffer.release(
 					NFAStateNameHandler.getOriginalNameFromInternal(
-							computationState.getPreviousState().getName()),
+							computationState.getPreviousState()),
 					computationState.getEvent(),
 					computationState.getTimestamp(),
 					computationState.getCounter());
@@ -627,6 +599,7 @@ public class NFA<T> implements Serializable {
 	}
 
 	private void addComputationState(
+			SharedBuffer<String, T> eventSharedBuffer,
 			List<ComputationState<T>> computationStates,
 			State<T> currentState,
 			State<T> previousState,
@@ -636,14 +609,17 @@ public class NFA<T> implements Serializable {
 			DeweyNumber version,
 			long startTimestamp) {
 		ComputationState<T> computationState = ComputationState.createState(
-				this, currentState, previousState, event, counter, timestamp, version, startTimestamp);
+				currentState.getName(), previousState.getName(), event, counter, timestamp, version, startTimestamp);
 		computationStates.add(computationState);
 
 		String originalStateName = NFAStateNameHandler.getOriginalNameFromInternal(previousState.getName());
 		eventSharedBuffer.lock(originalStateName, event, timestamp, counter);
 	}
 
-	private State<T> findFinalStateAfterProceed(State<T> state, T event, ComputationState<T> computationState) {
+	private State<T> findFinalStateAfterProceed(
+		SharedBuffer<String, T> eventSharedBuffer,
+		State<T> state, T event,
+		ComputationState<T> computationState) {
 		final Stack<State<T>> statesToCheck = new Stack<>();
 		statesToCheck.push(state);
 
@@ -652,7 +628,7 @@ public class NFA<T> implements Serializable {
 				final State<T> currentState = statesToCheck.pop();
 				for (StateTransition<T> transition : currentState.getStateTransitions()) {
 					if (transition.getAction() == StateTransitionAction.PROCEED &&
-							checkFilterCondition(computationState, transition.getCondition(), event)) {
+							checkFilterCondition(eventSharedBuffer, computationState, transition.getCondition(), event)) {
 						if (transition.getTargetState().isFinal()) {
 							return transition.getTargetState();
 						} else {
@@ -672,11 +648,14 @@ public class NFA<T> implements Serializable {
 		return takeBranches == 0 && ignoreBranches == 0 ? 0 : ignoreBranches + Math.max(1, takeBranches);
 	}
 
-	private OutgoingEdges<T> createDecisionGraph(ComputationState<T> computationState, T event) {
-		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(computationState.getState());
+	private OutgoingEdges<T> createDecisionGraph(
+		SharedBuffer<String, T> eventSharedBuffer,
+		ComputationState<T> computationState, T event) {
+		State<T> state = getState(computationState);
+		final OutgoingEdges<T> outgoingEdges = new OutgoingEdges<>(state);
 
 		final Stack<State<T>> states = new Stack<>();
-		states.push(computationState.getState());
+		states.push(state);
 
 		//First create all outgoing edges, so to be able to reason about the Dewey version
 		while (!states.isEmpty()) {
@@ -686,7 +665,7 @@ public class NFA<T> implements Serializable {
 			// check all state transitions for each state
 			for (StateTransition<T> stateTransition : stateTransitions) {
 				try {
-					if (checkFilterCondition(computationState, stateTransition.getCondition(), event)) {
+					if (checkFilterCondition(eventSharedBuffer, computationState, stateTransition.getCondition(), event)) {
 						// filter condition is true
 						switch (stateTransition.getAction()) {
 							case PROCEED:
@@ -708,8 +687,12 @@ public class NFA<T> implements Serializable {
 		return outgoingEdges;
 	}
 
-	private boolean checkFilterCondition(ComputationState<T> computationState, IterativeCondition<T> condition, T event) throws Exception {
-		return condition == null || condition.filter(event, computationState.getConditionContext());
+	private boolean checkFilterCondition(
+		SharedBuffer<String, T> eventSharedBuffer,
+		ComputationState<T> computationState,
+		IterativeCondition<T> condition,
+		T event) throws Exception {
+		return condition == null || condition.filter(event, new ConditionContext<>(this, eventSharedBuffer, computationState));
 	}
 
 	/**
@@ -717,17 +700,19 @@ public class NFA<T> implements Serializable {
 	 * sequence is returned as a map which contains the events and the names of the states to which
 	 * the events were mapped.
 	 *
+	 * @param eventSharedBuffer The {@link SharedBuffer} from which to extract the matches
 	 * @param computationState The end computation state of the extracted event sequences
 	 * @return Collection of event sequences which end in the given computation state
 	 */
-	Map<String, List<T>> extractCurrentMatches(final ComputationState<T> computationState) {
+	Map<String, List<T>> extractCurrentMatches(
+		SharedBuffer<String, T> eventSharedBuffer, final ComputationState<T> computationState) {
 		if (computationState.getPreviousState() == null) {
 			return new HashMap<>();
 		}
 
 		List<Map<String, List<T>>> paths = eventSharedBuffer.extractPatterns(
 				NFAStateNameHandler.getOriginalNameFromInternal(
-						computationState.getPreviousState().getName()),
+						computationState.getPreviousState()),
 				computationState.getEvent(),
 				computationState.getTimestamp(),
 				computationState.getCounter(),
@@ -758,392 +743,62 @@ public class NFA<T> implements Serializable {
 		return result;
 	}
 
-	//////////////////////			Serialization			//////////////////////
-
-	/**
-	 * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state.
-	 */
-	public static final class NFASerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
-
-		private static final int VERSION = 1;
-
-		/** This empty constructor is required for deserializing the configuration. */
-		public NFASerializerConfigSnapshot() {}
-
-		public NFASerializerConfigSnapshot(
-				TypeSerializer<T> eventSerializer,
-				TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
-
-			super(eventSerializer, sharedBufferSerializer);
-		}
-
-		@Override
-		public int getVersion() {
-			return VERSION;
-		}
-	}
-
 	/**
-	 * A {@link TypeSerializer} for {@link NFA} that uses Java Serialization.
+	 * The context used when evaluating this computation state.
 	 */
-	public static class NFASerializer<T> extends TypeSerializer<NFA<T>> {
-
-		private static final long serialVersionUID = 2098282423980597010L;
-
-		private final TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer;
-
-		private final TypeSerializer<T> eventSerializer;
-
-		public NFASerializer(TypeSerializer<T> typeSerializer) {
-			this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer));
-		}
-
-		public NFASerializer(
-				TypeSerializer<T> typeSerializer,
-				TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
-			this.eventSerializer = typeSerializer;
-			this.sharedBufferSerializer = sharedBufferSerializer;
-		}
-
-		@Override
-		public boolean isImmutableType() {
-			return false;
-		}
-
-		@Override
-		public NFASerializer<T> duplicate() {
-			return new NFASerializer<>(eventSerializer.duplicate());
-		}
-
-		@Override
-		public NFA<T> createInstance() {
-			return null;
-		}
-
-		@Override
-		public NFA<T> copy(NFA<T> from) {
-			try {
-				ByteArrayOutputStream baos = new ByteArrayOutputStream();
-				serialize(from, new DataOutputViewStreamWrapper(baos));
-				baos.close();
-
-				byte[] data = baos.toByteArray();
-
-				ByteArrayInputStream bais = new ByteArrayInputStream(data);
-				NFA<T> copy = deserialize(new DataInputViewStreamWrapper(bais));
-				bais.close();
-				return copy;
-			} catch (IOException e) {
-				throw new RuntimeException("Could not copy NFA.", e);
-			}
-		}
-
-		@Override
-		public NFA<T> copy(NFA<T> from, NFA<T> reuse) {
-			return copy(from);
-		}
-
-		@Override
-		public int getLength() {
-			return -1;
-		}
-
-		@Override
-		public void serialize(NFA<T> record, DataOutputView target) throws IOException {
-			serializeStates(record.states, target);
-			target.writeLong(record.windowTime);
-			target.writeBoolean(record.handleTimeout);
-
-			sharedBufferSerializer.serialize(record.eventSharedBuffer, target);
+	public static class ConditionContext<T> implements IterativeCondition.Context<T> {
 
-			target.writeInt(record.computationStates.size());
+		/**
+		 * A flag indicating if we should recompute the matching pattern, so that
+		 * the {@link IterativeCondition iterative condition} can be evaluated.
+		 */
+		private boolean shouldUpdate;
 
-			StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
-			LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-			DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+		/** The current computation state. */
+		private ComputationState<T> computationState;
 
-			for (ComputationState<T> computationState: record.computationStates) {
-				stateNameSerializer.serialize(computationState.getState().getName(), target);
-				stateNameSerializer.serialize(computationState.getPreviousState() == null
-						? null : computationState.getPreviousState().getName(), target);
+		/**
+		 * The matched pattern so far. A condition will be evaluated over this
+		 * pattern. This is evaluated <b>only once</b>, as this is an expensive
+		 * operation that traverses a path in the {@link SharedBuffer}.
+		 */
+		private Map<String, List<T>> matchedEvents;
 
-				timestampSerializer.serialize(computationState.getTimestamp(), target);
-				versionSerializer.serialize(computationState.getVersion(), target);
-				timestampSerializer.serialize(computationState.getStartTimestamp(), target);
-				target.writeInt(computationState.getCounter());
+		private NFA<T> nfa;
 
-				if (computationState.getEvent() == null) {
-					target.writeBoolean(false);
-				} else {
-					target.writeBoolean(true);
-					eventSerializer.serialize(computationState.getEvent(), target);
-				}
-			}
-		}
-
-		@Override
-		public NFA<T> deserialize(DataInputView source) throws IOException {
-			Set<State<T>> states = deserializeStates(source);
-			long windowTime = source.readLong();
-			boolean handleTimeout = source.readBoolean();
-
-			NFA<T> nfa = new NFA<>(eventSerializer, windowTime, handleTimeout);
-			nfa.states = states;
-
-			nfa.eventSharedBuffer = sharedBufferSerializer.deserialize(source);
-
-			Queue<ComputationState<T>> computationStates = new LinkedList<>();
-			StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
-			LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-			DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
-
-			int computationStateNo = source.readInt();
-			for (int i = 0; i < computationStateNo; i++) {
-				State<T> state = getStateByName(stateNameSerializer.deserialize(source), nfa);
-				State<T> prevState = getStateByName(stateNameSerializer.deserialize(source), nfa);
-				long timestamp = timestampSerializer.deserialize(source);
-				DeweyNumber version = versionSerializer.deserialize(source);
-				long startTimestamp = timestampSerializer.deserialize(source);
-				int counter = source.readInt();
-
-				T event = null;
-				if (source.readBoolean()) {
-					event = eventSerializer.deserialize(source);
-				}
-
-				computationStates.add(ComputationState.createState(
-						nfa, state, prevState, event, counter, timestamp, version, startTimestamp));
-			}
-
-			nfa.computationStates = computationStates;
-			return nfa;
-		}
-
-		private State<T> getStateByName(String name, NFA<T> nfa) {
-			for (State<T> state: nfa.states) {
-				if (state.getName().equals(name)) {
-					return state;
-				}
-			}
-			return null;
-		}
-
-		@Override
-		public NFA<T> deserialize(NFA<T> reuse, DataInputView source) throws IOException {
-			return deserialize(source);
-		}
-
-		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			Set<State<T>> states = deserializeStates(source);
-			serializeStates(states, target);
-
-			long windowTime = source.readLong();
-			target.writeLong(windowTime);
-
-			boolean handleTimeout = source.readBoolean();
-			target.writeBoolean(handleTimeout);
-
-			SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
-			sharedBufferSerializer.serialize(sharedBuffer, target);
-
-			StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
-			LongSerializer timestampSerializer = LongSerializer.INSTANCE;
-			DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
-
-			int computationStateNo = source.readInt();
-			target.writeInt(computationStateNo);
-
-			for (int i = 0; i < computationStateNo; i++) {
-				String stateName = stateNameSerializer.deserialize(source);
-				stateNameSerializer.serialize(stateName, target);
-
-				String prevStateName = stateNameSerializer.deserialize(source);
-				stateNameSerializer.serialize(prevStateName, target);
-
-				long timestamp = timestampSerializer.deserialize(source);
-				timestampSerializer.serialize(timestamp, target);
-
-				DeweyNumber version = versionSerializer.deserialize(source);
-				versionSerializer.serialize(version, target);
-
-				long startTimestamp = timestampSerializer.deserialize(source);
-				timestampSerializer.serialize(startTimestamp, target);
-
-				int counter = source.readInt();
-				target.writeInt(counter);
-
-				boolean hasEvent = source.readBoolean();
-				target.writeBoolean(hasEvent);
-				if (hasEvent) {
-					T event = eventSerializer.deserialize(source);
-					eventSerializer.serialize(event, target);
-				}
-			}
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			return obj == this ||
-					(obj != null && obj.getClass().equals(getClass()) &&
-							sharedBufferSerializer.equals(((NFASerializer) obj).sharedBufferSerializer) &&
-							eventSerializer.equals(((NFASerializer) obj).eventSerializer));
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return true;
-		}
+		private SharedBuffer<String, T> eventSharedBuffer;
 
-		@Override
-		public int hashCode() {
-			return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode();
+		public ConditionContext(
+				NFA<T> nfa,
+				SharedBuffer<String, T> eventSharedBuffer,
+				ComputationState<T> computationState) {
+			this.computationState = computationState;
+			this.nfa = nfa;
+			this.eventSharedBuffer = eventSharedBuffer;
+			this.shouldUpdate = true;
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new NFASerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer);
-		}
+		public Iterable<T> getEventsForPattern(final String key) {
+			Preconditions.checkNotNull(key);
 
-		@Override
-		public CompatibilityResult<NFA<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			if (configSnapshot instanceof NFASerializerConfigSnapshot) {
-				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
-						((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-				CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						serializersAndConfigs.get(0).f0,
-						UnloadableDummyTypeSerializer.class,
-						serializersAndConfigs.get(0).f1,
-						eventSerializer);
-
-				CompatibilityResult<SharedBuffer<String, T>> sharedBufCompatResult =
-						CompatibilityUtil.resolveCompatibilityResult(
-								serializersAndConfigs.get(1).f0,
-								UnloadableDummyTypeSerializer.class,
-								serializersAndConfigs.get(1).f1,
-								sharedBufferSerializer);
-
-				if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else {
-					if (eventCompatResult.getConvertDeserializer() != null &&
-						sharedBufCompatResult.getConvertDeserializer() != null) {
-						return CompatibilityResult.requiresMigration(
-							new NFASerializer<>(
-								new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()),
-								new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer())));
-					}
-				}
-			}
-
-			return CompatibilityResult.requiresMigration();
-		}
+			// the (partially) matched pattern is computed lazily when this method is called.
+			// this is to avoid any overheads when using a simple, non-iterative condition.
 
-		private void serializeStates(Set<State<T>> states, DataOutputView out) throws IOException {
-			TypeSerializer<String> nameSerializer = StringSerializer.INSTANCE;
-			TypeSerializer<State.StateType> stateTypeSerializer = new EnumSerializer<>(State.StateType.class);
-			TypeSerializer<StateTransitionAction> actionSerializer = new EnumSerializer<>(StateTransitionAction.class);
-
-			out.writeInt(states.size());
-			for (State<T> state: states) {
-				nameSerializer.serialize(state.getName(), out);
-				stateTypeSerializer.serialize(state.getStateType(), out);
+			if (shouldUpdate) {
+				this.matchedEvents = nfa.extractCurrentMatches(eventSharedBuffer, computationState);
+				shouldUpdate = false;
 			}
 
-			for (State<T> state: states) {
-				nameSerializer.serialize(state.getName(), out);
-
-				out.writeInt(state.getStateTransitions().size());
-				for (StateTransition<T> transition : state.getStateTransitions()) {
-					nameSerializer.serialize(transition.getSourceState().getName(), out);
-					nameSerializer.serialize(transition.getTargetState().getName(), out);
-					actionSerializer.serialize(transition.getAction(), out);
-
-					serializeCondition(transition.getCondition(), out);
+			return new Iterable<T>() {
+				@Override
+				public Iterator<T> iterator() {
+					List<T> elements = matchedEvents.get(key);
+					return elements == null
+						? Collections.EMPTY_LIST.<T>iterator()
+						: elements.iterator();
 				}
-			}
-		}
-
-		private Set<State<T>> deserializeStates(DataInputView in) throws IOException {
-			TypeSerializer<String> nameSerializer = StringSerializer.INSTANCE;
-			TypeSerializer<State.StateType> stateTypeSerializer = new EnumSerializer<>(State.StateType.class);
-			TypeSerializer<StateTransitionAction> actionSerializer = new EnumSerializer<>(StateTransitionAction.class);
-
-			final int noOfStates = in.readInt();
-			Map<String, State<T>> states = new HashMap<>(noOfStates);
-
-			for (int i = 0; i < noOfStates; i++) {
-				String stateName = nameSerializer.deserialize(in);
-				State.StateType stateType = stateTypeSerializer.deserialize(in);
-
-				State<T> state = new State<>(stateName, stateType);
-				states.put(stateName, state);
-			}
-
-			for (int i = 0; i < noOfStates; i++) {
-				String srcName = nameSerializer.deserialize(in);
-
-				int noOfTransitions = in.readInt();
-				for (int j = 0; j < noOfTransitions; j++) {
-					String src = nameSerializer.deserialize(in);
-					Preconditions.checkState(src.equals(srcName),
-							"Source Edge names do not match (" + srcName + " - " + src + ").");
-
-					String trgt = nameSerializer.deserialize(in);
-					StateTransitionAction action = actionSerializer.deserialize(in);
-
-					IterativeCondition<T> condition = null;
-					try {
-						condition = deserializeCondition(in);
-					} catch (ClassNotFoundException e) {
-						e.printStackTrace();
-					}
-
-					State<T> srcState = states.get(src);
-					State<T> trgtState = states.get(trgt);
-					srcState.addStateTransition(action, trgtState, condition);
-				}
-
-			}
-			return new HashSet<>(states.values());
-		}
-
-		private void serializeCondition(IterativeCondition<T> condition, DataOutputView out) throws IOException {
-			out.writeBoolean(condition != null);
-			if (condition != null) {
-				ByteArrayOutputStream baos = new ByteArrayOutputStream();
-				ObjectOutputStream oos = new ObjectOutputStream(baos);
-
-				oos.writeObject(condition);
-
-				oos.close();
-				baos.close();
-
-				byte[] serCondition = baos.toByteArray();
-				out.writeInt(serCondition.length);
-				out.write(serCondition);
-			}
-		}
-
-		private IterativeCondition<T> deserializeCondition(DataInputView in) throws IOException, ClassNotFoundException {
-			boolean hasCondition = in.readBoolean();
-			if (hasCondition) {
-				int length = in.readInt();
-
-				byte[] serCondition = new byte[length];
-				in.readFully(serCondition);
-
-				ByteArrayInputStream bais = new ByteArrayInputStream(serCondition);
-				ObjectInputStream ois = new ObjectInputStream(bais);
-
-				IterativeCondition<T> condition = (IterativeCondition<T>) ois.readObject();
-				ois.close();
-				bais.close();
-
-				return condition;
-			}
-			return null;
+			};
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
new file mode 100644
index 0000000..e9414c3
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAState.java
@@ -0,0 +1,144 @@
+/*
+ * 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.nfa;
+
+import org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+/**
+ * State kept for a {@link NFA}.
+ *
+ * <p>The {@link AbstractKeyedCEPPatternOperator CEP operator}
+ * keeps one NFA per key, for keyed input streams, and a single global NFA for non-keyed ones.
+ * When an event gets processed, it updates the NFA's internal state machine.
+ *
+ * <p>An event that belongs to a partially matched sequence is kept in an internal
+ * {@link SharedBuffer buffer}, which is a memory-optimized data-structure exactly for
+ * this purpose. Events in the buffer are removed when all the matched sequences that
+ * contain them are:
+ * <ol>
+ *  <li>emitted (success)</li>
+ *  <li>discarded (patterns containing NOT)</li>
+ *  <li>timed-out (windowed patterns)</li>
+ * </ol>
+ *
+ * <p>The implementation is strongly based on the paper "Efficient Pattern Matching over Event Streams".
+ *
+ * @param <T> Type of the processed events
+ * @see <a href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf">
+ * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
+ */
+public class NFAState<T> {
+
+	/**
+	 * Current set of {@link ComputationState computation states} within the state machine.
+	 * These are the "active" intermediate states that are waiting for new matching
+	 * events to transition to new valid states.
+	 */
+	private Queue<ComputationState<T>> computationStates;
+
+	/**
+	 * Buffer used to store the matched events.
+	 */
+	private SharedBuffer<String, T> eventSharedBuffer;
+
+	/**
+	 * Flag indicating whether the matching status of the state machine has changed.
+	 */
+	private boolean stateChanged;
+
+	public NFAState(
+			Queue<ComputationState<T>> computationStates,
+			SharedBuffer<String, T> eventSharedBuffer,
+			boolean stateChanged) {
+		this.computationStates = computationStates;
+		this.eventSharedBuffer = eventSharedBuffer;
+		this.stateChanged = stateChanged;
+	}
+
+	public NFAState() {
+		this(new LinkedList<>(), new SharedBuffer<>(), false);
+	}
+
+	public NFAState(Iterable<ComputationState<T>> startingStates) {
+		this();
+
+		for (ComputationState<T> startingState : startingStates) {
+			computationStates.add(startingState);
+		}
+	}
+
+	/**
+	 * Check if the NFA has finished processing all incoming data so far. That is
+	 * when the buffer keeping the matches is empty.
+	 *
+	 * @return {@code true} if there are no elements in the {@link SharedBuffer},
+	 * {@code false} otherwise.
+	 */
+	public boolean isEmpty() {
+		return eventSharedBuffer.isEmpty();
+	}
+
+	/**
+	 * Check if the matching status of the NFA has changed so far.
+	 *
+	 * @return {@code true} if matching status has changed, {@code false} otherwise
+	 */
+	public boolean isStateChanged() {
+		return stateChanged;
+	}
+
+	/**
+	 * Reset the changed bit checked via {@link #isStateChanged()} to {@code false}.
+	 */
+	public void resetStateChanged() {
+		this.stateChanged = false;
+	}
+
+	public void setStateChanged(boolean stateChanged) {
+		this.stateChanged = stateChanged;
+	}
+
+	public Queue<ComputationState<T>> getComputationStates() {
+		return computationStates;
+	}
+
+	public SharedBuffer<String, T> getEventSharedBuffer() {
+		return eventSharedBuffer;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof NFAState) {
+			@SuppressWarnings("unchecked")
+			NFAState<T> other = (NFAState<T>) obj;
+
+			return eventSharedBuffer.equals(other.eventSharedBuffer);
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		return eventSharedBuffer.hashCode();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4dae798
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializer.java
@@ -0,0 +1,267 @@
+/*
+ * 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.nfa;
+
+import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+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 java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+/**
+ * A {@link TypeSerializer} for {@link NFAState} that uses Java Serialization.
+ */
+public class NFAStateSerializer<T> extends TypeSerializer<NFAState<T>> {
+
+	private static final long serialVersionUID = 2098282423980597010L;
+
+	private final TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer;
+
+	private final TypeSerializer<T> eventSerializer;
+
+	public NFAStateSerializer(TypeSerializer<T> typeSerializer) {
+		this(typeSerializer, new SharedBuffer.SharedBufferSerializer<>(StringSerializer.INSTANCE, typeSerializer));
+	}
+
+	public NFAStateSerializer(
+			TypeSerializer<T> typeSerializer,
+			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
+		this.eventSerializer = typeSerializer;
+		this.sharedBufferSerializer = sharedBufferSerializer;
+	}
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public NFAStateSerializer<T> duplicate() {
+		return new NFAStateSerializer<>(eventSerializer.duplicate());
+	}
+
+	@Override
+	public NFAState<T> createInstance() {
+		return null;
+	}
+
+	@Override
+	public NFAState<T> copy(NFAState<T> from) {
+		try {
+			ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			serialize(from, new DataOutputViewStreamWrapper(baos));
+			baos.close();
+
+			byte[] data = baos.toByteArray();
+
+			ByteArrayInputStream bais = new ByteArrayInputStream(data);
+			NFAState<T> copy = deserialize(new DataInputViewStreamWrapper(bais));
+			bais.close();
+			return copy;
+		} catch (IOException e) {
+			throw new RuntimeException("Could not copy NFA.", e);
+		}
+	}
+
+	@Override
+	public NFAState<T> copy(NFAState<T> from, NFAState<T> reuse) {
+		return copy(from);
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void serialize(NFAState<T> record, DataOutputView target) throws IOException {
+		sharedBufferSerializer.serialize(record.getEventSharedBuffer(), target);
+
+		target.writeInt(record.getComputationStates().size());
+
+		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
+		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
+		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+
+		for (ComputationState<T> computationState: record.getComputationStates()) {
+			stateNameSerializer.serialize(computationState.getState(), target);
+			stateNameSerializer.serialize(computationState.getPreviousState(), target);
+
+			timestampSerializer.serialize(computationState.getTimestamp(), target);
+			versionSerializer.serialize(computationState.getVersion(), target);
+			timestampSerializer.serialize(computationState.getStartTimestamp(), target);
+			target.writeInt(computationState.getCounter());
+
+			if (computationState.getEvent() == null) {
+				target.writeBoolean(false);
+			} else {
+				target.writeBoolean(true);
+				eventSerializer.serialize(computationState.getEvent(), target);
+			}
+		}
+	}
+
+	@Override
+	public NFAState<T> deserialize(DataInputView source) throws IOException {
+		SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
+
+		Queue<ComputationState<T>> computationStates = new LinkedList<>();
+		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
+		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
+		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+
+		int computationStateNo = source.readInt();
+		for (int i = 0; i < computationStateNo; i++) {
+			String state = stateNameSerializer.deserialize(source);
+			String prevState = stateNameSerializer.deserialize(source);
+			long timestamp = timestampSerializer.deserialize(source);
+			DeweyNumber version = versionSerializer.deserialize(source);
+			long startTimestamp = timestampSerializer.deserialize(source);
+			int counter = source.readInt();
+
+			T event = null;
+			if (source.readBoolean()) {
+				event = eventSerializer.deserialize(source);
+			}
+
+			computationStates.add(ComputationState.createState(
+					state, prevState, event, counter, timestamp, version, startTimestamp));
+		}
+
+		return new NFAState<>(computationStates, sharedBuffer, false);
+	}
+
+	@Override
+	public NFAState<T> deserialize(NFAState<T> reuse, DataInputView source) throws IOException {
+		return deserialize(source);
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		SharedBuffer<String, T> sharedBuffer = sharedBufferSerializer.deserialize(source);
+		sharedBufferSerializer.serialize(sharedBuffer, target);
+
+		StringSerializer stateNameSerializer = StringSerializer.INSTANCE;
+		LongSerializer timestampSerializer = LongSerializer.INSTANCE;
+		DeweyNumber.DeweyNumberSerializer versionSerializer = new DeweyNumber.DeweyNumberSerializer();
+
+		int computationStateNo = source.readInt();
+		target.writeInt(computationStateNo);
+
+		for (int i = 0; i < computationStateNo; i++) {
+			String stateName = stateNameSerializer.deserialize(source);
+			stateNameSerializer.serialize(stateName, target);
+
+			String prevStateName = stateNameSerializer.deserialize(source);
+			stateNameSerializer.serialize(prevStateName, target);
+
+			long timestamp = timestampSerializer.deserialize(source);
+			timestampSerializer.serialize(timestamp, target);
+
+			DeweyNumber version = versionSerializer.deserialize(source);
+			versionSerializer.serialize(version, target);
+
+			long startTimestamp = timestampSerializer.deserialize(source);
+			timestampSerializer.serialize(startTimestamp, target);
+
+			int counter = source.readInt();
+			target.writeInt(counter);
+
+			boolean hasEvent = source.readBoolean();
+			target.writeBoolean(hasEvent);
+			if (hasEvent) {
+				T event = eventSerializer.deserialize(source);
+				eventSerializer.serialize(event, target);
+			}
+		}
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		return obj == this ||
+				(obj != null && obj.getClass().equals(getClass()) &&
+						sharedBufferSerializer.equals(((NFAStateSerializer) obj).sharedBufferSerializer) &&
+						eventSerializer.equals(((NFAStateSerializer) obj).eventSerializer));
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return true;
+	}
+
+	@Override
+	public int hashCode() {
+		return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode();
+	}
+
+	@Override
+	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		return new NFAStateSerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer);
+	}
+
+	@Override
+	public CompatibilityResult<NFAState<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		if (configSnapshot instanceof NFAStateSerializerConfigSnapshot) {
+			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
+					((NFAStateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
+
+			CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
+					serializersAndConfigs.get(0).f0,
+					UnloadableDummyTypeSerializer.class,
+					serializersAndConfigs.get(0).f1,
+					eventSerializer);
+
+			CompatibilityResult<SharedBuffer<String, T>> sharedBufCompatResult =
+					CompatibilityUtil.resolveCompatibilityResult(
+							serializersAndConfigs.get(1).f0,
+							UnloadableDummyTypeSerializer.class,
+							serializersAndConfigs.get(1).f1,
+							sharedBufferSerializer);
+
+			if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
+				return CompatibilityResult.compatible();
+			} else {
+				if (eventCompatResult.getConvertDeserializer() != null &&
+					sharedBufCompatResult.getConvertDeserializer() != null) {
+					return CompatibilityResult.requiresMigration(
+						new NFAStateSerializer<>(
+							new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()),
+							new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer())));
+				}
+			}
+		}
+
+		return CompatibilityResult.requiresMigration();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java
new file mode 100644
index 0000000..ab72348
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFAStateSerializerConfigSnapshot.java
@@ -0,0 +1,45 @@
+/*
+ * 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.nfa;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+/**
+ * {@link CompositeTypeSerializerConfigSnapshot} for {@link NFAStateSerializer}.
+ */
+public final class NFAStateSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
+
+	private static final int VERSION = 1;
+
+	/** This empty constructor is required for deserializing the configuration. */
+	public NFAStateSerializerConfigSnapshot() {}
+
+	public NFAStateSerializerConfigSnapshot(
+			TypeSerializer<T> eventSerializer,
+			TypeSerializer<SharedBuffer<String, T>> sharedBufferSerializer) {
+
+		super(eventSerializer, sharedBufferSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index adfa737..a00a310 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -36,7 +36,6 @@ import org.apache.commons.lang3.StringUtils;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -67,11 +66,9 @@ import java.util.Stack;
  * @param <K> Type of the keys
  * @param <V> Type of the values
  */
-public class SharedBuffer<K extends Serializable, V> implements Serializable {
+public class SharedBuffer<K, V> {
 
-	private static final long serialVersionUID = 9213251042562206495L;
-
-	private transient Map<K, SharedBufferPage<K, V>> pages;
+	private Map<K, SharedBufferPage<K, V>> pages;
 
 	public SharedBuffer() {
 		this.pages = new HashMap<>(4);
@@ -807,7 +804,7 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 	/**
 	 * A {@link TypeSerializer} for the {@link SharedBuffer}.
 	 */
-	public static class SharedBufferSerializer<K extends Serializable, V> extends TypeSerializer<SharedBuffer<K, V>> {
+	public static class SharedBufferSerializer<K, V> extends TypeSerializer<SharedBuffer<K, V>> {
 
 		private static final long serialVersionUID = -3254176794680331560L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index 39e8d34..5b9522b 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -919,11 +919,8 @@ public class NFACompiler {
 
 		@Override
 		public NFA<T> createNFA() {
-			NFA<T> result =  new NFA<>(inputTypeSerializer.duplicate(), windowTime, timeoutHandling);
-
-			result.addStates(states);
-
-			return result;
+			return new NFA<>(
+				inputTypeSerializer.duplicate(), windowTime, timeoutHandling, states);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index 36f0283..312549f 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -31,6 +31,8 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.EventComparator;
 import org.apache.flink.cep.nfa.AfterMatchSkipStrategy;
 import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.nfa.NFAState;
+import org.apache.flink.cep.nfa.NFAStateSerializer;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.VoidNamespace;
@@ -79,13 +81,15 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	private static final String NFA_OPERATOR_STATE_NAME = "nfaOperatorStateName";
 	private static final String EVENT_QUEUE_STATE_NAME = "eventQueuesStateName";
 
-	private transient ValueState<NFA<IN>> nfaOperatorState;
+	private transient ValueState<NFAState<IN>> nfaValueState;
 	private transient MapState<Long, List<IN>> elementQueueState;
 
 	private final NFACompiler.NFAFactory<IN> nfaFactory;
 
 	private transient InternalTimerService<VoidNamespace> timerService;
 
+	private transient NFA<IN> nfa;
+
 	/**
 	 * The last seen watermark. This will be used to
 	 * decide if an incoming element is late or not.
@@ -129,11 +133,11 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	public void initializeState(StateInitializationContext context) throws Exception {
 		super.initializeState(context);
 
-		if (nfaOperatorState == null) {
-			nfaOperatorState = getRuntimeContext().getState(
+		if (nfaValueState == null) {
+			nfaValueState = getRuntimeContext().getState(
 				new ValueStateDescriptor<>(
 						NFA_OPERATOR_STATE_NAME,
-						new NFA.NFASerializer<>(inputSerializer)));
+						new NFAStateSerializer<>(inputSerializer)));
 		}
 
 		if (elementQueueState == null) {
@@ -155,6 +159,8 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 				"watermark-callbacks",
 				VoidNamespaceSerializer.INSTANCE,
 				this);
+
+		this.nfa = nfaFactory.createNFA();
 	}
 
 	@Override
@@ -162,9 +168,9 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		if (isProcessingTime) {
 			if (comparator == null) {
 				// there can be no out of order elements in processing time
-				NFA<IN> nfa = getNFA();
-				processEvent(nfa, element.getValue(), getProcessingTimeService().getCurrentProcessingTime());
-				updateNFA(nfa);
+				NFAState<IN> nfaState = getNFAState();
+				processEvent(nfaState, element.getValue(), getProcessingTimeService().getCurrentProcessingTime());
+				updateNFA(nfaState);
 			} else {
 				long currentTime = timerService.currentProcessingTime();
 				bufferEvent(element.getValue(), currentTime);
@@ -238,27 +244,27 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 
 		// STEP 1
 		PriorityQueue<Long> sortedTimestamps = getSortedTimestamps();
-		NFA<IN> nfa = getNFA();
+		NFAState<IN> nfaState = getNFAState();
 
 		// STEP 2
 		while (!sortedTimestamps.isEmpty() && sortedTimestamps.peek() <= timerService.currentWatermark()) {
 			long timestamp = sortedTimestamps.poll();
 			sort(elementQueueState.get(timestamp)).forEachOrdered(
-				event -> processEvent(nfa, event, timestamp)
+				event -> processEvent(nfaState, event, timestamp)
 			);
 			elementQueueState.remove(timestamp);
 		}
 
 		// STEP 3
-		advanceTime(nfa, timerService.currentWatermark());
+		advanceTime(nfaState, timerService.currentWatermark());
 
 		// STEP 4
 		if (sortedTimestamps.isEmpty()) {
 			elementQueueState.clear();
 		}
-		updateNFA(nfa);
+		updateNFA(nfaState);
 
-		if (!sortedTimestamps.isEmpty() || !nfa.isEmpty()) {
+		if (!sortedTimestamps.isEmpty() || !nfaState.isEmpty()) {
 			saveRegisterWatermarkTimer();
 		}
 
@@ -276,7 +282,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 
 		// STEP 1
 		PriorityQueue<Long> sortedTimestamps = getSortedTimestamps();
-		NFA<IN> nfa = getNFA();
+		NFAState<IN> nfa = getNFAState();
 
 		// STEP 2
 		while (!sortedTimestamps.isEmpty()) {
@@ -307,18 +313,18 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 		this.lastWatermark = timestamp;
 	}
 
-	private NFA<IN> getNFA() throws IOException {
-		NFA<IN> nfa = nfaOperatorState.value();
-		return nfa != null ? nfa : nfaFactory.createNFA();
+	private NFAState<IN> getNFAState() throws IOException {
+		NFAState<IN> nfaState = nfaValueState.value();
+		return nfaState != null ? nfaState : nfa.createNFAState();
 	}
 
-	private void updateNFA(NFA<IN> nfa) throws IOException {
-		if (nfa.isNFAChanged()) {
-			if (nfa.isEmpty()) {
-				nfaOperatorState.clear();
+	private void updateNFA(NFAState<IN> nfaState) throws IOException {
+		if (nfaState.isStateChanged()) {
+			if (nfaState.isEmpty()) {
+				nfaValueState.clear();
 			} else {
-				nfa.resetNFAChanged();
-				nfaOperatorState.update(nfa);
+				nfaState.resetStateChanged();
+				nfaValueState.update(nfaState);
 			}
 		}
 	}
@@ -335,13 +341,13 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	 * Process the given event by giving it to the NFA and outputting the produced set of matched
 	 * event sequences.
 	 *
-	 * @param nfa NFA to be used for the event detection
+	 * @param nfaState Our NFAState object
 	 * @param event The current event to be processed
 	 * @param timestamp The timestamp of the event
 	 */
-	private void processEvent(NFA<IN> nfa, IN event, long timestamp)  {
+	private void processEvent(NFAState<IN> nfaState, IN event, long timestamp)  {
 		Tuple2<Collection<Map<String, List<IN>>>, Collection<Tuple2<Map<String, List<IN>>, Long>>> patterns =
-			nfa.process(event, timestamp, afterMatchSkipStrategy);
+			nfa.process(nfaState, event, timestamp, afterMatchSkipStrategy);
 
 		try {
 			processMatchedSequences(patterns.f0, timestamp);
@@ -355,12 +361,9 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	/**
 	 * Advances the time for the given NFA to the given timestamp. This can lead to pruning and
 	 * timeouts.
-	 *
-	 * @param nfa to advance the time for
-	 * @param timestamp to advance the time to
 	 */
-	private void advanceTime(NFA<IN> nfa, long timestamp) throws Exception {
-		processEvent(nfa, null, timestamp);
+	private void advanceTime(NFAState<IN> nfaState, long timestamp) throws Exception {
+		processEvent(nfaState, null, timestamp);
 	}
 
 	protected abstract void processMatchedSequences(Iterable<Map<String, List<IN>>> matchingSequences, long timestamp) throws Exception;
@@ -373,9 +376,9 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	//////////////////////			Testing Methods			//////////////////////
 
 	@VisibleForTesting
-	public boolean hasNonEmptyNFA(KEY key) throws IOException {
+	public boolean hasNonEmptyNFAState(KEY key) throws IOException {
 		setCurrentKey(key);
-		return nfaOperatorState.value() != null;
+		return nfaValueState.value() != null;
 	}
 
 	@VisibleForTesting

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
index e7c814f..0386d0e 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/IterativeCondition.java
@@ -84,7 +84,7 @@ public abstract class IterativeCondition<T> implements Function, Serializable {
 	/**
 	 * The context used when evaluating the {@link IterativeCondition condition}.
 	 */
-	public interface Context<T> extends Serializable {
+	public interface Context<T> {
 
 		/**
 		 * @return An {@link Iterable} over the already accepted elements

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
index c2c7cda..f9b9aa2 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java
@@ -1075,14 +1075,16 @@ public class GroupITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		final List<List<Event>> resultingPatterns = feedNFA(inputEvents, nfa, nfaState);
 
 		compareMaps(resultingPatterns, Lists.<List<Event>>newArrayList(
 			Lists.newArrayList(c, a1, b1, d),
 			Lists.newArrayList(c, a1, b1, a2, b2, d)
 		));
 
-		assertTrue(nfa.isEmpty());
+		assertTrue(nfaState.isEmpty());
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/55cd059a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
index 84278b1..76bc2d0 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java
@@ -386,9 +386,11 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), true);
 
+		NFAState<Event> nfaState = nfa.createNFAState();
+
 		for (StreamRecord<Event> event: events) {
 			Tuple2<Collection<Map<String, List<Event>>>, Collection<Tuple2<Map<String, List<Event>>, Long>>> patterns =
-					nfa.process(event.getValue(), event.getTimestamp());
+					nfa.process(nfaState, event.getValue(), event.getTimestamp());
 
 			Collection<Map<String, List<Event>>> matchedPatterns = patterns.f0;
 			Collection<Tuple2<Map<String, List<Event>>, Long>> timeoutPatterns = patterns.f1;
@@ -2317,16 +2319,18 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		nfa.process(startEvent, 1);
-		nfa.process(middleEvent1, 2);
-		nfa.process(middleEvent2, 3);
-		nfa.process(middleEvent3, 4);
-		nfa.process(end1, 6);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		nfa.process(nfaState, startEvent, 1);
+		nfa.process(nfaState, middleEvent1, 2);
+		nfa.process(nfaState, middleEvent2, 3);
+		nfa.process(nfaState, middleEvent3, 4);
+		nfa.process(nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(null, 10);
+		nfa.process(nfaState, null, 10);
 
-		assertEquals(true, nfa.isEmpty());
+		assertEquals(true, nfaState.isEmpty());
 	}
 
 	@Test
@@ -2360,14 +2364,16 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		nfa.process(startEvent, 1);
-		nfa.process(middleEvent, 5);
-		nfa.process(end1, 6);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		nfa.process(nfaState, startEvent, 1);
+		nfa.process(nfaState, middleEvent, 5);
+		nfa.process(nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(null, 10);
+		nfa.process(nfaState, null, 10);
 
-		assertEquals(true, nfa.isEmpty());
+		assertEquals(true, nfaState.isEmpty());
 	}
 
 	@Test
@@ -2402,15 +2408,17 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		nfa.process(startEvent, 1);
-		nfa.process(middleEvent1, 3);
-		nfa.process(middleEvent2, 4);
-		nfa.process(end1, 6);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		nfa.process(nfaState, startEvent, 1);
+		nfa.process(nfaState, middleEvent1, 3);
+		nfa.process(nfaState, middleEvent2, 4);
+		nfa.process(nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(null, 10);
+		nfa.process(nfaState, null, 10);
 
-		assertEquals(true, nfa.isEmpty());
+		assertEquals(true, nfaState.isEmpty());
 	}
 
 	@Test
@@ -2445,15 +2453,17 @@ public class NFAITCase extends TestLogger {
 
 		NFA<Event> nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false);
 
-		nfa.process(startEvent, 1);
-		nfa.process(middleEvent1, 3);
-		nfa.process(middleEvent2, 4);
-		nfa.process(end1, 6);
+		NFAState<Event> nfaState = nfa.createNFAState();
+
+		nfa.process(nfaState, startEvent, 1);
+		nfa.process(nfaState, middleEvent1, 3);
+		nfa.process(nfaState, middleEvent2, 4);
+		nfa.process(nfaState, end1, 6);
 
 		//pruning element
-		nfa.process(null, 10);
+		nfa.process(nfaState, null, 10);
 
-		assertEquals(true, nfa.isEmpty());
+		assertEquals(true, nfaState.isEmpty());
 	}
 
 	///////////////////////////////////////   Skip till next     /////////////////////////////
@@ -2703,8 +2713,11 @@ public class NFAITCase extends TestLogger {
 
 		List<Map<String, List<Event>>> resultingPatterns = new ArrayList<>();
 
+		NFAState<Event> nfaState = nfa.createNFAState();
+
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
+					nfaState,
 					inputEvent.getValue(),
 					inputEvent.getTimestamp()).f0;
 
@@ -2774,8 +2787,11 @@ public class NFAITCase extends TestLogger {
 
 		List<Map<String, List<Event>>> resultingPatterns = new ArrayList<>();
 
+		NFAState<Event> nfaState = nfa.createNFAState();
+
 		for (StreamRecord<Event> inputEvent : inputEvents) {
 			Collection<Map<String, List<Event>>> patterns = nfa.process(
+				nfaState,
 				inputEvent.getValue(),
 				inputEvent.getTimestamp()).f0;