You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by hs...@apache.org on 2015/10/06 17:14:21 UTC

[1/5] flink git commit: [FLINK-2815] [REFACTOR] Remove Pact from class and file names since it is no longer valid reference

Repository: flink
Updated Branches:
  refs/heads/master e494c2795 -> b08669abf


http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java
new file mode 100644
index 0000000..fd5d238
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.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.runtime.operators;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+
+/**
+ * The task context gives a driver (e.g., {@link MapDriver}, or {@link JoinDriver}) access to
+ * the runtime components and configuration that they can use to fulfil their task.
+ *
+ * @param <S> The UDF type.
+ * @param <OT> The produced data type.
+ *
+ * @see Driver
+ */
+public interface TaskContext<S, OT> {
+	
+	TaskConfig getTaskConfig();
+	
+	TaskManagerRuntimeInfo getTaskManagerInfo();
+
+	ClassLoader getUserCodeClassLoader();
+	
+	MemoryManager getMemoryManager();
+	
+	IOManager getIOManager();
+
+	<X> MutableObjectIterator<X> getInput(int index);
+	
+	<X> TypeSerializerFactory<X> getInputSerializer(int index);
+	
+	<X> TypeComparator<X> getDriverComparator(int index);
+	
+	S getStub();
+
+	ExecutionConfig getExecutionConfig();
+
+	Collector<OT> getOutputCollector();
+	
+	AbstractInvokable getOwningNepheleTask();
+	
+	String formatLogString(String message);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
index 098686c..4791761 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
@@ -22,18 +22,18 @@ import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
-public class UnionWithTempOperator<T> implements PactDriver<Function, T> {
+public class UnionWithTempOperator<T> implements Driver<Function, T> {
 	
 	private static final int CACHED_INPUT = 0;
 	private static final int STREAMED_INPUT = 1;
 	
-	private PactTaskContext<Function, T> taskContext;
+	private TaskContext<Function, T> taskContext;
 	
 	private volatile boolean running;
 	
 	
 	@Override
-	public void setup(PactTaskContext<Function, T> context) {
+	public void setup(TaskContext<Function, T> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriver.java
index 4641fce..46ee41b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedAllReduceDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,7 +41,7 @@ public class ChainedAllReduceDriver<IT> extends ChainedDriver<IT, IT> {
 	@Override
 	public void setup(AbstractInvokable parent) {
 		@SuppressWarnings("unchecked")
-		final ReduceFunction<IT> red = RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, ReduceFunction.class);
+		final ReduceFunction<IT> red = BatchTask.instantiateUserCode(this.config, userCodeClassLoader, ReduceFunction.class);
 		this.reducer = red;
 		FunctionUtils.setFunctionRuntimeContext(red, getUdfRuntimeContext());
 
@@ -56,12 +56,12 @@ public class ChainedAllReduceDriver<IT> extends ChainedDriver<IT, IT> {
 	@Override
 	public void openTask() throws Exception {
 		Configuration stubConfig = this.config.getStubParameters();
-		RegularPactTask.openUserCode(this.reducer, stubConfig);
+		BatchTask.openUserCode(this.reducer, stubConfig);
 	}
 
 	@Override
 	public void closeTask() throws Exception {
-		RegularPactTask.closeUserCode(this.reducer);
+		BatchTask.closeUserCode(this.reducer);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java
index 482103c..8900ed7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedCollectorMapDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.RichFunction;
 import org.apache.flink.api.common.functions.GenericCollectorMap;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 
 @SuppressWarnings("deprecation")
 public class ChainedCollectorMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
@@ -35,7 +35,7 @@ public class ChainedCollectorMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 	public void setup(AbstractInvokable parent) {
 		@SuppressWarnings("unchecked")
 		final GenericCollectorMap<IT, OT> mapper =
-			RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, GenericCollectorMap.class);
+			BatchTask.instantiateUserCode(this.config, userCodeClassLoader, GenericCollectorMap.class);
 		this.mapper = mapper;
 		mapper.setRuntimeContext(getUdfRuntimeContext());
 	}
@@ -43,12 +43,12 @@ public class ChainedCollectorMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 	@Override
 	public void openTask() throws Exception {
 		Configuration stubConfig = this.config.getStubParameters();
-		RegularPactTask.openUserCode(this.mapper, stubConfig);
+		BatchTask.openUserCode(this.mapper, stubConfig);
 	}
 
 	@Override
 	public void closeTask() throws Exception {
-		RegularPactTask.closeUserCode(this.mapper);
+		BatchTask.closeUserCode(this.mapper);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java
index ea6cfe3..6edeb84 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.Collector;
@@ -32,7 +32,7 @@ import org.apache.flink.util.Collector;
 import java.util.Map;
 
 /**
- * The interface to be implemented by drivers that do not run in an own pact task context, but are chained to other
+ * The interface to be implemented by drivers that do not run in an own task context, but are chained to other
  * tasks.
  */
 public abstract class ChainedDriver<IT, OT> implements Collector<IT> {
@@ -63,8 +63,8 @@ public abstract class ChainedDriver<IT, OT> implements Collector<IT> {
 
 		Environment env = parent.getEnvironment();
 
-		if (parent instanceof RegularPactTask) {
-			this.udfContext = ((RegularPactTask<?, ?>) parent).createRuntimeContext(taskName);
+		if (parent instanceof BatchTask) {
+			this.udfContext = ((BatchTask<?, ?>) parent).createRuntimeContext(taskName);
 		} else {
 			this.udfContext = new DistributedRuntimeUDFContext(taskName, env.getNumberOfSubtasks(),
 					env.getIndexInSubtaskGroup(), userCodeClassLoader, parent.getExecutionConfig(),

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java
index bc3b6a1..f51cb68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedFlatMapDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 
 public class ChainedFlatMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 
@@ -36,7 +36,7 @@ public class ChainedFlatMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 	public void setup(AbstractInvokable parent) {
 		@SuppressWarnings("unchecked")
 		final FlatMapFunction<IT, OT> mapper =
-			RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, FlatMapFunction.class);
+			BatchTask.instantiateUserCode(this.config, userCodeClassLoader, FlatMapFunction.class);
 		this.mapper = mapper;
 		FunctionUtils.setFunctionRuntimeContext(mapper, getUdfRuntimeContext());
 	}
@@ -44,12 +44,12 @@ public class ChainedFlatMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 	@Override
 	public void openTask() throws Exception {
 		Configuration stubConfig = this.config.getStubParameters();
-		RegularPactTask.openUserCode(this.mapper, stubConfig);
+		BatchTask.openUserCode(this.mapper, stubConfig);
 	}
 
 	@Override
 	public void closeTask() throws Exception {
-		RegularPactTask.closeUserCode(this.mapper);
+		BatchTask.closeUserCode(this.mapper);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java
index db192df..9b888f2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedMapDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 
 public class ChainedMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 
@@ -36,7 +36,7 @@ public class ChainedMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 	public void setup(AbstractInvokable parent) {
 		@SuppressWarnings("unchecked")
 		final MapFunction<IT, OT> mapper =
-			RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, MapFunction.class);
+			BatchTask.instantiateUserCode(this.config, userCodeClassLoader, MapFunction.class);
 		this.mapper = mapper;
 		FunctionUtils.setFunctionRuntimeContext(mapper, getUdfRuntimeContext());
 	}
@@ -44,12 +44,12 @@ public class ChainedMapDriver<IT, OT> extends ChainedDriver<IT, OT> {
 	@Override
 	public void openTask() throws Exception {
 		Configuration stubConfig = this.config.getStubParameters();
-		RegularPactTask.openUserCode(this.mapper, stubConfig);
+		BatchTask.openUserCode(this.mapper, stubConfig);
 	}
 
 	@Override
 	public void closeTask() throws Exception {
-		RegularPactTask.closeUserCode(this.mapper);
+		BatchTask.closeUserCode(this.mapper);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
index cf0fc85..4a04fb5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
@@ -29,7 +29,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
 import org.apache.flink.runtime.operators.sort.InMemorySorter;
 import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;
@@ -87,7 +87,7 @@ public class GroupCombineChainedDriver<IN, OUT> extends ChainedDriver<IN, OUT> {
 
 		@SuppressWarnings("unchecked")
 		final GroupReduceFunction<IN, OUT> combiner =
-			RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, GroupReduceFunction.class);
+			BatchTask.instantiateUserCode(this.config, userCodeClassLoader, GroupReduceFunction.class);
 		this.reducer = combiner;
 		FunctionUtils.setFunctionRuntimeContext(combiner, getUdfRuntimeContext());
 	}
@@ -96,7 +96,7 @@ public class GroupCombineChainedDriver<IN, OUT> extends ChainedDriver<IN, OUT> {
 	public void openTask() throws Exception {
 		// open the stub first
 		final Configuration stubConfig = this.config.getStubParameters();
-		RegularPactTask.openUserCode(this.reducer, stubConfig);
+		BatchTask.openUserCode(this.reducer, stubConfig);
 
 		// ----------------- Set up the sorter -------------------------
 
@@ -135,7 +135,7 @@ public class GroupCombineChainedDriver<IN, OUT> extends ChainedDriver<IN, OUT> {
 		this.parent.getEnvironment().getMemoryManager().release(this.memory);
 
 		if (this.running) {
-			RegularPactTask.closeUserCode(this.reducer);
+			BatchTask.closeUserCode(this.reducer);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
index da9698c..408abc2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
@@ -32,7 +32,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
 import org.apache.flink.runtime.operators.sort.InMemorySorter;
 import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;
@@ -87,7 +87,7 @@ public class SynchronousChainedCombineDriver<IN, OUT> extends ChainedDriver<IN,
 
 		@SuppressWarnings("unchecked")
 		final GroupCombineFunction<IN, OUT> combiner =
-			RegularPactTask.instantiateUserCode(this.config, userCodeClassLoader, GroupCombineFunction.class);
+			BatchTask.instantiateUserCode(this.config, userCodeClassLoader, GroupCombineFunction.class);
 		this.combiner = combiner;
 		FunctionUtils.setFunctionRuntimeContext(combiner, getUdfRuntimeContext());
 	}
@@ -96,7 +96,7 @@ public class SynchronousChainedCombineDriver<IN, OUT> extends ChainedDriver<IN,
 	public void openTask() throws Exception {
 		// open the stub first
 		final Configuration stubConfig = this.config.getStubParameters();
-		RegularPactTask.openUserCode(this.combiner, stubConfig);
+		BatchTask.openUserCode(this.combiner, stubConfig);
 
 		// ----------------- Set up the sorter -------------------------
 
@@ -134,7 +134,7 @@ public class SynchronousChainedCombineDriver<IN, OUT> extends ChainedDriver<IN,
 		this.parent.getEnvironment().getMemoryManager().release(this.memory);
 
 		if (this.running) {
-			RegularPactTask.closeUserCode(this.combiner);
+			BatchTask.closeUserCode(this.combiner);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java
index 6c97097..0254c8c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java
@@ -48,7 +48,7 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
 import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
 import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.PactDriver;
+import org.apache.flink.runtime.operators.Driver;
 import org.apache.flink.runtime.operators.chaining.ChainedDriver;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.types.Value;
@@ -307,11 +307,11 @@ public class TaskConfig implements Serializable {
 	//                                      Driver
 	// --------------------------------------------------------------------------------------------
 	
-	public void setDriver(@SuppressWarnings("rawtypes") Class<? extends PactDriver> driver) {
+	public void setDriver(@SuppressWarnings("rawtypes") Class<? extends Driver> driver) {
 		this.config.setString(DRIVER_CLASS, driver.getName());
 	}
 	
-	public <S extends Function, OT> Class<? extends PactDriver<S, OT>> getDriver() {
+	public <S extends Function, OT> Class<? extends Driver<S, OT>> getDriver() {
 		final String className = this.config.getString(DRIVER_CLASS, null);
 		if (className == null) {
 			throw new CorruptConfigurationException("The pact driver class is missing.");
@@ -319,7 +319,7 @@ public class TaskConfig implements Serializable {
 		
 		try {
 			@SuppressWarnings("unchecked")
-			final Class<PactDriver<S, OT>> pdClazz = (Class<PactDriver<S, OT>>) (Class<?>) PactDriver.class;
+			final Class<Driver<S, OT>> pdClazz = (Class<Driver<S, OT>>) (Class<?>) Driver.class;
 			return Class.forName(className).asSubclass(pdClazz);
 		} catch (ClassNotFoundException cnfex) {
 			throw new CorruptConfigurationException("The given driver class cannot be found.");

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
index 3a36fe8..58755f3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.core.testutils.CommonTestUtils;
 import org.junit.Test;
 
@@ -47,7 +47,7 @@ public class TaskDeploymentDescriptorTest {
 			final int currentNumberOfSubtasks = 1;
 			final Configuration jobConfiguration = new Configuration();
 			final Configuration taskConfiguration = new Configuration();
-			final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
+			final Class<? extends AbstractInvokable> invokableClass = BatchTask.class;
 			final List<ResultPartitionDeploymentDescriptor> producedResults = new ArrayList<ResultPartitionDeploymentDescriptor>(0);
 			final List<InputGateDeploymentDescriptor> inputGates = new ArrayList<InputGateDeploymentDescriptor>(0);
 			final List<BlobKey> requiredJars = new ArrayList<BlobKey>(0);

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index e3fc852..bea7c22 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -43,7 +43,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.junit.Test;
 
@@ -69,10 +69,10 @@ public class ExecutionGraphDeploymentTest {
 			v3.setParallelism(10);
 			v4.setParallelism(10);
 
-			v1.setInvokableClass(RegularPactTask.class);
-			v2.setInvokableClass(RegularPactTask.class);
-			v3.setInvokableClass(RegularPactTask.class);
-			v4.setInvokableClass(RegularPactTask.class);
+			v1.setInvokableClass(BatchTask.class);
+			v2.setInvokableClass(BatchTask.class);
+			v3.setInvokableClass(BatchTask.class);
+			v4.setInvokableClass(BatchTask.class);
 
 			v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL);
 			v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL);
@@ -111,7 +111,7 @@ public class ExecutionGraphDeploymentTest {
 			assertEquals(jid2, descr.getVertexID());
 			assertEquals(3, descr.getIndexInSubtaskGroup());
 			assertEquals(10, descr.getNumberOfSubtasks());
-			assertEquals(RegularPactTask.class.getName(), descr.getInvokableClassName());
+			assertEquals(BatchTask.class.getName(), descr.getInvokableClassName());
 			assertEquals("v2", descr.getTaskName());
 
 			List<ResultPartitionDeploymentDescriptor> producedPartitions = descr.getProducedPartitions();
@@ -276,8 +276,8 @@ public class ExecutionGraphDeploymentTest {
 		v1.setParallelism(dop1);
 		v2.setParallelism(dop2);
 
-		v1.setInvokableClass(RegularPactTask.class);
-		v2.setInvokableClass(RegularPactTask.class);
+		v1.setInvokableClass(BatchTask.class);
+		v2.setInvokableClass(BatchTask.class);
 
 		// execution graph that executes actions synchronously
 		ExecutionGraph eg = new ExecutionGraph(

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
index 88a71c4..1f19699 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/chaining/ChainTaskTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.MapTaskTest.MockMapStub;
 import org.apache.flink.runtime.operators.ReduceTaskTest.MockReduceStub;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
@@ -102,8 +102,8 @@ public class ChainTaskTest extends TaskTestBase {
 			
 			// chained map+combine
 			{
-				RegularPactTask<GenericCollectorMap<Record, Record>, Record> testTask = 
-											new RegularPactTask<GenericCollectorMap<Record, Record>, Record>();
+				BatchTask<GenericCollectorMap<Record, Record>, Record> testTask =
+											new BatchTask<GenericCollectorMap<Record, Record>, Record>();
 				registerTask(testTask, CollectorMapDriver.class, MockMapStub.class);
 				
 				try {
@@ -163,8 +163,8 @@ public class ChainTaskTest extends TaskTestBase {
 			
 			// chained map+combine
 			{
-				final RegularPactTask<GenericCollectorMap<Record, Record>, Record> testTask = 
-											new RegularPactTask<GenericCollectorMap<Record, Record>, Record>();
+				final BatchTask<GenericCollectorMap<Record, Record>, Record> testTask =
+											new BatchTask<GenericCollectorMap<Record, Record>, Record>();
 				
 				super.registerTask(testTask, CollectorMapDriver.class, MockMapStub.class);
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
index 0a02f30..9be957a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
@@ -29,14 +29,14 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.PactTaskContext;
+import org.apache.flink.runtime.operators.TaskContext;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
-public class TestTaskContext<S, T> implements PactTaskContext<S, T> {
+public class TestTaskContext<S, T> implements TaskContext<S, T> {
 	
 	private final AbstractInvokable owner = new DummyInvokable();
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
index 5136aea..7043a63 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
@@ -30,9 +30,9 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.PactDriver;
-import org.apache.flink.runtime.operators.PactTaskContext;
-import org.apache.flink.runtime.operators.ResettablePactDriver;
+import org.apache.flink.runtime.operators.Driver;
+import org.apache.flink.runtime.operators.TaskContext;
+import org.apache.flink.runtime.operators.ResettableDriver;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
@@ -51,7 +51,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 @RunWith(Parameterized.class)
-public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogger implements PactTaskContext<S, OUT> {
+public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogger implements TaskContext<S, OUT> {
 	
 	protected static final int PAGE_SIZE = 32 * 1024;
 	
@@ -81,7 +81,7 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
 	
 	private S stub;
 	
-	private PactDriver<S, IN> driver;
+	private Driver<S, IN> driver;
 	
 	private volatile boolean running = true;
 	
@@ -176,12 +176,12 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
 	}
 	
 	@SuppressWarnings("rawtypes")
-	public void testDriver(PactDriver driver, Class stubClass) throws Exception {
+	public void testDriver(Driver driver, Class stubClass) throws Exception {
 		testDriverInternal(driver, stubClass);
 	}
 	
 	@SuppressWarnings({"unchecked", "rawtypes"})
-	public void testDriverInternal(PactDriver driver, Class stubClass) throws Exception {
+	public void testDriverInternal(Driver driver, Class stubClass) throws Exception {
 		
 		this.driver = driver;
 		driver.setup(this);
@@ -232,8 +232,8 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
 			}
 			
 			// if resettable driver invoke tear down
-			if (this.driver instanceof ResettablePactDriver) {
-				final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
+			if (this.driver instanceof ResettableDriver) {
+				final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
 				try {
 					resDriver.teardown();
 				} catch (Throwable t) {
@@ -252,7 +252,7 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
 	}
 	
 	@SuppressWarnings({"unchecked", "rawtypes"})
-	public void testResettableDriver(ResettablePactDriver driver, Class stubClass, int iterations) throws Exception {
+	public void testResettableDriver(ResettableDriver driver, Class stubClass, int iterations) throws Exception {
 		driver.setup(this);
 		
 		for (int i = 0; i < iterations; i++) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
index 116fdec..c442940 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
@@ -24,6 +24,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.runtime.operators.Driver;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
@@ -38,9 +39,8 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.PactDriver;
-import org.apache.flink.runtime.operators.PactTaskContext;
-import org.apache.flink.runtime.operators.ResettablePactDriver;
+import org.apache.flink.runtime.operators.TaskContext;
+import org.apache.flink.runtime.operators.ResettableDriver;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.types.Record;
@@ -51,7 +51,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 @RunWith(Parameterized.class)
-public class DriverTestBase<S extends Function> extends TestLogger implements PactTaskContext<S, Record> {
+public class DriverTestBase<S extends Function> extends TestLogger implements TaskContext<S, Record> {
 	
 	protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024;
 	
@@ -83,7 +83,7 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Pa
 	
 	private S stub;
 	
-	private PactDriver<S, Record> driver;
+	private Driver<S, Record> driver;
 	
 	private volatile boolean running = true;
 
@@ -168,12 +168,12 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Pa
 	}
 
 	@SuppressWarnings("rawtypes")
-	public void testDriver(PactDriver driver, Class stubClass) throws Exception {
+	public void testDriver(Driver driver, Class stubClass) throws Exception {
 		testDriverInternal(driver, stubClass);
 	}
 
 	@SuppressWarnings({"unchecked","rawtypes"})
-	public void testDriverInternal(PactDriver driver, Class stubClass) throws Exception {
+	public void testDriverInternal(Driver driver, Class stubClass) throws Exception {
 
 		this.driver = driver;
 		driver.setup(this);
@@ -226,8 +226,8 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Pa
 			}
 
 			// if resettable driver invoke tear down
-			if (this.driver instanceof ResettablePactDriver) {
-				final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
+			if (this.driver instanceof ResettableDriver) {
+				final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
 				try {
 					resDriver.teardown();
 				} catch (Throwable t) {
@@ -247,7 +247,7 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Pa
 	}
 
 	@SuppressWarnings({"unchecked","rawtypes"})
-	public void testResettableDriver(ResettablePactDriver driver, Class stubClass, int iterations) throws Exception {
+	public void testResettableDriver(ResettableDriver driver, Class stubClass, int iterations) throws Exception {
 
 		driver.setup(this);
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
index 4662762..777bfc8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
@@ -30,7 +30,7 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.io.network.partition.consumer.IteratorWrappingTestSingleInputGate;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.PactDriver;
+import org.apache.flink.runtime.operators.Driver;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.types.Record;
@@ -89,7 +89,7 @@ public abstract class TaskTestBase extends TestLogger {
 	}
 
 	public void registerTask(AbstractInvokable task, 
-								@SuppressWarnings("rawtypes") Class<? extends PactDriver> driver,
+								@SuppressWarnings("rawtypes") Class<? extends Driver> driver,
 								Class<? extends RichFunction> stubClass) {
 		
 		final TaskConfig config = new TaskConfig(this.mockEnv.getTaskConfiguration());

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
index e2b2430..886c881 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
@@ -30,9 +30,9 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.PactDriver;
-import org.apache.flink.runtime.operators.PactTaskContext;
-import org.apache.flink.runtime.operators.ResettablePactDriver;
+import org.apache.flink.runtime.operators.Driver;
+import org.apache.flink.runtime.operators.TaskContext;
+import org.apache.flink.runtime.operators.ResettableDriver;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
@@ -51,7 +51,7 @@ import java.util.Collection;
 import java.util.List;
 
 @RunWith(Parameterized.class)
-public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogger implements PactTaskContext<S, OUT> {
+public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogger implements TaskContext<S, OUT> {
 	
 	protected static final long DEFAULT_PER_SORT_MEM = 16 * 1024 * 1024;
 	
@@ -85,7 +85,7 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
 	
 	private S stub;
 	
-	private PactDriver<S, OUT> driver;
+	private Driver<S, OUT> driver;
 	
 	private volatile boolean running;
 
@@ -170,12 +170,12 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
 	}
 
 	@SuppressWarnings("rawtypes")
-	public void testDriver(PactDriver driver, Class stubClass) throws Exception {
+	public void testDriver(Driver driver, Class stubClass) throws Exception {
 		testDriverInternal(driver, stubClass);
 	}
 
 	@SuppressWarnings({"unchecked","rawtypes"})
-	public void testDriverInternal(PactDriver driver, Class stubClass) throws Exception {
+	public void testDriverInternal(Driver driver, Class stubClass) throws Exception {
 
 		this.driver = driver;
 		driver.setup(this);
@@ -227,8 +227,8 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
 			}
 
 			// if resettable driver invoke tear-down
-			if (this.driver instanceof ResettablePactDriver) {
-				final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
+			if (this.driver instanceof ResettableDriver) {
+				final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
 				try {
 					resDriver.teardown();
 				} catch (Throwable t) {
@@ -248,7 +248,7 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
 	}
 
 	@SuppressWarnings({"unchecked","rawtypes"})
-	public void testResettableDriver(ResettablePactDriver driver, Class stubClass, int iterations) throws Exception {
+	public void testResettableDriver(ResettableDriver driver, Class stubClass, int iterations) throws Exception {
 		driver.setup(this);
 		
 		for (int i = 0; i < iterations; i++) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/logback-test.xml b/flink-runtime/src/test/resources/logback-test.xml
index 17f7020..1d64d46 100644
--- a/flink-runtime/src/test/resources/logback-test.xml
+++ b/flink-runtime/src/test/resources/logback-test.xml
@@ -31,7 +31,7 @@
          throw error to test failing scenarios. Logging those would overflow the log. -->
          <!---->
     <logger name="org.apache.flink.runtime.operators.DataSinkTask" level="OFF"/>
-    <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
+    <logger name="org.apache.flink.runtime.operators.BatchTask" level="OFF"/>
     <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-staging/flink-ml/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-ml/src/test/resources/logback-test.xml b/flink-staging/flink-ml/src/test/resources/logback-test.xml
index 17f7020..1d64d46 100644
--- a/flink-staging/flink-ml/src/test/resources/logback-test.xml
+++ b/flink-staging/flink-ml/src/test/resources/logback-test.xml
@@ -31,7 +31,7 @@
          throw error to test failing scenarios. Logging those would overflow the log. -->
          <!---->
     <logger name="org.apache.flink.runtime.operators.DataSinkTask" level="OFF"/>
-    <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
+    <logger name="org.apache.flink.runtime.operators.BatchTask" level="OFF"/>
     <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/RegularProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/RegularProcessor.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/RegularProcessor.java
index b117bab..287129d 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/RegularProcessor.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/RegularProcessor.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.operators.PactDriver;
+import org.apache.flink.runtime.operators.Driver;
 import org.apache.flink.api.common.functions.util.RuntimeUDFContext;
 import org.apache.flink.tez.util.EncodingUtils;
 import org.apache.flink.util.InstantiationUtil;
@@ -93,8 +93,8 @@ public class RegularProcessor<S extends Function, OT> extends AbstractLogicalIOP
 
 		this.inputs = inputs;
 		this.outputs = outputs;
-		final Class<? extends PactDriver<S, OT>> driverClass = this.task.getTaskConfig().getDriver();
-		PactDriver<S,OT> driver = InstantiationUtil.instantiate(driverClass, PactDriver.class);
+		final Class<? extends Driver<S, OT>> driverClass = this.task.getTaskConfig().getDriver();
+		Driver<S,OT> driver = InstantiationUtil.instantiate(driverClass, Driver.class);
 		this.numInputs = driver.getNumberOfInputs();
 		this.numOutputs = outputs.size();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
index b7cbfb4..89e4642 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
@@ -36,8 +36,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.PactDriver;
-import org.apache.flink.runtime.operators.PactTaskContext;
+import org.apache.flink.runtime.operators.Driver;
+import org.apache.flink.runtime.operators.TaskContext;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
@@ -64,7 +64,7 @@ import java.util.Arrays;
 import java.util.List;
 
 
-public class TezTask<S extends Function,OT>  implements PactTaskContext<S, OT> {
+public class TezTask<S extends Function,OT>  implements TaskContext<S, OT> {
 
 	protected static final Log LOG = LogFactory.getLog(TezTask.class);
 
@@ -74,7 +74,7 @@ public class TezTask<S extends Function,OT>  implements PactTaskContext<S, OT> {
 	 * The driver that invokes the user code (the stub implementation). The central driver in this task
 	 * (further drivers may be chained behind this driver).
 	 */
-	protected volatile PactDriver<S, OT> driver;
+	protected volatile Driver<S, OT> driver;
 
 	/**
 	 * The instantiated user code of this task's main operator (driver). May be null if the operator has no udf.
@@ -150,8 +150,8 @@ public class TezTask<S extends Function,OT>  implements PactTaskContext<S, OT> {
 
 	public TezTask(TezTaskConfig config, RuntimeUDFContext runtimeUdfContext, long availableMemory) {
 		this.config = config;
-		final Class<? extends PactDriver<S, OT>> driverClass = this.config.getDriver();
-		this.driver = InstantiationUtil.instantiate(driverClass, PactDriver.class);
+		final Class<? extends Driver<S, OT>> driverClass = this.config.getDriver();
+		this.driver = InstantiationUtil.instantiate(driverClass, Driver.class);
 		
 		LOG.info("ClassLoader URLs: " + Arrays.toString(((URLClassLoader) this.userCodeClassLoader).getURLs()));
 		
@@ -244,7 +244,7 @@ public class TezTask<S extends Function,OT>  implements PactTaskContext<S, OT> {
 
 
 	// --------------------------------------------------------------------
-	// PactTaskContext interface
+	// TaskContext interface
 	// --------------------------------------------------------------------
 
 	@Override
@@ -356,7 +356,7 @@ public class TezTask<S extends Function,OT>  implements PactTaskContext<S, OT> {
 
 
 	// --------------------------------------------------------------------
-	// Adapted from RegularPactTask
+	// Adapted from BatchTask
 	// --------------------------------------------------------------------
 
 	private void initInputLocalStrategy(int inputNum) throws Exception {
@@ -402,7 +402,7 @@ public class TezTask<S extends Function,OT>  implements PactTaskContext<S, OT> {
 						localStub = initStub(userCodeFunctionType);
 					} catch (Exception e) {
 						throw new RuntimeException("Initializing the user code and the configuration failed" +
-								e.getMessage() == null ? "." : ": " + e.getMessage(), e);
+								(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
 					}
 
 					if (!(localStub instanceof GroupCombineFunction)) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-staging/flink-tez/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/test/resources/logback-test.xml b/flink-staging/flink-tez/src/test/resources/logback-test.xml
index 9c2e75f..48e4374 100644
--- a/flink-staging/flink-tez/src/test/resources/logback-test.xml
+++ b/flink-staging/flink-tez/src/test/resources/logback-test.xml
@@ -27,7 +27,7 @@
         <appender-ref ref="STDOUT"/>
     </root>
 
-    <!--<logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.operators.BatchTask" level="OFF"/>-->
     <!--<logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>-->
     <!--<logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>-->
     <!--<logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>-->

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-tests/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/logback-test.xml b/flink-tests/src/test/resources/logback-test.xml
index 9c2e75f..48e4374 100644
--- a/flink-tests/src/test/resources/logback-test.xml
+++ b/flink-tests/src/test/resources/logback-test.xml
@@ -27,7 +27,7 @@
         <appender-ref ref="STDOUT"/>
     </root>
 
-    <!--<logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.operators.BatchTask" level="OFF"/>-->
     <!--<logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>-->
     <!--<logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>-->
     <!--<logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>-->


[2/5] flink git commit: [FLINK-2815] [REFACTOR] Remove Pact from class and file names since it is no longer valid reference

Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
index c6a872c..988e903 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
@@ -55,14 +55,14 @@ import java.util.List;
  * @param <IN> The data type consumed by the combiner.
  * @param <OUT> The data type produced by the combiner.
  */
-public class GroupReduceCombineDriver<IN, OUT> implements PactDriver<GroupCombineFunction<IN, OUT>, OUT> {
+public class GroupReduceCombineDriver<IN, OUT> implements Driver<GroupCombineFunction<IN, OUT>, OUT> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(GroupReduceCombineDriver.class);
 
 	/** Fix length records with a length below this threshold will be in-place sorted, if possible. */
 	private static final int THRESHOLD_FOR_IN_PLACE_SORTING = 32;
 
-	private PactTaskContext<GroupCombineFunction<IN, OUT>, OUT> taskContext;
+	private TaskContext<GroupCombineFunction<IN, OUT>, OUT> taskContext;
 
 	private InMemorySorter<IN> sorter;
 
@@ -87,7 +87,7 @@ public class GroupReduceCombineDriver<IN, OUT> implements PactDriver<GroupCombin
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<GroupCombineFunction<IN, OUT>, OUT> context) {
+	public void setup(TaskContext<GroupCombineFunction<IN, OUT>, OUT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java
index 59fb603..a03e42d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java
@@ -40,11 +40,11 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @see org.apache.flink.api.common.functions.GroupReduceFunction
  */
-public class GroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunction<IT, OT>, OT> {
+public class GroupReduceDriver<IT, OT> implements Driver<GroupReduceFunction<IT, OT>, OT> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(GroupReduceDriver.class);
 
-	private PactTaskContext<GroupReduceFunction<IT, OT>, OT> taskContext;
+	private TaskContext<GroupReduceFunction<IT, OT>, OT> taskContext;
 	
 	private MutableObjectIterator<IT> input;
 
@@ -59,7 +59,7 @@ public class GroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunction
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<GroupReduceFunction<IT, OT>, OT> context) {
+	public void setup(TaskContext<GroupReduceFunction<IT, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
index 811f00c..7a9c8e6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
@@ -46,11 +46,11 @@ import org.slf4j.LoggerFactory;
  * 
  * @see org.apache.flink.api.common.functions.FlatJoinFunction
  */
-public class JoinDriver<IT1, IT2, OT> implements PactDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
+public class JoinDriver<IT1, IT2, OT> implements Driver<FlatJoinFunction<IT1, IT2, OT>, OT> {
 	
 	protected static final Logger LOG = LoggerFactory.getLogger(JoinDriver.class);
 	
-	protected PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
+	protected TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	private volatile JoinTaskIterator<IT1, IT2, OT> joinIterator; // the iterator that does the actual join 
 	
@@ -59,7 +59,7 @@ public class JoinDriver<IT1, IT2, OT> implements PactDriver<FlatJoinFunction<IT1
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
index fe926cb..51f9197 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
@@ -27,15 +27,15 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
-import org.apache.flink.runtime.iterative.task.AbstractIterativePactTask;
+import org.apache.flink.runtime.iterative.task.AbstractIterativeTask;
 import org.apache.flink.runtime.operators.hash.CompactingHashTable;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
-public class JoinWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettablePactDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
+public class JoinWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettableDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
 	
-	private PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
+	private TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	private CompactingHashTable<IT1> hashTable;
 	
@@ -55,7 +55,7 @@ public class JoinWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettableP
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
-	public void setup(PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}
@@ -99,8 +99,8 @@ public class JoinWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettableP
 		final TypeComparator<IT1> solutionSetComparator;
 		
 		// grab a handle to the hash table from the iteration broker
-		if (taskContext instanceof AbstractIterativePactTask) {
-			AbstractIterativePactTask<?, ?> iterativeTaskContext = (AbstractIterativePactTask<?, ?>) taskContext;
+		if (taskContext instanceof AbstractIterativeTask) {
+			AbstractIterativeTask<?, ?> iterativeTaskContext = (AbstractIterativeTask<?, ?>) taskContext;
 			String identifier = iterativeTaskContext.brokerKey();
 			
 			Object table = SolutionSetBroker.instance().get(identifier);

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
index 20079fc..e1fad47 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
@@ -27,15 +27,15 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
-import org.apache.flink.runtime.iterative.task.AbstractIterativePactTask;
+import org.apache.flink.runtime.iterative.task.AbstractIterativeTask;
 import org.apache.flink.runtime.operators.hash.CompactingHashTable;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
-public class JoinWithSolutionSetSecondDriver<IT1, IT2, OT> implements ResettablePactDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
+public class JoinWithSolutionSetSecondDriver<IT1, IT2, OT> implements ResettableDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
 	
-	private PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
+	private TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	private CompactingHashTable<IT2> hashTable;
 	
@@ -55,7 +55,7 @@ public class JoinWithSolutionSetSecondDriver<IT1, IT2, OT> implements Resettable
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
-	public void setup(PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}
@@ -99,8 +99,8 @@ public class JoinWithSolutionSetSecondDriver<IT1, IT2, OT> implements Resettable
 		final TypeComparator<IT2> solutionSetComparator;
 		
 		// grab a handle to the hash table from the iteration broker
-		if (taskContext instanceof AbstractIterativePactTask) {
-			AbstractIterativePactTask<?, ?> iterativeTaskContext = (AbstractIterativePactTask<?, ?>) taskContext;
+		if (taskContext instanceof AbstractIterativeTask) {
+			AbstractIterativeTask<?, ?> iterativeTaskContext = (AbstractIterativeTask<?, ?>) taskContext;
 			String identifier = iterativeTaskContext.brokerKey();
 			Object table = SolutionSetBroker.instance().get(identifier);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java
index d861cbd..eefe8e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapDriver.java
@@ -36,9 +36,9 @@ import org.apache.flink.util.MutableObjectIterator;
  * @param <IT> The mapper's input data type.
  * @param <OT> The mapper's output data type.
  */
-public class MapDriver<IT, OT> implements PactDriver<MapFunction<IT, OT>, OT> {
+public class MapDriver<IT, OT> implements Driver<MapFunction<IT, OT>, OT> {
 	
-	private PactTaskContext<MapFunction<IT, OT>, OT> taskContext;
+	private TaskContext<MapFunction<IT, OT>, OT> taskContext;
 	
 	private volatile boolean running;
 
@@ -46,7 +46,7 @@ public class MapDriver<IT, OT> implements PactDriver<MapFunction<IT, OT>, OT> {
 	
 	
 	@Override
-	public void setup(PactTaskContext<MapFunction<IT, OT>, OT> context) {
+	public void setup(TaskContext<MapFunction<IT, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java
index eaab904..8792ef1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/MapPartitionDriver.java
@@ -41,16 +41,16 @@ import org.slf4j.LoggerFactory;
  * @param <IT> The mapper's input data type.
  * @param <OT> The mapper's output data type.
  */
-public class MapPartitionDriver<IT, OT> implements PactDriver<MapPartitionFunction<IT, OT>, OT> {
+public class MapPartitionDriver<IT, OT> implements Driver<MapPartitionFunction<IT, OT>, OT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(MapPartitionDriver.class);
 
-	private PactTaskContext<MapPartitionFunction<IT, OT>, OT> taskContext;
+	private TaskContext<MapPartitionFunction<IT, OT>, OT> taskContext;
 
 	private boolean objectReuseEnabled = false;
 
 	@Override
-	public void setup(PactTaskContext<MapPartitionFunction<IT, OT>, OT> context) {
+	public void setup(TaskContext<MapPartitionFunction<IT, OT>, OT> context) {
 		this.taskContext = context;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java
index 1fb4813..fcd2716 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/NoOpDriver.java
@@ -32,18 +32,18 @@ import org.slf4j.LoggerFactory;
  * 
  * @param <T> The data type.
  */
-public class NoOpDriver<T> implements PactDriver<AbstractRichFunction, T> {
+public class NoOpDriver<T> implements Driver<AbstractRichFunction, T> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(MapPartitionDriver.class);
 
-	private PactTaskContext<AbstractRichFunction, T> taskContext;
+	private TaskContext<AbstractRichFunction, T> taskContext;
 	
 	private volatile boolean running;
 
 	private boolean objectReuseEnabled = false;
 
 	@Override
-	public void setup(PactTaskContext<AbstractRichFunction, T> context) {
+	public void setup(TaskContext<AbstractRichFunction, T> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java
deleted file mode 100644
index 288f7ca..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactDriver.java
+++ /dev/null
@@ -1,90 +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.runtime.operators;
-
-import org.apache.flink.api.common.functions.Function;
-
-/**
- * The interface to be implemented by all drivers that run alone (or as the primary driver) in a task.
- * A driver implements the actual code to perform a batch operation, like <i>map()</i>,
- * <i>reduce()</i>, <i>join()</i>, or <i>coGroup()</i>.
- *
- * @see PactTaskContext
- * 
- * @param <S> The type of stub driven by this driver.
- * @param <OT> The data type of the records produced by this driver.
- */
-public interface PactDriver<S extends Function, OT> {
-	
-	void setup(PactTaskContext<S, OT> context);
-	
-	/**
-	 * Gets the number of inputs that the task has.
-	 * 
-	 * @return The number of inputs.
-	 */
-	int getNumberOfInputs();
-	
-	/**
-	 * Gets the number of comparators required for this driver.
-	 * 
-	 * @return The number of comparators required for this driver.
-	 */
-	int getNumberOfDriverComparators();
-	
-	/**
-	 * Gets the class of the stub type that is run by this task. For example, a <tt>MapTask</tt> should return
-	 * <code>MapFunction.class</code>.   
-	 * 
-	 * @return The class of the stub type run by the task.
-	 */
-	Class<S> getStubType();
-	
-	/**
-	 * This method is called before the user code is opened. An exception thrown by this method
-	 * signals failure of the task.
-	 * 
-	 * @throws Exception Exceptions may be forwarded and signal task failure.
-	 */
-	void prepare() throws Exception;
-	
-	/**
-	 * The main operation method of the task. It should call the user code with the data subsets until
-	 * the input is depleted.
-	 * 
-	 * @throws Exception Any exception thrown by this method signals task failure. Because exceptions in the user
-	 *                   code typically signal situations where this instance in unable to proceed, exceptions
-	 *                   from the user code should be forwarded.
-	 */
-	void run() throws Exception;
-	
-	/**
-	 * This method is invoked in any case (clean termination and exception) at the end of the tasks operation.
-	 * 
-	 * @throws Exception Exceptions may be forwarded.
-	 */
-	void cleanup() throws Exception;
-	
-	/**
-	 * This method is invoked when the driver must aborted in mid processing. It is invoked asynchronously by a different thread.
-	 * 
-	 * @throws Exception Exceptions may be forwarded.
-	 */
-	void cancel() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
deleted file mode 100644
index baeda3a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.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.runtime.operators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-
-
-/**
- * The task context gives a driver (e.g., {@link MapDriver}, or {@link JoinDriver}) access to
- * the runtime components and configuration that they can use to fulfil their task.
- *
- * @param <S> The UDF type.
- * @param <OT> The produced data type.
- *
- * @see PactDriver
- */
-public interface PactTaskContext<S, OT> {
-	
-	TaskConfig getTaskConfig();
-	
-	TaskManagerRuntimeInfo getTaskManagerInfo();
-
-	ClassLoader getUserCodeClassLoader();
-	
-	MemoryManager getMemoryManager();
-	
-	IOManager getIOManager();
-
-	<X> MutableObjectIterator<X> getInput(int index);
-	
-	<X> TypeSerializerFactory<X> getInputSerializer(int index);
-	
-	<X> TypeComparator<X> getDriverComparator(int index);
-	
-	S getStub();
-
-	ExecutionConfig getExecutionConfig();
-
-	Collector<OT> getOutputCollector();
-	
-	AbstractInvokable getOwningNepheleTask();
-	
-	String formatLogString(String message);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
index f990156..c77e746 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
@@ -45,7 +45,7 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @param <T> The data type consumed and produced by the combiner.
  */
-public class ReduceCombineDriver<T> implements PactDriver<ReduceFunction<T>, T> {
+public class ReduceCombineDriver<T> implements Driver<ReduceFunction<T>, T> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(ReduceCombineDriver.class);
 
@@ -53,7 +53,7 @@ public class ReduceCombineDriver<T> implements PactDriver<ReduceFunction<T>, T>
 	private static final int THRESHOLD_FOR_IN_PLACE_SORTING = 32;
 	
 	
-	private PactTaskContext<ReduceFunction<T>, T> taskContext;
+	private TaskContext<ReduceFunction<T>, T> taskContext;
 
 	private TypeSerializer<T> serializer;
 
@@ -77,7 +77,7 @@ public class ReduceCombineDriver<T> implements PactDriver<ReduceFunction<T>, T>
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<ReduceFunction<T>, T> context) {
+	public void setup(TaskContext<ReduceFunction<T>, T> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java
index 8d15ef2..395beab 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceDriver.java
@@ -39,11 +39,11 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @see org.apache.flink.api.common.functions.ReduceFunction
  */
-public class ReduceDriver<T> implements PactDriver<ReduceFunction<T>, T> {
+public class ReduceDriver<T> implements Driver<ReduceFunction<T>, T> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(ReduceDriver.class);
 
-	private PactTaskContext<ReduceFunction<T>, T> taskContext;
+	private TaskContext<ReduceFunction<T>, T> taskContext;
 	
 	private MutableObjectIterator<T> input;
 
@@ -58,7 +58,7 @@ public class ReduceDriver<T> implements PactDriver<ReduceFunction<T>, T> {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<ReduceFunction<T>, T> context) {
+	public void setup(TaskContext<ReduceFunction<T>, T> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
deleted file mode 100644
index 89963af..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
+++ /dev/null
@@ -1,1499 +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.runtime.operators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.distributions.DataDistribution;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.broadcast.BroadcastVariableMaterialization;
-import org.apache.flink.runtime.execution.CancelTaskException;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.reader.MutableReader;
-import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader;
-import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
-import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.chaining.ChainedDriver;
-import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
-import org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator;
-import org.apache.flink.runtime.operators.shipping.OutputCollector;
-import org.apache.flink.runtime.operators.shipping.OutputEmitter;
-import org.apache.flink.runtime.operators.shipping.RecordOutputCollector;
-import org.apache.flink.runtime.operators.shipping.RecordOutputEmitter;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger;
-import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
-import org.apache.flink.runtime.operators.util.CloseableInputProvider;
-import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.runtime.operators.util.ReaderIterator;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.MutableObjectIterator;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * The base class for all tasks. Encapsulated common behavior and implements the main life-cycle
- * of the user code.
- */
-public class RegularPactTask<S extends Function, OT> extends AbstractInvokable implements PactTaskContext<S, OT> {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(RegularPactTask.class);
-	
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * The driver that invokes the user code (the stub implementation). The central driver in this task
-	 * (further drivers may be chained behind this driver).
-	 */
-	protected volatile PactDriver<S, OT> driver;
-
-	/**
-	 * The instantiated user code of this task's main operator (driver). May be null if the operator has no udf.
-	 */
-	protected S stub;
-
-	/**
-	 * The udf's runtime context.
-	 */
-	protected DistributedRuntimeUDFContext runtimeUdfContext;
-
-	/**
-	 * The collector that forwards the user code's results. May forward to a channel or to chained drivers within
-	 * this task.
-	 */
-	protected Collector<OT> output;
-
-	/**
-	 * The output writers for the data that this task forwards to the next task. The latest driver (the central, if no chained
-	 * drivers exist, otherwise the last chained driver) produces its output to these writers.
-	 */
-	protected List<RecordWriter<?>> eventualOutputs;
-
-	/**
-	 * The input readers of this task.
-	 */
-	protected MutableReader<?>[] inputReaders;
-
-	/**
-	 * The input readers for the configured broadcast variables for this task.
-	 */
-	protected MutableReader<?>[] broadcastInputReaders;
-	
-	/**
-	 * The inputs reader, wrapped in an iterator. Prior to the local strategies, etc...
-	 */
-	protected MutableObjectIterator<?>[] inputIterators;
-
-	/**
-	 * The indices of the iterative inputs. Empty, if the task is not iterative. 
-	 */
-	protected int[] iterativeInputs;
-	
-	/**
-	 * The indices of the iterative broadcast inputs. Empty, if non of the inputs is iteratve. 
-	 */
-	protected int[] iterativeBroadcastInputs;
-	
-	/**
-	 * The local strategies that are applied on the inputs.
-	 */
-	protected volatile CloseableInputProvider<?>[] localStrategies;
-
-	/**
-	 * The optional temp barriers on the inputs for dead-lock breaking. Are
-	 * optionally resettable.
-	 */
-	protected volatile TempBarrier<?>[] tempBarriers;
-
-	/**
-	 * The resettable inputs in the case where no temp barrier is needed.
-	 */
-	protected volatile SpillingResettableMutableObjectIterator<?>[] resettableInputs;
-
-	/**
-	 * The inputs to the operator. Return the readers' data after the application of the local strategy
-	 * and the temp-table barrier.
-	 */
-	protected MutableObjectIterator<?>[] inputs;
-
-	/**
-	 * The serializers for the input data type.
-	 */
-	protected TypeSerializerFactory<?>[] inputSerializers;
-
-	/**
-	 * The serializers for the broadcast input data types.
-	 */
-	protected TypeSerializerFactory<?>[] broadcastInputSerializers;
-
-	/**
-	 * The comparators for the central driver.
-	 */
-	protected TypeComparator<?>[] inputComparators;
-
-	/**
-	 * The task configuration with the setup parameters.
-	 */
-	protected TaskConfig config;
-
-	/**
-	 * A list of chained drivers, if there are any.
-	 */
-	protected ArrayList<ChainedDriver<?, ?>> chainedTasks;
-
-	/**
-	 * Certain inputs may be excluded from resetting. For example, the initial partial solution
-	 * in an iteration head must not be reseted (it is read through the back channel), when all
-	 * others are reseted.
-	 */
-	private boolean[] excludeFromReset;
-
-	/**
-	 * Flag indicating for each input whether it is cached and can be reseted.
-	 */
-	private boolean[] inputIsCached;
-
-	/**
-	 * flag indicating for each input whether it must be asynchronously materialized.
-	 */
-	private boolean[] inputIsAsyncMaterialized;
-
-	/**
-	 * The amount of memory per input that is dedicated to the materialization.
-	 */
-	private int[] materializationMemory;
-
-	/**
-	 * The flag that tags the task as still running. Checked periodically to abort processing.
-	 */
-	protected volatile boolean running = true;
-
-	/**
-	 * The accumulator map used in the RuntimeContext.
-	 */
-	protected Map<String, Accumulator<?,?>> accumulatorMap;
-
-	// --------------------------------------------------------------------------------------------
-	//                                  Task Interface
-	// --------------------------------------------------------------------------------------------
-
-
-	/**
-	 * Initialization method. Runs in the execution graph setup phase in the JobManager
-	 * and as a setup method on the TaskManager.
-	 */
-	@Override
-	public void registerInputOutput() throws Exception {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Start registering input and output."));
-		}
-
-		// obtain task configuration (including stub parameters)
-		Configuration taskConf = getTaskConfiguration();
-		this.config = new TaskConfig(taskConf);
-
-		// now get the operator class which drives the operation
-		final Class<? extends PactDriver<S, OT>> driverClass = this.config.getDriver();
-		this.driver = InstantiationUtil.instantiate(driverClass, PactDriver.class);
-
-		// initialize the readers.
-		// this does not yet trigger any stream consuming or processing.
-		initInputReaders();
-		initBroadcastInputReaders();
-
-		// initialize the writers.
-		initOutputs();
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Finished registering input and output."));
-		}
-	}
-
-
-	/**
-	 * The main work method.
-	 */
-	@Override
-	public void invoke() throws Exception {
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Start task code."));
-		}
-
-		Environment env = getEnvironment();
-
-		this.runtimeUdfContext = createRuntimeContext(env.getTaskName());
-
-		// whatever happens in this scope, make sure that the local strategies are cleaned up!
-		// note that the initialization of the local strategies is in the try-finally block as well,
-		// so that the thread that creates them catches its own errors that may happen in that process.
-		// this is especially important, since there may be asynchronous closes (such as through canceling).
-		try {
-			// initialize the remaining data structures on the input and trigger the local processing
-			// the local processing includes building the dams / caches
-			try {
-				int numInputs = driver.getNumberOfInputs();
-				int numComparators = driver.getNumberOfDriverComparators();
-				int numBroadcastInputs = this.config.getNumBroadcastInputs();
-				
-				initInputsSerializersAndComparators(numInputs, numComparators);
-				initBroadcastInputsSerializers(numBroadcastInputs);
-				
-				// set the iterative status for inputs and broadcast inputs
-				{
-					List<Integer> iterativeInputs = new ArrayList<Integer>();
-					
-					for (int i = 0; i < numInputs; i++) {
-						final int numberOfEventsUntilInterrupt = getTaskConfig().getNumberOfEventsUntilInterruptInIterativeGate(i);
-			
-						if (numberOfEventsUntilInterrupt < 0) {
-							throw new IllegalArgumentException();
-						}
-						else if (numberOfEventsUntilInterrupt > 0) {
-							this.inputReaders[i].setIterativeReader();
-							iterativeInputs.add(i);
-				
-							if (LOG.isDebugEnabled()) {
-								LOG.debug(formatLogString("Input [" + i + "] reads in supersteps with [" +
-										+ numberOfEventsUntilInterrupt + "] event(s) till next superstep."));
-							}
-						}
-					}
-					this.iterativeInputs = asArray(iterativeInputs);
-				}
-				
-				{
-					List<Integer> iterativeBcInputs = new ArrayList<Integer>();
-					
-					for (int i = 0; i < numBroadcastInputs; i++) {
-						final int numberOfEventsUntilInterrupt = getTaskConfig().getNumberOfEventsUntilInterruptInIterativeBroadcastGate(i);
-						
-						if (numberOfEventsUntilInterrupt < 0) {
-							throw new IllegalArgumentException();
-						}
-						else if (numberOfEventsUntilInterrupt > 0) {
-							this.broadcastInputReaders[i].setIterativeReader();
-							iterativeBcInputs.add(i);
-				
-							if (LOG.isDebugEnabled()) {
-								LOG.debug(formatLogString("Broadcast input [" + i + "] reads in supersteps with [" +
-										+ numberOfEventsUntilInterrupt + "] event(s) till next superstep."));
-							}
-						}
-					}
-					this.iterativeBroadcastInputs = asArray(iterativeBcInputs);
-				}
-				
-				initLocalStrategies(numInputs);
-			}
-			catch (Exception e) {
-				throw new RuntimeException("Initializing the input processing failed" +
-						(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
-			}
-
-			if (!this.running) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(formatLogString("Task cancelled before task code was started."));
-				}
-				return;
-			}
-
-			// pre main-function initialization
-			initialize();
-
-			// read the broadcast variables. they will be released in the finally clause 
-			for (int i = 0; i < this.config.getNumBroadcastInputs(); i++) {
-				final String name = this.config.getBroadcastInputName(i);
-				readAndSetBroadcastInput(i, name, this.runtimeUdfContext, 1 /* superstep one for the start */);
-			}
-
-			// the work goes here
-			run();
-		}
-		finally {
-			// clean up in any case!
-			closeLocalStrategiesAndCaches();
-
-			clearReaders(inputReaders);
-			clearWriters(eventualOutputs);
-
-		}
-
-		if (this.running) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(formatLogString("Finished task code."));
-			}
-		} else {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(formatLogString("Task code cancelled."));
-			}
-		}
-	}
-
-	@Override
-	public void cancel() throws Exception {
-		this.running = false;
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Cancelling task code"));
-		}
-
-		try {
-			if (this.driver != null) {
-				this.driver.cancel();
-			}
-		} finally {
-			closeLocalStrategiesAndCaches();
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//                                  Main Work Methods
-	// --------------------------------------------------------------------------------------------
-
-	protected void initialize() throws Exception {
-		// create the operator
-		try {
-			this.driver.setup(this);
-		}
-		catch (Throwable t) {
-			throw new Exception("The driver setup for '" + this.getEnvironment().getTaskName() +
-				"' , caused an error: " + t.getMessage(), t);
-		}
-		
-		// instantiate the UDF
-		try {
-			final Class<? super S> userCodeFunctionType = this.driver.getStubType();
-			// if the class is null, the driver has no user code
-			if (userCodeFunctionType != null) {
-				this.stub = initStub(userCodeFunctionType);
-			}
-		} catch (Exception e) {
-			throw new RuntimeException("Initializing the UDF" +
-					(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
-		}
-	}
-	
-	protected <X> void readAndSetBroadcastInput(int inputNum, String bcVarName, DistributedRuntimeUDFContext context, int superstep) throws IOException {
-		
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Setting broadcast variable '" + bcVarName + "'" + 
-				(superstep > 1 ? ", superstep " + superstep : "")));
-		}
-		
-		@SuppressWarnings("unchecked")
-		final TypeSerializerFactory<X> serializerFactory =  (TypeSerializerFactory<X>) this.broadcastInputSerializers[inputNum];
-		
-		final MutableReader<?> reader = this.broadcastInputReaders[inputNum];
-
-		BroadcastVariableMaterialization<X, ?> variable = getEnvironment().getBroadcastVariableManager().materializeBroadcastVariable(bcVarName, superstep, this, reader, serializerFactory);
-		context.setBroadcastVariable(bcVarName, variable);
-	}
-	
-	protected void releaseBroadcastVariables(String bcVarName, int superstep, DistributedRuntimeUDFContext context) {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Releasing broadcast variable '" + bcVarName + "'" + 
-				(superstep > 1 ? ", superstep " + superstep : "")));
-		}
-		
-		getEnvironment().getBroadcastVariableManager().releaseReference(bcVarName, superstep, this);
-		context.clearBroadcastVariable(bcVarName);
-	}
-	
-
-	protected void run() throws Exception {
-		// ---------------------------- Now, the actual processing starts ------------------------
-		// check for asynchronous canceling
-		if (!this.running) {
-			return;
-		}
-
-		boolean stubOpen = false;
-
-		try {
-			// run the data preparation
-			try {
-				this.driver.prepare();
-			}
-			catch (Throwable t) {
-				// if the preparation caused an error, clean up
-				// errors during clean-up are swallowed, because we have already a root exception
-				throw new Exception("The data preparation for task '" + this.getEnvironment().getTaskName() +
-					"' , caused an error: " + t.getMessage(), t);
-			}
-
-			// check for canceling
-			if (!this.running) {
-				return;
-			}
-
-			// start all chained tasks
-			RegularPactTask.openChainedTasks(this.chainedTasks, this);
-
-			// open stub implementation
-			if (this.stub != null) {
-				try {
-					Configuration stubConfig = this.config.getStubParameters();
-					FunctionUtils.openFunction(this.stub, stubConfig);
-					stubOpen = true;
-				}
-				catch (Throwable t) {
-					throw new Exception("The user defined 'open()' method caused an exception: " + t.getMessage(), t);
-				}
-			}
-
-			// run the user code
-			this.driver.run();
-
-			// close. We close here such that a regular close throwing an exception marks a task as failed.
-			if (this.running && this.stub != null) {
-				FunctionUtils.closeFunction(this.stub);
-				stubOpen = false;
-			}
-
-			this.output.close();
-
-			// close all chained tasks letting them report failure
-			RegularPactTask.closeChainedTasks(this.chainedTasks, this);
-		}
-		catch (Exception ex) {
-			// close the input, but do not report any exceptions, since we already have another root cause
-			if (stubOpen) {
-				try {
-					FunctionUtils.closeFunction(this.stub);
-				}
-				catch (Throwable t) {
-					// do nothing
-				}
-			}
-			
-			// if resettable driver invoke teardown
-			if (this.driver instanceof ResettablePactDriver) {
-				final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
-				try {
-					resDriver.teardown();
-				} catch (Throwable t) {
-					throw new Exception("Error while shutting down an iterative operator: " + t.getMessage(), t);
-				}
-			}
-
-			RegularPactTask.cancelChainedTasks(this.chainedTasks);
-
-			ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
-
-			if (ex instanceof CancelTaskException) {
-				// forward canceling exception
-				throw ex;
-			}
-			else if (this.running) {
-				// throw only if task was not cancelled. in the case of canceling, exceptions are expected 
-				RegularPactTask.logAndThrowException(ex, this);
-			}
-		}
-		finally {
-			this.driver.cleanup();
-		}
-	}
-
-	protected void closeLocalStrategiesAndCaches() {
-		
-		// make sure that all broadcast variable references held by this task are released
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(formatLogString("Releasing all broadcast variables."));
-		}
-		
-		getEnvironment().getBroadcastVariableManager().releaseAllReferencesFromTask(this);
-		if (runtimeUdfContext != null) {
-			runtimeUdfContext.clearAllBroadcastVariables();
-		}
-		
-		// clean all local strategies and caches/pipeline breakers. 
-		
-		if (this.localStrategies != null) {
-			for (int i = 0; i < this.localStrategies.length; i++) {
-				if (this.localStrategies[i] != null) {
-					try {
-						this.localStrategies[i].close();
-					} catch (Throwable t) {
-						LOG.error("Error closing local strategy for input " + i, t);
-					}
-				}
-			}
-		}
-		if (this.tempBarriers != null) {
-			for (int i = 0; i < this.tempBarriers.length; i++) {
-				if (this.tempBarriers[i] != null) {
-					try {
-						this.tempBarriers[i].close();
-					} catch (Throwable t) {
-						LOG.error("Error closing temp barrier for input " + i, t);
-					}
-				}
-			}
-		}
-		if (this.resettableInputs != null) {
-			for (int i = 0; i < this.resettableInputs.length; i++) {
-				if (this.resettableInputs[i] != null) {
-					try {
-						this.resettableInputs[i].close();
-					} catch (Throwable t) {
-						LOG.error("Error closing cache for input " + i, t);
-					}
-				}
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//                                 Task Setup and Teardown
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * @return the last output collector in the collector chain
-	 */
-	@SuppressWarnings("unchecked")
-	protected Collector<OT> getLastOutputCollector() {
-		int numChained = this.chainedTasks.size();
-		return (numChained == 0) ? output : (Collector<OT>) chainedTasks.get(numChained - 1).getOutputCollector();
-	}
-
-	/**
-	 * Sets the last output {@link Collector} of the collector chain of this {@link RegularPactTask}.
-	 * <p>
-	 * In case of chained tasks, the output collector of the last {@link ChainedDriver} is set. Otherwise it is the
-	 * single collector of the {@link RegularPactTask}.
-	 *
-	 * @param newOutputCollector new output collector to set as last collector
-	 */
-	protected void setLastOutputCollector(Collector<OT> newOutputCollector) {
-		int numChained = this.chainedTasks.size();
-
-		if (numChained == 0) {
-			output = newOutputCollector;
-			return;
-		}
-
-		chainedTasks.get(numChained - 1).setOutputCollector(newOutputCollector);
-	}
-
-	public TaskConfig getLastTasksConfig() {
-		int numChained = this.chainedTasks.size();
-		return (numChained == 0) ? config : chainedTasks.get(numChained - 1).getTaskConfig();
-	}
-
-	protected S initStub(Class<? super S> stubSuperClass) throws Exception {
-		try {
-			ClassLoader userCodeClassLoader = getUserCodeClassLoader();
-			S stub = config.<S>getStubWrapper(userCodeClassLoader).getUserCodeObject(stubSuperClass, userCodeClassLoader);
-			// check if the class is a subclass, if the check is required
-			if (stubSuperClass != null && !stubSuperClass.isAssignableFrom(stub.getClass())) {
-				throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + 
-						stubSuperClass.getName() + "' as is required.");
-			}
-			FunctionUtils.setFunctionRuntimeContext(stub, this.runtimeUdfContext);
-			return stub;
-		}
-		catch (ClassCastException ccex) {
-			throw new Exception("The stub class is not a proper subclass of " + stubSuperClass.getName(), ccex);
-		}
-	}
-
-	/**
-	 * Creates the record readers for the number of inputs as defined by {@link #getNumTaskInputs()}.
-	 *
-	 * This method requires that the task configuration, the driver, and the user-code class loader are set.
-	 */
-	protected void initInputReaders() throws Exception {
-		final int numInputs = getNumTaskInputs();
-		final MutableReader<?>[] inputReaders = new MutableReader<?>[numInputs];
-
-		int currentReaderOffset = 0;
-
-		AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry();
-		AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter();
-
-		for (int i = 0; i < numInputs; i++) {
-			//  ---------------- create the input readers ---------------------
-			// in case where a logical input unions multiple physical inputs, create a union reader
-			final int groupSize = this.config.getGroupSize(i);
-
-			if (groupSize == 1) {
-				// non-union case
-				inputReaders[i] = new MutableRecordReader<IOReadableWritable>(getEnvironment().getInputGate(currentReaderOffset));
-			} else if (groupSize > 1){
-				// union case
-				InputGate[] readers = new InputGate[groupSize];
-				for (int j = 0; j < groupSize; ++j) {
-					readers[j] = getEnvironment().getInputGate(currentReaderOffset + j);
-				}
-				inputReaders[i] = new MutableRecordReader<IOReadableWritable>(new UnionInputGate(readers));
-			} else {
-				throw new Exception("Illegal input group size in task configuration: " + groupSize);
-			}
-
-			inputReaders[i].setReporter(reporter);
-
-			currentReaderOffset += groupSize;
-		}
-		this.inputReaders = inputReaders;
-
-		// final sanity check
-		if (currentReaderOffset != this.config.getNumInputs()) {
-			throw new Exception("Illegal configuration: Number of input gates and group sizes are not consistent.");
-		}
-	}
-
-	/**
-	 * Creates the record readers for the extra broadcast inputs as configured by {@link TaskConfig#getNumBroadcastInputs()}.
-	 *
-	 * This method requires that the task configuration, the driver, and the user-code class loader are set.
-	 */
-	protected void initBroadcastInputReaders() throws Exception {
-		final int numBroadcastInputs = this.config.getNumBroadcastInputs();
-		final MutableReader<?>[] broadcastInputReaders = new MutableReader<?>[numBroadcastInputs];
-
-		int currentReaderOffset = config.getNumInputs();
-
-		for (int i = 0; i < this.config.getNumBroadcastInputs(); i++) {
-			//  ---------------- create the input readers ---------------------
-			// in case where a logical input unions multiple physical inputs, create a union reader
-			final int groupSize = this.config.getBroadcastGroupSize(i);
-			if (groupSize == 1) {
-				// non-union case
-				broadcastInputReaders[i] = new MutableRecordReader<IOReadableWritable>(getEnvironment().getInputGate(currentReaderOffset));
-			} else if (groupSize > 1){
-				// union case
-				InputGate[] readers = new InputGate[groupSize];
-				for (int j = 0; j < groupSize; ++j) {
-					readers[j] = getEnvironment().getInputGate(currentReaderOffset + j);
-				}
-				broadcastInputReaders[i] = new MutableRecordReader<IOReadableWritable>(new UnionInputGate(readers));
-			} else {
-				throw new Exception("Illegal input group size in task configuration: " + groupSize);
-			}
-
-			currentReaderOffset += groupSize;
-		}
-		this.broadcastInputReaders = broadcastInputReaders;
-	}
-	
-	/**
-	 * Creates all the serializers and comparators.
-	 */
-	protected void initInputsSerializersAndComparators(int numInputs, int numComparators) throws Exception {
-		this.inputSerializers = new TypeSerializerFactory<?>[numInputs];
-		this.inputComparators = numComparators > 0 ? new TypeComparator<?>[numComparators] : null;
-		this.inputIterators = new MutableObjectIterator<?>[numInputs];
-
-		ClassLoader userCodeClassLoader = getUserCodeClassLoader();
-		
-		for (int i = 0; i < numInputs; i++) {
-			
-			final TypeSerializerFactory<?> serializerFactory = this.config.getInputSerializer(i, userCodeClassLoader);
-			this.inputSerializers[i] = serializerFactory;
-			
-			this.inputIterators[i] = createInputIterator(this.inputReaders[i], this.inputSerializers[i]);
-		}
-		
-		//  ---------------- create the driver's comparators ---------------------
-		for (int i = 0; i < numComparators; i++) {
-			
-			if (this.inputComparators != null) {
-				final TypeComparatorFactory<?> comparatorFactory = this.config.getDriverComparator(i, userCodeClassLoader);
-				this.inputComparators[i] = comparatorFactory.createComparator();
-			}
-		}
-	}
-	
-	/**
-	 * Creates all the serializers and iterators for the broadcast inputs.
-	 */
-	protected void initBroadcastInputsSerializers(int numBroadcastInputs) throws Exception {
-		this.broadcastInputSerializers = new TypeSerializerFactory<?>[numBroadcastInputs];
-
-		ClassLoader userCodeClassLoader = getUserCodeClassLoader();
-
-		for (int i = 0; i < numBroadcastInputs; i++) {
-			//  ---------------- create the serializer first ---------------------
-			final TypeSerializerFactory<?> serializerFactory = this.config.getBroadcastInputSerializer(i, userCodeClassLoader);
-			this.broadcastInputSerializers[i] = serializerFactory;
-		}
-	}
-
-	/**
-	 *
-	 * NOTE: This method must be invoked after the invocation of {@code #initInputReaders()} and
-	 * {@code #initInputSerializersAndComparators(int)}!
-	 *
-	 * @param numInputs
-	 */
-	protected void initLocalStrategies(int numInputs) throws Exception {
-
-		final MemoryManager memMan = getMemoryManager();
-		final IOManager ioMan = getIOManager();
-
-		this.localStrategies = new CloseableInputProvider<?>[numInputs];
-		this.inputs = new MutableObjectIterator<?>[numInputs];
-		this.excludeFromReset = new boolean[numInputs];
-		this.inputIsCached = new boolean[numInputs];
-		this.inputIsAsyncMaterialized = new boolean[numInputs];
-		this.materializationMemory = new int[numInputs];
-
-		// set up the local strategies first, such that the can work before any temp barrier is created
-		for (int i = 0; i < numInputs; i++) {
-			initInputLocalStrategy(i);
-		}
-
-		// we do another loop over the inputs, because we want to instantiate all
-		// sorters, etc before requesting the first input (as this call may block)
-
-		// we have two types of materialized inputs, and both are replayable (can act as a cache)
-		// The first variant materializes in a different thread and hence
-		// acts as a pipeline breaker. this one should only be there, if a pipeline breaker is needed.
-		// the second variant spills to the side and will not read unless the result is also consumed
-		// in a pipelined fashion.
-		this.resettableInputs = new SpillingResettableMutableObjectIterator<?>[numInputs];
-		this.tempBarriers = new TempBarrier<?>[numInputs];
-
-		for (int i = 0; i < numInputs; i++) {
-			final int memoryPages;
-			final boolean async = this.config.isInputAsynchronouslyMaterialized(i);
-			final boolean cached =  this.config.isInputCached(i);
-
-			this.inputIsAsyncMaterialized[i] = async;
-			this.inputIsCached[i] = cached;
-
-			if (async || cached) {
-				memoryPages = memMan.computeNumberOfPages(this.config.getRelativeInputMaterializationMemory(i));
-				if (memoryPages <= 0) {
-					throw new Exception("Input marked as materialized/cached, but no memory for materialization provided.");
-				}
-				this.materializationMemory[i] = memoryPages;
-			} else {
-				memoryPages = 0;
-			}
-
-			if (async) {
-				@SuppressWarnings({ "unchecked", "rawtypes" })
-				TempBarrier<?> barrier = new TempBarrier(this, getInput(i), this.inputSerializers[i], memMan, ioMan, memoryPages);
-				barrier.startReading();
-				this.tempBarriers[i] = barrier;
-				this.inputs[i] = null;
-			} else if (cached) {
-				@SuppressWarnings({ "unchecked", "rawtypes" })
-				SpillingResettableMutableObjectIterator<?> iter = new SpillingResettableMutableObjectIterator(
-					getInput(i), this.inputSerializers[i].getSerializer(), getMemoryManager(), getIOManager(), memoryPages, this);
-				this.resettableInputs[i] = iter;
-				this.inputs[i] = iter;
-			}
-		}
-	}
-
-	protected void resetAllInputs() throws Exception {
-
-		// first we need to make sure that caches consume remaining data
-		// NOTE: we need to do this before closing the local strategies
-		for (int i = 0; i < this.inputs.length; i++) {
-
-			if (this.inputIsCached[i] && this.resettableInputs[i] != null) {
-				this.resettableInputs[i].consumeAndCacheRemainingData();
-			}
-		}
-
-		// close all local-strategies. they will either get re-initialized, or we have
-		// read them now and their data is cached
-		for (int i = 0; i < this.localStrategies.length; i++) {
-			if (this.localStrategies[i] != null) {
-				this.localStrategies[i].close();
-				this.localStrategies[i] = null;
-			}
-		}
-
-		final MemoryManager memMan = getMemoryManager();
-		final IOManager ioMan = getIOManager();
-
-		// reset the caches, or re-run the input local strategy
-		for (int i = 0; i < this.inputs.length; i++) {
-			if (this.excludeFromReset[i]) {
-				if (this.tempBarriers[i] != null) {
-					this.tempBarriers[i].close();
-					this.tempBarriers[i] = null;
-				} else if (this.resettableInputs[i] != null) {
-					this.resettableInputs[i].close();
-					this.resettableInputs[i] = null;
-				}
-			} else {
-				// make sure the input is not available directly, but are lazily fetched again
-				this.inputs[i] = null;
-
-				if (this.inputIsCached[i]) {
-					if (this.tempBarriers[i] != null) {
-						this.inputs[i] = this.tempBarriers[i].getIterator();
-					} else if (this.resettableInputs[i] != null) {
-						this.resettableInputs[i].reset();
-						this.inputs[i] = this.resettableInputs[i];
-					} else {
-						throw new RuntimeException("Found a resettable input, but no temp barrier and no resettable iterator.");
-					}
-				} else {
-					// close the async barrier if there is one
-					if (this.tempBarriers[i] != null) {
-						this.tempBarriers[i].close();
-					}
-
-					// recreate the local strategy
-					initInputLocalStrategy(i);
-
-					if (this.inputIsAsyncMaterialized[i]) {
-						final int pages = this.materializationMemory[i];
-						@SuppressWarnings({ "unchecked", "rawtypes" })
-						TempBarrier<?> barrier = new TempBarrier(this, getInput(i), this.inputSerializers[i], memMan, ioMan, pages);
-						barrier.startReading();
-						this.tempBarriers[i] = barrier;
-						this.inputs[i] = null;
-					}
-				}
-			}
-		}
-	}
-
-	protected void excludeFromReset(int inputNum) {
-		this.excludeFromReset[inputNum] = true;
-	}
-
-	private void initInputLocalStrategy(int inputNum) throws Exception {
-		// check if there is already a strategy
-		if (this.localStrategies[inputNum] != null) {
-			throw new IllegalStateException();
-		}
-
-		// now set up the local strategy
-		final LocalStrategy localStrategy = this.config.getInputLocalStrategy(inputNum);
-		if (localStrategy != null) {
-			switch (localStrategy) {
-			case NONE:
-				// the input is as it is
-				this.inputs[inputNum] = this.inputIterators[inputNum];
-				break;
-			case SORT:
-				@SuppressWarnings({ "rawtypes", "unchecked" })
-				UnilateralSortMerger<?> sorter = new UnilateralSortMerger(getMemoryManager(), getIOManager(),
-					this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum),
-					this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
-					this.config.getSpillingThresholdInput(inputNum), this.getExecutionConfig().isObjectReuseEnabled());
-				// set the input to null such that it will be lazily fetched from the input strategy
-				this.inputs[inputNum] = null;
-				this.localStrategies[inputNum] = sorter;
-				break;
-			case COMBININGSORT:
-				// sanity check this special case!
-				// this still breaks a bit of the abstraction!
-				// we should have nested configurations for the local strategies to solve that
-				if (inputNum != 0) {
-					throw new IllegalStateException("Performing combining sort outside a (group)reduce task!");
-				}
-
-				// instantiate ourselves a combiner. we should not use the stub, because the sort and the
-				// subsequent (group)reduce would otherwise share it multi-threaded
-				final Class<S> userCodeFunctionType = this.driver.getStubType();
-				if (userCodeFunctionType == null) {
-					throw new IllegalStateException("Performing combining sort outside a reduce task!");
-				}
-				final S localStub;
-				try {
-					localStub = initStub(userCodeFunctionType);
-				} catch (Exception e) {
-					throw new RuntimeException("Initializing the user code and the configuration failed" +
-							(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
-				}
-				
-				if (!(localStub instanceof GroupCombineFunction)) {
-					throw new IllegalStateException("Performing combining sort outside a reduce task!");
-				}
-
-				@SuppressWarnings({ "rawtypes", "unchecked" })
-				CombiningUnilateralSortMerger<?> cSorter = new CombiningUnilateralSortMerger(
-					(GroupCombineFunction) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum],
-					this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum),
-					this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
-					this.config.getSpillingThresholdInput(inputNum), this.getExecutionConfig().isObjectReuseEnabled());
-				cSorter.setUdfConfiguration(this.config.getStubParameters());
-
-				// set the input to null such that it will be lazily fetched from the input strategy
-				this.inputs[inputNum] = null;
-				this.localStrategies[inputNum] = cSorter;
-				break;
-			default:
-				throw new Exception("Unrecognized local strategy provided: " + localStrategy.name());
-			}
-		} else {
-			// no local strategy in the config
-			this.inputs[inputNum] = this.inputIterators[inputNum];
-		}
-	}
-
-	private <T> TypeComparator<T> getLocalStrategyComparator(int inputNum) throws Exception {
-		TypeComparatorFactory<T> compFact = this.config.getInputComparator(inputNum, getUserCodeClassLoader());
-		if (compFact == null) {
-			throw new Exception("Missing comparator factory for local strategy on input " + inputNum);
-		}
-		return compFact.createComparator();
-	}
-	
-	protected MutableObjectIterator<?> createInputIterator(MutableReader<?> inputReader, TypeSerializerFactory<?> serializerFactory) {
-		@SuppressWarnings("unchecked")
-		MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		final MutableObjectIterator<?> iter = new ReaderIterator(reader, serializerFactory.getSerializer());
-		return iter;
-	}
-
-	protected int getNumTaskInputs() {
-		return this.driver.getNumberOfInputs();
-	}
-
-	/**
-	 * Creates a writer for each output. Creates an OutputCollector which forwards its input to all writers.
-	 * The output collector applies the configured shipping strategies for each writer.
-	 */
-	protected void initOutputs() throws Exception {
-		this.chainedTasks = new ArrayList<ChainedDriver<?, ?>>();
-		this.eventualOutputs = new ArrayList<RecordWriter<?>>();
-
-		ClassLoader userCodeClassLoader = getUserCodeClassLoader();
-
-		AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
-		AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter();
-
-		this.accumulatorMap = accumulatorRegistry.getUserMap();
-
-		this.output = initOutputs(this, userCodeClassLoader, this.config, this.chainedTasks, this.eventualOutputs,
-				this.getExecutionConfig(), reporter, this.accumulatorMap);
-	}
-
-	public DistributedRuntimeUDFContext createRuntimeContext(String taskName) {
-		Environment env = getEnvironment();
-
-		return new DistributedRuntimeUDFContext(taskName, env.getNumberOfSubtasks(),
-				env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(),
-				env.getDistributedCacheEntries(), this.accumulatorMap);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//                                   Task Context Signature
-	// -------------------------------------------------------------------------------------------
-
-	@Override
-	public TaskConfig getTaskConfig() {
-		return this.config;
-	}
-
-	@Override
-	public TaskManagerRuntimeInfo getTaskManagerInfo() {
-		return getEnvironment().getTaskManagerInfo();
-	}
-
-	@Override
-	public MemoryManager getMemoryManager() {
-		return getEnvironment().getMemoryManager();
-	}
-
-	@Override
-	public IOManager getIOManager() {
-		return getEnvironment().getIOManager();
-	}
-
-	@Override
-	public S getStub() {
-		return this.stub;
-	}
-
-	@Override
-	public Collector<OT> getOutputCollector() {
-		return this.output;
-	}
-
-	@Override
-	public AbstractInvokable getOwningNepheleTask() {
-		return this;
-	}
-
-	@Override
-	public String formatLogString(String message) {
-		return constructLogString(message, getEnvironment().getTaskName(), this);
-	}
-
-	@Override
-	public <X> MutableObjectIterator<X> getInput(int index) {
-		if (index < 0 || index > this.driver.getNumberOfInputs()) {
-			throw new IndexOutOfBoundsException();
-		}
-
-		// check for lazy assignment from input strategies
-		if (this.inputs[index] != null) {
-			@SuppressWarnings("unchecked")
-			MutableObjectIterator<X> in = (MutableObjectIterator<X>) this.inputs[index];
-			return in;
-		} else {
-			final MutableObjectIterator<X> in;
-			try {
-				if (this.tempBarriers[index] != null) {
-					@SuppressWarnings("unchecked")
-					MutableObjectIterator<X> iter = (MutableObjectIterator<X>) this.tempBarriers[index].getIterator();
-					in = iter;
-				} else if (this.localStrategies[index] != null) {
-					@SuppressWarnings("unchecked")
-					MutableObjectIterator<X> iter = (MutableObjectIterator<X>) this.localStrategies[index].getIterator();
-					in = iter;
-				} else {
-					throw new RuntimeException("Bug: null input iterator, null temp barrier, and null local strategy.");
-				}
-				this.inputs[index] = in;
-				return in;
-			} catch (InterruptedException iex) {
-				throw new RuntimeException("Interrupted while waiting for input " + index + " to become available.");
-			} catch (IOException ioex) {
-				throw new RuntimeException("An I/O Exception occurred while obtaining input " + index + ".");
-			}
-		}
-	}
-
-
-	@Override
-	public <X> TypeSerializerFactory<X> getInputSerializer(int index) {
-		if (index < 0 || index >= this.driver.getNumberOfInputs()) {
-			throw new IndexOutOfBoundsException();
-		}
-
-		@SuppressWarnings("unchecked")
-		final TypeSerializerFactory<X> serializerFactory = (TypeSerializerFactory<X>) this.inputSerializers[index];
-		return serializerFactory;
-	}
-
-
-	@Override
-	public <X> TypeComparator<X> getDriverComparator(int index) {
-		if (this.inputComparators == null) {
-			throw new IllegalStateException("Comparators have not been created!");
-		}
-		else if (index < 0 || index >= this.driver.getNumberOfDriverComparators()) {
-			throw new IndexOutOfBoundsException();
-		}
-
-		@SuppressWarnings("unchecked")
-		final TypeComparator<X> comparator = (TypeComparator<X>) this.inputComparators[index];
-		return comparator;
-	}
-
-	// ============================================================================================
-	//                                     Static Utilities
-	//
-	//            Utilities are consolidated here to ensure a uniform way of running,
-	//                   logging, exception handling, and error messages.
-	// ============================================================================================
-
-	// --------------------------------------------------------------------------------------------
-	//                                       Logging
-	// --------------------------------------------------------------------------------------------
-	/**
-	 * Utility function that composes a string for logging purposes. The string includes the given message,
-	 * the given name of the task and the index in its subtask group as well as the number of instances
-	 * that exist in its subtask group.
-	 *
-	 * @param message The main message for the log.
-	 * @param taskName The name of the task.
-	 * @param parent The nephele task that contains the code producing the message.
-	 *
-	 * @return The string for logging.
-	 */
-	public static String constructLogString(String message, String taskName, AbstractInvokable parent) {
-		return message + ":  " + taskName + " (" + (parent.getEnvironment().getIndexInSubtaskGroup() + 1) +
-				'/' + parent.getEnvironment().getNumberOfSubtasks() + ')';
-	}
-
-	/**
-	 * Prints an error message and throws the given exception. If the exception is of the type
-	 * {@link ExceptionInChainedStubException} then the chain of contained exceptions is followed
-	 * until an exception of a different type is found.
-	 *
-	 * @param ex The exception to be thrown.
-	 * @param parent The parent task, whose information is included in the log message.
-	 * @throws Exception Always thrown.
-	 */
-	public static void logAndThrowException(Exception ex, AbstractInvokable parent) throws Exception {
-		String taskName;
-		if (ex instanceof ExceptionInChainedStubException) {
-			do {
-				ExceptionInChainedStubException cex = (ExceptionInChainedStubException) ex;
-				taskName = cex.getTaskName();
-				ex = cex.getWrappedException();
-			} while (ex instanceof ExceptionInChainedStubException);
-		} else {
-			taskName = parent.getEnvironment().getTaskName();
-		}
-
-		if (LOG.isErrorEnabled()) {
-			LOG.error(constructLogString("Error in task code", taskName, parent), ex);
-		}
-
-		throw ex;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//                             Result Shipping and Chained Tasks
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Creates the {@link Collector} for the given task, as described by the given configuration. The
-	 * output collector contains the writers that forward the data to the different tasks that the given task
-	 * is connected to. Each writer applies a the partitioning as described in the configuration.
-	 *
-	 * @param task The task that the output collector is created for.
-	 * @param config The configuration describing the output shipping strategies.
-	 * @param cl The classloader used to load user defined types.
-	 * @param eventualOutputs The output writers that this task forwards to the next task for each output.
-	 * @param outputOffset The offset to start to get the writers for the outputs
-	 * @param numOutputs The number of outputs described in the configuration.
-	 *
-	 * @return The OutputCollector that data produced in this task is submitted to.
-	 */
-	public static <T> Collector<T> getOutputCollector(AbstractInvokable task, TaskConfig config, ClassLoader cl,
-			List<RecordWriter<?>> eventualOutputs, int outputOffset, int numOutputs, AccumulatorRegistry.Reporter reporter) throws Exception
-	{
-		if (numOutputs == 0) {
-			return null;
-		}
-
-		// get the factory for the serializer
-		final TypeSerializerFactory<T> serializerFactory = config.getOutputSerializer(cl);
-
-		// special case the Record
-		if (serializerFactory.getDataType().equals(Record.class)) {
-			final List<RecordWriter<Record>> writers = new ArrayList<RecordWriter<Record>>(numOutputs);
-
-			// create a writer for each output
-			for (int i = 0; i < numOutputs; i++) {
-				// create the OutputEmitter from output ship strategy
-				final ShipStrategyType strategy = config.getOutputShipStrategy(i);
-				final TypeComparatorFactory<?> compFact = config.getOutputComparator(i, cl);
-				final RecordOutputEmitter oe;
-				if (compFact == null) {
-					oe = new RecordOutputEmitter(strategy);
-				} else {
-					@SuppressWarnings("unchecked")
-					TypeComparator<Record> comparator = (TypeComparator<Record>) compFact.createComparator();
-					if (!comparator.supportsCompareAgainstReference()) {
-						throw new Exception("Incompatibe serializer-/comparator factories.");
-					}
-					final DataDistribution distribution = config.getOutputDataDistribution(i, cl);
-					final Partitioner<?> partitioner = config.getOutputPartitioner(i, cl);
-
-					oe = new RecordOutputEmitter(strategy, comparator, partitioner, distribution);
-				}
-
-				// setup accumulator counters
-				final RecordWriter<Record> recordWriter = new RecordWriter<Record>(task.getEnvironment().getWriter(outputOffset + i), oe);
-				recordWriter.setReporter(reporter);
-
-				writers.add(recordWriter);
-			}
-			if (eventualOutputs != null) {
-				eventualOutputs.addAll(writers);
-			}
-
-			@SuppressWarnings("unchecked")
-			final Collector<T> outColl = (Collector<T>) new RecordOutputCollector(writers);
-			return outColl;
-		}
-		else {
-			// generic case
-			final List<RecordWriter<SerializationDelegate<T>>> writers = new ArrayList<RecordWriter<SerializationDelegate<T>>>(numOutputs);
-
-			// create a writer for each output
-			for (int i = 0; i < numOutputs; i++)
-			{
-				// create the OutputEmitter from output ship strategy
-				final ShipStrategyType strategy = config.getOutputShipStrategy(i);
-				final TypeComparatorFactory<T> compFactory = config.getOutputComparator(i, cl);
-
-				final ChannelSelector<SerializationDelegate<T>> oe;
-				if (compFactory == null) {
-					oe = new OutputEmitter<T>(strategy);
-				}
-				else {
-					final DataDistribution dataDist = config.getOutputDataDistribution(i, cl);
-					final Partitioner<?> partitioner = config.getOutputPartitioner(i, cl);
-
-					final TypeComparator<T> comparator = compFactory.createComparator();
-					oe = new OutputEmitter<T>(strategy, comparator, partitioner, dataDist);
-				}
-
-				final RecordWriter<SerializationDelegate<T>> recordWriter =
-						new RecordWriter<SerializationDelegate<T>>(task.getEnvironment().getWriter(outputOffset + i), oe);
-
-				// setup live accumulator counters
-				recordWriter.setReporter(reporter);
-
-				writers.add(recordWriter);
-			}
-			if (eventualOutputs != null) {
-				eventualOutputs.addAll(writers);
-			}
-			return new OutputCollector<T>(writers, serializerFactory.getSerializer());
-		}
-	}
-
-	/**
-	 * Creates a writer for each output. Creates an OutputCollector which forwards its input to all writers.
-	 * The output collector applies the configured shipping strategy.
-	 */
-	@SuppressWarnings("unchecked")
-	public static <T> Collector<T> initOutputs(AbstractInvokable nepheleTask, ClassLoader cl, TaskConfig config,
-										List<ChainedDriver<?, ?>> chainedTasksTarget,
-										List<RecordWriter<?>> eventualOutputs,
-										ExecutionConfig executionConfig,
-										AccumulatorRegistry.Reporter reporter,
-										Map<String, Accumulator<?,?>> accumulatorMap)
-	throws Exception
-	{
-		final int numOutputs = config.getNumOutputs();
-
-		// check whether we got any chained tasks
-		final int numChained = config.getNumberOfChainedStubs();
-		if (numChained > 0) {
-			// got chained stubs. that means that this one may only have a single forward connection
-			if (numOutputs != 1 || config.getOutputShipStrategy(0) != ShipStrategyType.FORWARD) {
-				throw new RuntimeException("Plan Generation Bug: Found a chained stub that is not connected via an only forward connection.");
-			}
-
-			// instantiate each task
-			@SuppressWarnings("rawtypes")
-			Collector previous = null;
-			for (int i = numChained - 1; i >= 0; --i)
-			{
-				// get the task first
-				final ChainedDriver<?, ?> ct;
-				try {
-					Class<? extends ChainedDriver<?, ?>> ctc = config.getChainedTask(i);
-					ct = ctc.newInstance();
-				}
-				catch (Exception ex) {
-					throw new RuntimeException("Could not instantiate chained task driver.", ex);
-				}
-
-				// get the configuration for the task
-				final TaskConfig chainedStubConf = config.getChainedStubConfig(i);
-				final String taskName = config.getChainedTaskName(i);
-
-				if (i == numChained - 1) {
-					// last in chain, instantiate the output collector for this task
-					previous = getOutputCollector(nepheleTask, chainedStubConf, cl, eventualOutputs, 0, chainedStubConf.getNumOutputs(), reporter);
-				}
-
-				ct.setup(chainedStubConf, taskName, previous, nepheleTask, cl, executionConfig, accumulatorMap);
-				chainedTasksTarget.add(0, ct);
-
-				previous = ct;
-			}
-			// the collector of the first in the chain is the collector for the nephele task
-			return (Collector<T>) previous;
-		}
-		// else
-
-		// instantiate the output collector the default way from this configuration
-		return getOutputCollector(nepheleTask , config, cl, eventualOutputs, 0, numOutputs, reporter);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  User Code LifeCycle
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Opens the given stub using its {@link org.apache.flink.api.common.functions.RichFunction#open(Configuration)} method. If the open call produces
-	 * an exception, a new exception with a standard error message is created, using the encountered exception
-	 * as its cause.
-	 * 
-	 * @param stub The user code instance to be opened.
-	 * @param parameters The parameters supplied to the user code.
-	 * 
-	 * @throws Exception Thrown, if the user code's open method produces an exception.
-	 */
-	public static void openUserCode(Function stub, Configuration parameters) throws Exception {
-		try {
-			FunctionUtils.openFunction(stub, parameters);
-		} catch (Throwable t) {
-			throw new Exception("The user defined 'open(Configuration)' method in " + stub.getClass().toString() + " caused an exception: " + t.getMessage(), t);
-		}
-	}
-	
-	/**
-	 * Closes the given stub using its {@link org.apache.flink.api.common.functions.RichFunction#close()} method. If the close call produces
-	 * an exception, a new exception with a standard error message is created, using the encountered exception
-	 * as its cause.
-	 * 
-	 * @param stub The user code instance to be closed.
-	 * 
-	 * @throws Exception Thrown, if the user code's close method produces an exception.
-	 */
-	public static void closeUserCode(Function stub) throws Exception {
-		try {
-			FunctionUtils.closeFunction(stub);
-		} catch (Throwable t) {
-			throw new Exception("The user defined 'close()' method caused an exception: " + t.getMessage(), t);
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                               Chained Task LifeCycle
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Opens all chained tasks, in the order as they are stored in the array. The opening process
-	 * creates a standardized log info message.
-	 * 
-	 * @param tasks The tasks to be opened.
-	 * @param parent The parent task, used to obtain parameters to include in the log message.
-	 * @throws Exception Thrown, if the opening encounters an exception.
-	 */
-	public static void openChainedTasks(List<ChainedDriver<?, ?>> tasks, AbstractInvokable parent) throws Exception {
-		// start all chained tasks
-		for (int i = 0; i < tasks.size(); i++) {
-			final ChainedDriver<?, ?> task = tasks.get(i);
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(constructLogString("Start task code", task.getTaskName(), parent));
-			}
-			task.openTask();
-		}
-	}
-	
-	/**
-	 * Closes all chained tasks, in the order as they are stored in the array. The closing process
-	 * creates a standardized log info message.
-	 * 
-	 * @param tasks The tasks to be closed.
-	 * @param parent The parent task, used to obtain parameters to include in the log message.
-	 * @throws Exception Thrown, if the closing encounters an exception.
-	 */
-	public static void closeChainedTasks(List<ChainedDriver<?, ?>> tasks, AbstractInvokable parent) throws Exception {
-		for (int i = 0; i < tasks.size(); i++) {
-			final ChainedDriver<?, ?> task = tasks.get(i);
-			task.closeTask();
-			
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(constructLogString("Finished task code", task.getTaskName(), parent));
-			}
-		}
-	}
-	
-	/**
-	 * Cancels all tasks via their {@link ChainedDriver#cancelTask()} method. Any occurring exception
-	 * and error is suppressed, such that the canceling method of every task is invoked in all cases.
-	 * 
-	 * @param tasks The tasks to be canceled.
-	 */
-	public static void cancelChainedTasks(List<ChainedDriver<?, ?>> tasks) {
-		for (int i = 0; i < tasks.size(); i++) {
-			try {
-				tasks.get(i).cancelTask();
-			} catch (Throwable t) {
-				// do nothing
-			}
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                     Miscellaneous Utilities
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Instantiates a user code class from is definition in the task configuration.
-	 * The class is instantiated without arguments using the null-ary constructor. Instantiation
-	 * will fail if this constructor does not exist or is not public.
-	 * 
-	 * @param <T> The generic type of the user code class.
-	 * @param config The task configuration containing the class description.
-	 * @param cl The class loader to be used to load the class.
-	 * @param superClass The super class that the user code class extends or implements, for type checking.
-	 * 
-	 * @return An instance of the user code class.
-	 */
-	public static <T> T instantiateUserCode(TaskConfig config, ClassLoader cl, Class<? super T> superClass) {
-		try {
-			T stub = config.<T>getStubWrapper(cl).getUserCodeObject(superClass, cl);
-			// check if the class is a subclass, if the check is required
-			if (superClass != null && !superClass.isAssignableFrom(stub.getClass())) {
-				throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + 
-						superClass.getName() + "' as is required.");
-			}
-			return stub;
-		}
-		catch (ClassCastException ccex) {
-			throw new RuntimeException("The UDF class is not a proper subclass of " + superClass.getName(), ccex);
-		}
-	}
-	
-	private static int[] asArray(List<Integer> list) {
-		int[] a = new int[list.size()];
-		
-		int i = 0;
-		for (int val : list) {
-			a[i++] = val;
-		}
-		return a;
-	}
-
-	public static void clearWriters(List<RecordWriter<?>> writers) {
-		for (RecordWriter<?> writer : writers) {
-			writer.clearBuffers();
-		}
-	}
-
-	public static void clearReaders(MutableReader<?>[] readers) {
-		for (MutableReader<?> reader : readers) {
-			reader.clearBuffers();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettableDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettableDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettableDriver.java
new file mode 100644
index 0000000..0ca7994
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettableDriver.java
@@ -0,0 +1,44 @@
+/*
+ * 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.runtime.operators;
+
+import org.apache.flink.api.common.functions.Function;
+
+
+/**
+ * This interface marks a {@code Driver} as resettable, meaning that will reset part of their internal state but
+ * otherwise reuse existing data structures.
+ *
+ * @see Driver
+ * @see TaskContext
+ * 
+ * @param <S> The type of stub driven by this driver.
+ * @param <OT> The data type of the records produced by this driver.
+ */
+public interface ResettableDriver<S extends Function, OT> extends Driver<S, OT> {
+	
+	boolean isInputResettable(int inputNum);
+	
+	void initialize() throws Exception;
+	
+	void reset() throws Exception;
+	
+	void teardown() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java
deleted file mode 100644
index 85cde1b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ResettablePactDriver.java
+++ /dev/null
@@ -1,44 +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.runtime.operators;
-
-import org.apache.flink.api.common.functions.Function;
-
-
-/**
- * This interface marks a {@code PactDriver} as resettable, meaning that will reset part of their internal state but
- * otherwise reuse existing data structures.
- *
- * @see PactDriver
- * @see PactTaskContext
- * 
- * @param <S> The type of stub driven by this driver.
- * @param <OT> The data type of the records produced by this driver.
- */
-public interface ResettablePactDriver<S extends Function, OT> extends PactDriver<S, OT> {
-	
-	boolean isInputResettable(int inputNum);
-	
-	void initialize() throws Exception;
-	
-	void reset() throws Exception;
-	
-	void teardown() throws Exception;
-}


[3/5] flink git commit: [FLINK-2815] [REFACTOR] Remove Pact from class and file names since it is no longer valid reference

Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java
new file mode 100644
index 0000000..3f94109
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java
@@ -0,0 +1,1499 @@
+/*
+ * 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.runtime.operators;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.distributions.DataDistribution;
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
+import org.apache.flink.runtime.broadcast.BroadcastVariableMaterialization;
+import org.apache.flink.runtime.execution.CancelTaskException;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.api.reader.MutableReader;
+import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader;
+import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.chaining.ChainedDriver;
+import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
+import org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator;
+import org.apache.flink.runtime.operators.shipping.OutputCollector;
+import org.apache.flink.runtime.operators.shipping.OutputEmitter;
+import org.apache.flink.runtime.operators.shipping.RecordOutputCollector;
+import org.apache.flink.runtime.operators.shipping.RecordOutputEmitter;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.runtime.operators.sort.CombiningUnilateralSortMerger;
+import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
+import org.apache.flink.runtime.operators.util.CloseableInputProvider;
+import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext;
+import org.apache.flink.runtime.operators.util.LocalStrategy;
+import org.apache.flink.runtime.operators.util.ReaderIterator;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+import org.apache.flink.runtime.plugable.DeserializationDelegate;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
+import org.apache.flink.types.Record;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.MutableObjectIterator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The base class for all batch tasks. Encapsulated common behavior and implements the main life-cycle
+ * of the user code.
+ */
+public class BatchTask<S extends Function, OT> extends AbstractInvokable implements TaskContext<S, OT> {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(BatchTask.class);
+	
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * The driver that invokes the user code (the stub implementation). The central driver in this task
+	 * (further drivers may be chained behind this driver).
+	 */
+	protected volatile Driver<S, OT> driver;
+
+	/**
+	 * The instantiated user code of this task's main operator (driver). May be null if the operator has no udf.
+	 */
+	protected S stub;
+
+	/**
+	 * The udf's runtime context.
+	 */
+	protected DistributedRuntimeUDFContext runtimeUdfContext;
+
+	/**
+	 * The collector that forwards the user code's results. May forward to a channel or to chained drivers within
+	 * this task.
+	 */
+	protected Collector<OT> output;
+
+	/**
+	 * The output writers for the data that this task forwards to the next task. The latest driver (the central, if no chained
+	 * drivers exist, otherwise the last chained driver) produces its output to these writers.
+	 */
+	protected List<RecordWriter<?>> eventualOutputs;
+
+	/**
+	 * The input readers of this task.
+	 */
+	protected MutableReader<?>[] inputReaders;
+
+	/**
+	 * The input readers for the configured broadcast variables for this task.
+	 */
+	protected MutableReader<?>[] broadcastInputReaders;
+	
+	/**
+	 * The inputs reader, wrapped in an iterator. Prior to the local strategies, etc...
+	 */
+	protected MutableObjectIterator<?>[] inputIterators;
+
+	/**
+	 * The indices of the iterative inputs. Empty, if the task is not iterative. 
+	 */
+	protected int[] iterativeInputs;
+	
+	/**
+	 * The indices of the iterative broadcast inputs. Empty, if non of the inputs is iteratve. 
+	 */
+	protected int[] iterativeBroadcastInputs;
+	
+	/**
+	 * The local strategies that are applied on the inputs.
+	 */
+	protected volatile CloseableInputProvider<?>[] localStrategies;
+
+	/**
+	 * The optional temp barriers on the inputs for dead-lock breaking. Are
+	 * optionally resettable.
+	 */
+	protected volatile TempBarrier<?>[] tempBarriers;
+
+	/**
+	 * The resettable inputs in the case where no temp barrier is needed.
+	 */
+	protected volatile SpillingResettableMutableObjectIterator<?>[] resettableInputs;
+
+	/**
+	 * The inputs to the operator. Return the readers' data after the application of the local strategy
+	 * and the temp-table barrier.
+	 */
+	protected MutableObjectIterator<?>[] inputs;
+
+	/**
+	 * The serializers for the input data type.
+	 */
+	protected TypeSerializerFactory<?>[] inputSerializers;
+
+	/**
+	 * The serializers for the broadcast input data types.
+	 */
+	protected TypeSerializerFactory<?>[] broadcastInputSerializers;
+
+	/**
+	 * The comparators for the central driver.
+	 */
+	protected TypeComparator<?>[] inputComparators;
+
+	/**
+	 * The task configuration with the setup parameters.
+	 */
+	protected TaskConfig config;
+
+	/**
+	 * A list of chained drivers, if there are any.
+	 */
+	protected ArrayList<ChainedDriver<?, ?>> chainedTasks;
+
+	/**
+	 * Certain inputs may be excluded from resetting. For example, the initial partial solution
+	 * in an iteration head must not be reseted (it is read through the back channel), when all
+	 * others are reseted.
+	 */
+	private boolean[] excludeFromReset;
+
+	/**
+	 * Flag indicating for each input whether it is cached and can be reseted.
+	 */
+	private boolean[] inputIsCached;
+
+	/**
+	 * flag indicating for each input whether it must be asynchronously materialized.
+	 */
+	private boolean[] inputIsAsyncMaterialized;
+
+	/**
+	 * The amount of memory per input that is dedicated to the materialization.
+	 */
+	private int[] materializationMemory;
+
+	/**
+	 * The flag that tags the task as still running. Checked periodically to abort processing.
+	 */
+	protected volatile boolean running = true;
+
+	/**
+	 * The accumulator map used in the RuntimeContext.
+	 */
+	protected Map<String, Accumulator<?,?>> accumulatorMap;
+
+	// --------------------------------------------------------------------------------------------
+	//                                  Task Interface
+	// --------------------------------------------------------------------------------------------
+
+
+	/**
+	 * Initialization method. Runs in the execution graph setup phase in the JobManager
+	 * and as a setup method on the TaskManager.
+	 */
+	@Override
+	public void registerInputOutput() throws Exception {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Start registering input and output."));
+		}
+
+		// obtain task configuration (including stub parameters)
+		Configuration taskConf = getTaskConfiguration();
+		this.config = new TaskConfig(taskConf);
+
+		// now get the operator class which drives the operation
+		final Class<? extends Driver<S, OT>> driverClass = this.config.getDriver();
+		this.driver = InstantiationUtil.instantiate(driverClass, Driver.class);
+
+		// initialize the readers.
+		// this does not yet trigger any stream consuming or processing.
+		initInputReaders();
+		initBroadcastInputReaders();
+
+		// initialize the writers.
+		initOutputs();
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Finished registering input and output."));
+		}
+	}
+
+
+	/**
+	 * The main work method.
+	 */
+	@Override
+	public void invoke() throws Exception {
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Start task code."));
+		}
+
+		Environment env = getEnvironment();
+
+		this.runtimeUdfContext = createRuntimeContext(env.getTaskName());
+
+		// whatever happens in this scope, make sure that the local strategies are cleaned up!
+		// note that the initialization of the local strategies is in the try-finally block as well,
+		// so that the thread that creates them catches its own errors that may happen in that process.
+		// this is especially important, since there may be asynchronous closes (such as through canceling).
+		try {
+			// initialize the remaining data structures on the input and trigger the local processing
+			// the local processing includes building the dams / caches
+			try {
+				int numInputs = driver.getNumberOfInputs();
+				int numComparators = driver.getNumberOfDriverComparators();
+				int numBroadcastInputs = this.config.getNumBroadcastInputs();
+				
+				initInputsSerializersAndComparators(numInputs, numComparators);
+				initBroadcastInputsSerializers(numBroadcastInputs);
+				
+				// set the iterative status for inputs and broadcast inputs
+				{
+					List<Integer> iterativeInputs = new ArrayList<Integer>();
+					
+					for (int i = 0; i < numInputs; i++) {
+						final int numberOfEventsUntilInterrupt = getTaskConfig().getNumberOfEventsUntilInterruptInIterativeGate(i);
+			
+						if (numberOfEventsUntilInterrupt < 0) {
+							throw new IllegalArgumentException();
+						}
+						else if (numberOfEventsUntilInterrupt > 0) {
+							this.inputReaders[i].setIterativeReader();
+							iterativeInputs.add(i);
+				
+							if (LOG.isDebugEnabled()) {
+								LOG.debug(formatLogString("Input [" + i + "] reads in supersteps with [" +
+										+ numberOfEventsUntilInterrupt + "] event(s) till next superstep."));
+							}
+						}
+					}
+					this.iterativeInputs = asArray(iterativeInputs);
+				}
+				
+				{
+					List<Integer> iterativeBcInputs = new ArrayList<Integer>();
+					
+					for (int i = 0; i < numBroadcastInputs; i++) {
+						final int numberOfEventsUntilInterrupt = getTaskConfig().getNumberOfEventsUntilInterruptInIterativeBroadcastGate(i);
+						
+						if (numberOfEventsUntilInterrupt < 0) {
+							throw new IllegalArgumentException();
+						}
+						else if (numberOfEventsUntilInterrupt > 0) {
+							this.broadcastInputReaders[i].setIterativeReader();
+							iterativeBcInputs.add(i);
+				
+							if (LOG.isDebugEnabled()) {
+								LOG.debug(formatLogString("Broadcast input [" + i + "] reads in supersteps with [" +
+										+ numberOfEventsUntilInterrupt + "] event(s) till next superstep."));
+							}
+						}
+					}
+					this.iterativeBroadcastInputs = asArray(iterativeBcInputs);
+				}
+				
+				initLocalStrategies(numInputs);
+			}
+			catch (Exception e) {
+				throw new RuntimeException("Initializing the input processing failed" +
+						(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
+			}
+
+			if (!this.running) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(formatLogString("Task cancelled before task code was started."));
+				}
+				return;
+			}
+
+			// pre main-function initialization
+			initialize();
+
+			// read the broadcast variables. they will be released in the finally clause 
+			for (int i = 0; i < this.config.getNumBroadcastInputs(); i++) {
+				final String name = this.config.getBroadcastInputName(i);
+				readAndSetBroadcastInput(i, name, this.runtimeUdfContext, 1 /* superstep one for the start */);
+			}
+
+			// the work goes here
+			run();
+		}
+		finally {
+			// clean up in any case!
+			closeLocalStrategiesAndCaches();
+
+			clearReaders(inputReaders);
+			clearWriters(eventualOutputs);
+
+		}
+
+		if (this.running) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(formatLogString("Finished task code."));
+			}
+		} else {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(formatLogString("Task code cancelled."));
+			}
+		}
+	}
+
+	@Override
+	public void cancel() throws Exception {
+		this.running = false;
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Cancelling task code"));
+		}
+
+		try {
+			if (this.driver != null) {
+				this.driver.cancel();
+			}
+		} finally {
+			closeLocalStrategiesAndCaches();
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                                  Main Work Methods
+	// --------------------------------------------------------------------------------------------
+
+	protected void initialize() throws Exception {
+		// create the operator
+		try {
+			this.driver.setup(this);
+		}
+		catch (Throwable t) {
+			throw new Exception("The driver setup for '" + this.getEnvironment().getTaskName() +
+				"' , caused an error: " + t.getMessage(), t);
+		}
+		
+		// instantiate the UDF
+		try {
+			final Class<? super S> userCodeFunctionType = this.driver.getStubType();
+			// if the class is null, the driver has no user code
+			if (userCodeFunctionType != null) {
+				this.stub = initStub(userCodeFunctionType);
+			}
+		} catch (Exception e) {
+			throw new RuntimeException("Initializing the UDF" +
+					(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
+		}
+	}
+	
+	protected <X> void readAndSetBroadcastInput(int inputNum, String bcVarName, DistributedRuntimeUDFContext context, int superstep) throws IOException {
+		
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Setting broadcast variable '" + bcVarName + "'" + 
+				(superstep > 1 ? ", superstep " + superstep : "")));
+		}
+		
+		@SuppressWarnings("unchecked")
+		final TypeSerializerFactory<X> serializerFactory =  (TypeSerializerFactory<X>) this.broadcastInputSerializers[inputNum];
+		
+		final MutableReader<?> reader = this.broadcastInputReaders[inputNum];
+
+		BroadcastVariableMaterialization<X, ?> variable = getEnvironment().getBroadcastVariableManager().materializeBroadcastVariable(bcVarName, superstep, this, reader, serializerFactory);
+		context.setBroadcastVariable(bcVarName, variable);
+	}
+	
+	protected void releaseBroadcastVariables(String bcVarName, int superstep, DistributedRuntimeUDFContext context) {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Releasing broadcast variable '" + bcVarName + "'" + 
+				(superstep > 1 ? ", superstep " + superstep : "")));
+		}
+		
+		getEnvironment().getBroadcastVariableManager().releaseReference(bcVarName, superstep, this);
+		context.clearBroadcastVariable(bcVarName);
+	}
+	
+
+	protected void run() throws Exception {
+		// ---------------------------- Now, the actual processing starts ------------------------
+		// check for asynchronous canceling
+		if (!this.running) {
+			return;
+		}
+
+		boolean stubOpen = false;
+
+		try {
+			// run the data preparation
+			try {
+				this.driver.prepare();
+			}
+			catch (Throwable t) {
+				// if the preparation caused an error, clean up
+				// errors during clean-up are swallowed, because we have already a root exception
+				throw new Exception("The data preparation for task '" + this.getEnvironment().getTaskName() +
+					"' , caused an error: " + t.getMessage(), t);
+			}
+
+			// check for canceling
+			if (!this.running) {
+				return;
+			}
+
+			// start all chained tasks
+			BatchTask.openChainedTasks(this.chainedTasks, this);
+
+			// open stub implementation
+			if (this.stub != null) {
+				try {
+					Configuration stubConfig = this.config.getStubParameters();
+					FunctionUtils.openFunction(this.stub, stubConfig);
+					stubOpen = true;
+				}
+				catch (Throwable t) {
+					throw new Exception("The user defined 'open()' method caused an exception: " + t.getMessage(), t);
+				}
+			}
+
+			// run the user code
+			this.driver.run();
+
+			// close. We close here such that a regular close throwing an exception marks a task as failed.
+			if (this.running && this.stub != null) {
+				FunctionUtils.closeFunction(this.stub);
+				stubOpen = false;
+			}
+
+			this.output.close();
+
+			// close all chained tasks letting them report failure
+			BatchTask.closeChainedTasks(this.chainedTasks, this);
+		}
+		catch (Exception ex) {
+			// close the input, but do not report any exceptions, since we already have another root cause
+			if (stubOpen) {
+				try {
+					FunctionUtils.closeFunction(this.stub);
+				}
+				catch (Throwable t) {
+					// do nothing
+				}
+			}
+			
+			// if resettable driver invoke teardown
+			if (this.driver instanceof ResettableDriver) {
+				final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
+				try {
+					resDriver.teardown();
+				} catch (Throwable t) {
+					throw new Exception("Error while shutting down an iterative operator: " + t.getMessage(), t);
+				}
+			}
+
+			BatchTask.cancelChainedTasks(this.chainedTasks);
+
+			ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+			if (ex instanceof CancelTaskException) {
+				// forward canceling exception
+				throw ex;
+			}
+			else if (this.running) {
+				// throw only if task was not cancelled. in the case of canceling, exceptions are expected 
+				BatchTask.logAndThrowException(ex, this);
+			}
+		}
+		finally {
+			this.driver.cleanup();
+		}
+	}
+
+	protected void closeLocalStrategiesAndCaches() {
+		
+		// make sure that all broadcast variable references held by this task are released
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(formatLogString("Releasing all broadcast variables."));
+		}
+		
+		getEnvironment().getBroadcastVariableManager().releaseAllReferencesFromTask(this);
+		if (runtimeUdfContext != null) {
+			runtimeUdfContext.clearAllBroadcastVariables();
+		}
+		
+		// clean all local strategies and caches/pipeline breakers. 
+		
+		if (this.localStrategies != null) {
+			for (int i = 0; i < this.localStrategies.length; i++) {
+				if (this.localStrategies[i] != null) {
+					try {
+						this.localStrategies[i].close();
+					} catch (Throwable t) {
+						LOG.error("Error closing local strategy for input " + i, t);
+					}
+				}
+			}
+		}
+		if (this.tempBarriers != null) {
+			for (int i = 0; i < this.tempBarriers.length; i++) {
+				if (this.tempBarriers[i] != null) {
+					try {
+						this.tempBarriers[i].close();
+					} catch (Throwable t) {
+						LOG.error("Error closing temp barrier for input " + i, t);
+					}
+				}
+			}
+		}
+		if (this.resettableInputs != null) {
+			for (int i = 0; i < this.resettableInputs.length; i++) {
+				if (this.resettableInputs[i] != null) {
+					try {
+						this.resettableInputs[i].close();
+					} catch (Throwable t) {
+						LOG.error("Error closing cache for input " + i, t);
+					}
+				}
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                                 Task Setup and Teardown
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * @return the last output collector in the collector chain
+	 */
+	@SuppressWarnings("unchecked")
+	protected Collector<OT> getLastOutputCollector() {
+		int numChained = this.chainedTasks.size();
+		return (numChained == 0) ? output : (Collector<OT>) chainedTasks.get(numChained - 1).getOutputCollector();
+	}
+
+	/**
+	 * Sets the last output {@link Collector} of the collector chain of this {@link BatchTask}.
+	 * <p>
+	 * In case of chained tasks, the output collector of the last {@link ChainedDriver} is set. Otherwise it is the
+	 * single collector of the {@link BatchTask}.
+	 *
+	 * @param newOutputCollector new output collector to set as last collector
+	 */
+	protected void setLastOutputCollector(Collector<OT> newOutputCollector) {
+		int numChained = this.chainedTasks.size();
+
+		if (numChained == 0) {
+			output = newOutputCollector;
+			return;
+		}
+
+		chainedTasks.get(numChained - 1).setOutputCollector(newOutputCollector);
+	}
+
+	public TaskConfig getLastTasksConfig() {
+		int numChained = this.chainedTasks.size();
+		return (numChained == 0) ? config : chainedTasks.get(numChained - 1).getTaskConfig();
+	}
+
+	protected S initStub(Class<? super S> stubSuperClass) throws Exception {
+		try {
+			ClassLoader userCodeClassLoader = getUserCodeClassLoader();
+			S stub = config.<S>getStubWrapper(userCodeClassLoader).getUserCodeObject(stubSuperClass, userCodeClassLoader);
+			// check if the class is a subclass, if the check is required
+			if (stubSuperClass != null && !stubSuperClass.isAssignableFrom(stub.getClass())) {
+				throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + 
+						stubSuperClass.getName() + "' as is required.");
+			}
+			FunctionUtils.setFunctionRuntimeContext(stub, this.runtimeUdfContext);
+			return stub;
+		}
+		catch (ClassCastException ccex) {
+			throw new Exception("The stub class is not a proper subclass of " + stubSuperClass.getName(), ccex);
+		}
+	}
+
+	/**
+	 * Creates the record readers for the number of inputs as defined by {@link #getNumTaskInputs()}.
+	 *
+	 * This method requires that the task configuration, the driver, and the user-code class loader are set.
+	 */
+	protected void initInputReaders() throws Exception {
+		final int numInputs = getNumTaskInputs();
+		final MutableReader<?>[] inputReaders = new MutableReader<?>[numInputs];
+
+		int currentReaderOffset = 0;
+
+		AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry();
+		AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter();
+
+		for (int i = 0; i < numInputs; i++) {
+			//  ---------------- create the input readers ---------------------
+			// in case where a logical input unions multiple physical inputs, create a union reader
+			final int groupSize = this.config.getGroupSize(i);
+
+			if (groupSize == 1) {
+				// non-union case
+				inputReaders[i] = new MutableRecordReader<IOReadableWritable>(getEnvironment().getInputGate(currentReaderOffset));
+			} else if (groupSize > 1){
+				// union case
+				InputGate[] readers = new InputGate[groupSize];
+				for (int j = 0; j < groupSize; ++j) {
+					readers[j] = getEnvironment().getInputGate(currentReaderOffset + j);
+				}
+				inputReaders[i] = new MutableRecordReader<IOReadableWritable>(new UnionInputGate(readers));
+			} else {
+				throw new Exception("Illegal input group size in task configuration: " + groupSize);
+			}
+
+			inputReaders[i].setReporter(reporter);
+
+			currentReaderOffset += groupSize;
+		}
+		this.inputReaders = inputReaders;
+
+		// final sanity check
+		if (currentReaderOffset != this.config.getNumInputs()) {
+			throw new Exception("Illegal configuration: Number of input gates and group sizes are not consistent.");
+		}
+	}
+
+	/**
+	 * Creates the record readers for the extra broadcast inputs as configured by {@link TaskConfig#getNumBroadcastInputs()}.
+	 *
+	 * This method requires that the task configuration, the driver, and the user-code class loader are set.
+	 */
+	protected void initBroadcastInputReaders() throws Exception {
+		final int numBroadcastInputs = this.config.getNumBroadcastInputs();
+		final MutableReader<?>[] broadcastInputReaders = new MutableReader<?>[numBroadcastInputs];
+
+		int currentReaderOffset = config.getNumInputs();
+
+		for (int i = 0; i < this.config.getNumBroadcastInputs(); i++) {
+			//  ---------------- create the input readers ---------------------
+			// in case where a logical input unions multiple physical inputs, create a union reader
+			final int groupSize = this.config.getBroadcastGroupSize(i);
+			if (groupSize == 1) {
+				// non-union case
+				broadcastInputReaders[i] = new MutableRecordReader<IOReadableWritable>(getEnvironment().getInputGate(currentReaderOffset));
+			} else if (groupSize > 1){
+				// union case
+				InputGate[] readers = new InputGate[groupSize];
+				for (int j = 0; j < groupSize; ++j) {
+					readers[j] = getEnvironment().getInputGate(currentReaderOffset + j);
+				}
+				broadcastInputReaders[i] = new MutableRecordReader<IOReadableWritable>(new UnionInputGate(readers));
+			} else {
+				throw new Exception("Illegal input group size in task configuration: " + groupSize);
+			}
+
+			currentReaderOffset += groupSize;
+		}
+		this.broadcastInputReaders = broadcastInputReaders;
+	}
+	
+	/**
+	 * Creates all the serializers and comparators.
+	 */
+	protected void initInputsSerializersAndComparators(int numInputs, int numComparators) throws Exception {
+		this.inputSerializers = new TypeSerializerFactory<?>[numInputs];
+		this.inputComparators = numComparators > 0 ? new TypeComparator<?>[numComparators] : null;
+		this.inputIterators = new MutableObjectIterator<?>[numInputs];
+
+		ClassLoader userCodeClassLoader = getUserCodeClassLoader();
+		
+		for (int i = 0; i < numInputs; i++) {
+			
+			final TypeSerializerFactory<?> serializerFactory = this.config.getInputSerializer(i, userCodeClassLoader);
+			this.inputSerializers[i] = serializerFactory;
+			
+			this.inputIterators[i] = createInputIterator(this.inputReaders[i], this.inputSerializers[i]);
+		}
+		
+		//  ---------------- create the driver's comparators ---------------------
+		for (int i = 0; i < numComparators; i++) {
+			
+			if (this.inputComparators != null) {
+				final TypeComparatorFactory<?> comparatorFactory = this.config.getDriverComparator(i, userCodeClassLoader);
+				this.inputComparators[i] = comparatorFactory.createComparator();
+			}
+		}
+	}
+	
+	/**
+	 * Creates all the serializers and iterators for the broadcast inputs.
+	 */
+	protected void initBroadcastInputsSerializers(int numBroadcastInputs) throws Exception {
+		this.broadcastInputSerializers = new TypeSerializerFactory<?>[numBroadcastInputs];
+
+		ClassLoader userCodeClassLoader = getUserCodeClassLoader();
+
+		for (int i = 0; i < numBroadcastInputs; i++) {
+			//  ---------------- create the serializer first ---------------------
+			final TypeSerializerFactory<?> serializerFactory = this.config.getBroadcastInputSerializer(i, userCodeClassLoader);
+			this.broadcastInputSerializers[i] = serializerFactory;
+		}
+	}
+
+	/**
+	 *
+	 * NOTE: This method must be invoked after the invocation of {@code #initInputReaders()} and
+	 * {@code #initInputSerializersAndComparators(int)}!
+	 *
+	 * @param numInputs
+	 */
+	protected void initLocalStrategies(int numInputs) throws Exception {
+
+		final MemoryManager memMan = getMemoryManager();
+		final IOManager ioMan = getIOManager();
+
+		this.localStrategies = new CloseableInputProvider<?>[numInputs];
+		this.inputs = new MutableObjectIterator<?>[numInputs];
+		this.excludeFromReset = new boolean[numInputs];
+		this.inputIsCached = new boolean[numInputs];
+		this.inputIsAsyncMaterialized = new boolean[numInputs];
+		this.materializationMemory = new int[numInputs];
+
+		// set up the local strategies first, such that the can work before any temp barrier is created
+		for (int i = 0; i < numInputs; i++) {
+			initInputLocalStrategy(i);
+		}
+
+		// we do another loop over the inputs, because we want to instantiate all
+		// sorters, etc before requesting the first input (as this call may block)
+
+		// we have two types of materialized inputs, and both are replayable (can act as a cache)
+		// The first variant materializes in a different thread and hence
+		// acts as a pipeline breaker. this one should only be there, if a pipeline breaker is needed.
+		// the second variant spills to the side and will not read unless the result is also consumed
+		// in a pipelined fashion.
+		this.resettableInputs = new SpillingResettableMutableObjectIterator<?>[numInputs];
+		this.tempBarriers = new TempBarrier<?>[numInputs];
+
+		for (int i = 0; i < numInputs; i++) {
+			final int memoryPages;
+			final boolean async = this.config.isInputAsynchronouslyMaterialized(i);
+			final boolean cached =  this.config.isInputCached(i);
+
+			this.inputIsAsyncMaterialized[i] = async;
+			this.inputIsCached[i] = cached;
+
+			if (async || cached) {
+				memoryPages = memMan.computeNumberOfPages(this.config.getRelativeInputMaterializationMemory(i));
+				if (memoryPages <= 0) {
+					throw new Exception("Input marked as materialized/cached, but no memory for materialization provided.");
+				}
+				this.materializationMemory[i] = memoryPages;
+			} else {
+				memoryPages = 0;
+			}
+
+			if (async) {
+				@SuppressWarnings({ "unchecked", "rawtypes" })
+				TempBarrier<?> barrier = new TempBarrier(this, getInput(i), this.inputSerializers[i], memMan, ioMan, memoryPages);
+				barrier.startReading();
+				this.tempBarriers[i] = barrier;
+				this.inputs[i] = null;
+			} else if (cached) {
+				@SuppressWarnings({ "unchecked", "rawtypes" })
+				SpillingResettableMutableObjectIterator<?> iter = new SpillingResettableMutableObjectIterator(
+					getInput(i), this.inputSerializers[i].getSerializer(), getMemoryManager(), getIOManager(), memoryPages, this);
+				this.resettableInputs[i] = iter;
+				this.inputs[i] = iter;
+			}
+		}
+	}
+
+	protected void resetAllInputs() throws Exception {
+
+		// first we need to make sure that caches consume remaining data
+		// NOTE: we need to do this before closing the local strategies
+		for (int i = 0; i < this.inputs.length; i++) {
+
+			if (this.inputIsCached[i] && this.resettableInputs[i] != null) {
+				this.resettableInputs[i].consumeAndCacheRemainingData();
+			}
+		}
+
+		// close all local-strategies. they will either get re-initialized, or we have
+		// read them now and their data is cached
+		for (int i = 0; i < this.localStrategies.length; i++) {
+			if (this.localStrategies[i] != null) {
+				this.localStrategies[i].close();
+				this.localStrategies[i] = null;
+			}
+		}
+
+		final MemoryManager memMan = getMemoryManager();
+		final IOManager ioMan = getIOManager();
+
+		// reset the caches, or re-run the input local strategy
+		for (int i = 0; i < this.inputs.length; i++) {
+			if (this.excludeFromReset[i]) {
+				if (this.tempBarriers[i] != null) {
+					this.tempBarriers[i].close();
+					this.tempBarriers[i] = null;
+				} else if (this.resettableInputs[i] != null) {
+					this.resettableInputs[i].close();
+					this.resettableInputs[i] = null;
+				}
+			} else {
+				// make sure the input is not available directly, but are lazily fetched again
+				this.inputs[i] = null;
+
+				if (this.inputIsCached[i]) {
+					if (this.tempBarriers[i] != null) {
+						this.inputs[i] = this.tempBarriers[i].getIterator();
+					} else if (this.resettableInputs[i] != null) {
+						this.resettableInputs[i].reset();
+						this.inputs[i] = this.resettableInputs[i];
+					} else {
+						throw new RuntimeException("Found a resettable input, but no temp barrier and no resettable iterator.");
+					}
+				} else {
+					// close the async barrier if there is one
+					if (this.tempBarriers[i] != null) {
+						this.tempBarriers[i].close();
+					}
+
+					// recreate the local strategy
+					initInputLocalStrategy(i);
+
+					if (this.inputIsAsyncMaterialized[i]) {
+						final int pages = this.materializationMemory[i];
+						@SuppressWarnings({ "unchecked", "rawtypes" })
+						TempBarrier<?> barrier = new TempBarrier(this, getInput(i), this.inputSerializers[i], memMan, ioMan, pages);
+						barrier.startReading();
+						this.tempBarriers[i] = barrier;
+						this.inputs[i] = null;
+					}
+				}
+			}
+		}
+	}
+
+	protected void excludeFromReset(int inputNum) {
+		this.excludeFromReset[inputNum] = true;
+	}
+
+	private void initInputLocalStrategy(int inputNum) throws Exception {
+		// check if there is already a strategy
+		if (this.localStrategies[inputNum] != null) {
+			throw new IllegalStateException();
+		}
+
+		// now set up the local strategy
+		final LocalStrategy localStrategy = this.config.getInputLocalStrategy(inputNum);
+		if (localStrategy != null) {
+			switch (localStrategy) {
+			case NONE:
+				// the input is as it is
+				this.inputs[inputNum] = this.inputIterators[inputNum];
+				break;
+			case SORT:
+				@SuppressWarnings({ "rawtypes", "unchecked" })
+				UnilateralSortMerger<?> sorter = new UnilateralSortMerger(getMemoryManager(), getIOManager(),
+					this.inputIterators[inputNum], this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum),
+					this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
+					this.config.getSpillingThresholdInput(inputNum), this.getExecutionConfig().isObjectReuseEnabled());
+				// set the input to null such that it will be lazily fetched from the input strategy
+				this.inputs[inputNum] = null;
+				this.localStrategies[inputNum] = sorter;
+				break;
+			case COMBININGSORT:
+				// sanity check this special case!
+				// this still breaks a bit of the abstraction!
+				// we should have nested configurations for the local strategies to solve that
+				if (inputNum != 0) {
+					throw new IllegalStateException("Performing combining sort outside a (group)reduce task!");
+				}
+
+				// instantiate ourselves a combiner. we should not use the stub, because the sort and the
+				// subsequent (group)reduce would otherwise share it multi-threaded
+				final Class<S> userCodeFunctionType = this.driver.getStubType();
+				if (userCodeFunctionType == null) {
+					throw new IllegalStateException("Performing combining sort outside a reduce task!");
+				}
+				final S localStub;
+				try {
+					localStub = initStub(userCodeFunctionType);
+				} catch (Exception e) {
+					throw new RuntimeException("Initializing the user code and the configuration failed" +
+							(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
+				}
+				
+				if (!(localStub instanceof GroupCombineFunction)) {
+					throw new IllegalStateException("Performing combining sort outside a reduce task!");
+				}
+
+				@SuppressWarnings({ "rawtypes", "unchecked" })
+				CombiningUnilateralSortMerger<?> cSorter = new CombiningUnilateralSortMerger(
+					(GroupCombineFunction) localStub, getMemoryManager(), getIOManager(), this.inputIterators[inputNum],
+					this, this.inputSerializers[inputNum], getLocalStrategyComparator(inputNum),
+					this.config.getRelativeMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
+					this.config.getSpillingThresholdInput(inputNum), this.getExecutionConfig().isObjectReuseEnabled());
+				cSorter.setUdfConfiguration(this.config.getStubParameters());
+
+				// set the input to null such that it will be lazily fetched from the input strategy
+				this.inputs[inputNum] = null;
+				this.localStrategies[inputNum] = cSorter;
+				break;
+			default:
+				throw new Exception("Unrecognized local strategy provided: " + localStrategy.name());
+			}
+		} else {
+			// no local strategy in the config
+			this.inputs[inputNum] = this.inputIterators[inputNum];
+		}
+	}
+
+	private <T> TypeComparator<T> getLocalStrategyComparator(int inputNum) throws Exception {
+		TypeComparatorFactory<T> compFact = this.config.getInputComparator(inputNum, getUserCodeClassLoader());
+		if (compFact == null) {
+			throw new Exception("Missing comparator factory for local strategy on input " + inputNum);
+		}
+		return compFact.createComparator();
+	}
+	
+	protected MutableObjectIterator<?> createInputIterator(MutableReader<?> inputReader, TypeSerializerFactory<?> serializerFactory) {
+		@SuppressWarnings("unchecked")
+		MutableReader<DeserializationDelegate<?>> reader = (MutableReader<DeserializationDelegate<?>>) inputReader;
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		final MutableObjectIterator<?> iter = new ReaderIterator(reader, serializerFactory.getSerializer());
+		return iter;
+	}
+
+	protected int getNumTaskInputs() {
+		return this.driver.getNumberOfInputs();
+	}
+
+	/**
+	 * Creates a writer for each output. Creates an OutputCollector which forwards its input to all writers.
+	 * The output collector applies the configured shipping strategies for each writer.
+	 */
+	protected void initOutputs() throws Exception {
+		this.chainedTasks = new ArrayList<ChainedDriver<?, ?>>();
+		this.eventualOutputs = new ArrayList<RecordWriter<?>>();
+
+		ClassLoader userCodeClassLoader = getUserCodeClassLoader();
+
+		AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
+		AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter();
+
+		this.accumulatorMap = accumulatorRegistry.getUserMap();
+
+		this.output = initOutputs(this, userCodeClassLoader, this.config, this.chainedTasks, this.eventualOutputs,
+				this.getExecutionConfig(), reporter, this.accumulatorMap);
+	}
+
+	public DistributedRuntimeUDFContext createRuntimeContext(String taskName) {
+		Environment env = getEnvironment();
+
+		return new DistributedRuntimeUDFContext(taskName, env.getNumberOfSubtasks(),
+				env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(),
+				env.getDistributedCacheEntries(), this.accumulatorMap);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                                   Task Context Signature
+	// -------------------------------------------------------------------------------------------
+
+	@Override
+	public TaskConfig getTaskConfig() {
+		return this.config;
+	}
+
+	@Override
+	public TaskManagerRuntimeInfo getTaskManagerInfo() {
+		return getEnvironment().getTaskManagerInfo();
+	}
+
+	@Override
+	public MemoryManager getMemoryManager() {
+		return getEnvironment().getMemoryManager();
+	}
+
+	@Override
+	public IOManager getIOManager() {
+		return getEnvironment().getIOManager();
+	}
+
+	@Override
+	public S getStub() {
+		return this.stub;
+	}
+
+	@Override
+	public Collector<OT> getOutputCollector() {
+		return this.output;
+	}
+
+	@Override
+	public AbstractInvokable getOwningNepheleTask() {
+		return this;
+	}
+
+	@Override
+	public String formatLogString(String message) {
+		return constructLogString(message, getEnvironment().getTaskName(), this);
+	}
+
+	@Override
+	public <X> MutableObjectIterator<X> getInput(int index) {
+		if (index < 0 || index > this.driver.getNumberOfInputs()) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		// check for lazy assignment from input strategies
+		if (this.inputs[index] != null) {
+			@SuppressWarnings("unchecked")
+			MutableObjectIterator<X> in = (MutableObjectIterator<X>) this.inputs[index];
+			return in;
+		} else {
+			final MutableObjectIterator<X> in;
+			try {
+				if (this.tempBarriers[index] != null) {
+					@SuppressWarnings("unchecked")
+					MutableObjectIterator<X> iter = (MutableObjectIterator<X>) this.tempBarriers[index].getIterator();
+					in = iter;
+				} else if (this.localStrategies[index] != null) {
+					@SuppressWarnings("unchecked")
+					MutableObjectIterator<X> iter = (MutableObjectIterator<X>) this.localStrategies[index].getIterator();
+					in = iter;
+				} else {
+					throw new RuntimeException("Bug: null input iterator, null temp barrier, and null local strategy.");
+				}
+				this.inputs[index] = in;
+				return in;
+			} catch (InterruptedException iex) {
+				throw new RuntimeException("Interrupted while waiting for input " + index + " to become available.");
+			} catch (IOException ioex) {
+				throw new RuntimeException("An I/O Exception occurred while obtaining input " + index + ".");
+			}
+		}
+	}
+
+
+	@Override
+	public <X> TypeSerializerFactory<X> getInputSerializer(int index) {
+		if (index < 0 || index >= this.driver.getNumberOfInputs()) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		@SuppressWarnings("unchecked")
+		final TypeSerializerFactory<X> serializerFactory = (TypeSerializerFactory<X>) this.inputSerializers[index];
+		return serializerFactory;
+	}
+
+
+	@Override
+	public <X> TypeComparator<X> getDriverComparator(int index) {
+		if (this.inputComparators == null) {
+			throw new IllegalStateException("Comparators have not been created!");
+		}
+		else if (index < 0 || index >= this.driver.getNumberOfDriverComparators()) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		@SuppressWarnings("unchecked")
+		final TypeComparator<X> comparator = (TypeComparator<X>) this.inputComparators[index];
+		return comparator;
+	}
+
+	// ============================================================================================
+	//                                     Static Utilities
+	//
+	//            Utilities are consolidated here to ensure a uniform way of running,
+	//                   logging, exception handling, and error messages.
+	// ============================================================================================
+
+	// --------------------------------------------------------------------------------------------
+	//                                       Logging
+	// --------------------------------------------------------------------------------------------
+	/**
+	 * Utility function that composes a string for logging purposes. The string includes the given message,
+	 * the given name of the task and the index in its subtask group as well as the number of instances
+	 * that exist in its subtask group.
+	 *
+	 * @param message The main message for the log.
+	 * @param taskName The name of the task.
+	 * @param parent The nephele task that contains the code producing the message.
+	 *
+	 * @return The string for logging.
+	 */
+	public static String constructLogString(String message, String taskName, AbstractInvokable parent) {
+		return message + ":  " + taskName + " (" + (parent.getEnvironment().getIndexInSubtaskGroup() + 1) +
+				'/' + parent.getEnvironment().getNumberOfSubtasks() + ')';
+	}
+
+	/**
+	 * Prints an error message and throws the given exception. If the exception is of the type
+	 * {@link ExceptionInChainedStubException} then the chain of contained exceptions is followed
+	 * until an exception of a different type is found.
+	 *
+	 * @param ex The exception to be thrown.
+	 * @param parent The parent task, whose information is included in the log message.
+	 * @throws Exception Always thrown.
+	 */
+	public static void logAndThrowException(Exception ex, AbstractInvokable parent) throws Exception {
+		String taskName;
+		if (ex instanceof ExceptionInChainedStubException) {
+			do {
+				ExceptionInChainedStubException cex = (ExceptionInChainedStubException) ex;
+				taskName = cex.getTaskName();
+				ex = cex.getWrappedException();
+			} while (ex instanceof ExceptionInChainedStubException);
+		} else {
+			taskName = parent.getEnvironment().getTaskName();
+		}
+
+		if (LOG.isErrorEnabled()) {
+			LOG.error(constructLogString("Error in task code", taskName, parent), ex);
+		}
+
+		throw ex;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                             Result Shipping and Chained Tasks
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Creates the {@link Collector} for the given task, as described by the given configuration. The
+	 * output collector contains the writers that forward the data to the different tasks that the given task
+	 * is connected to. Each writer applies a the partitioning as described in the configuration.
+	 *
+	 * @param task The task that the output collector is created for.
+	 * @param config The configuration describing the output shipping strategies.
+	 * @param cl The classloader used to load user defined types.
+	 * @param eventualOutputs The output writers that this task forwards to the next task for each output.
+	 * @param outputOffset The offset to start to get the writers for the outputs
+	 * @param numOutputs The number of outputs described in the configuration.
+	 *
+	 * @return The OutputCollector that data produced in this task is submitted to.
+	 */
+	public static <T> Collector<T> getOutputCollector(AbstractInvokable task, TaskConfig config, ClassLoader cl,
+			List<RecordWriter<?>> eventualOutputs, int outputOffset, int numOutputs, AccumulatorRegistry.Reporter reporter) throws Exception
+	{
+		if (numOutputs == 0) {
+			return null;
+		}
+
+		// get the factory for the serializer
+		final TypeSerializerFactory<T> serializerFactory = config.getOutputSerializer(cl);
+
+		// special case the Record
+		if (serializerFactory.getDataType().equals(Record.class)) {
+			final List<RecordWriter<Record>> writers = new ArrayList<RecordWriter<Record>>(numOutputs);
+
+			// create a writer for each output
+			for (int i = 0; i < numOutputs; i++) {
+				// create the OutputEmitter from output ship strategy
+				final ShipStrategyType strategy = config.getOutputShipStrategy(i);
+				final TypeComparatorFactory<?> compFact = config.getOutputComparator(i, cl);
+				final RecordOutputEmitter oe;
+				if (compFact == null) {
+					oe = new RecordOutputEmitter(strategy);
+				} else {
+					@SuppressWarnings("unchecked")
+					TypeComparator<Record> comparator = (TypeComparator<Record>) compFact.createComparator();
+					if (!comparator.supportsCompareAgainstReference()) {
+						throw new Exception("Incompatibe serializer-/comparator factories.");
+					}
+					final DataDistribution distribution = config.getOutputDataDistribution(i, cl);
+					final Partitioner<?> partitioner = config.getOutputPartitioner(i, cl);
+
+					oe = new RecordOutputEmitter(strategy, comparator, partitioner, distribution);
+				}
+
+				// setup accumulator counters
+				final RecordWriter<Record> recordWriter = new RecordWriter<Record>(task.getEnvironment().getWriter(outputOffset + i), oe);
+				recordWriter.setReporter(reporter);
+
+				writers.add(recordWriter);
+			}
+			if (eventualOutputs != null) {
+				eventualOutputs.addAll(writers);
+			}
+
+			@SuppressWarnings("unchecked")
+			final Collector<T> outColl = (Collector<T>) new RecordOutputCollector(writers);
+			return outColl;
+		}
+		else {
+			// generic case
+			final List<RecordWriter<SerializationDelegate<T>>> writers = new ArrayList<RecordWriter<SerializationDelegate<T>>>(numOutputs);
+
+			// create a writer for each output
+			for (int i = 0; i < numOutputs; i++)
+			{
+				// create the OutputEmitter from output ship strategy
+				final ShipStrategyType strategy = config.getOutputShipStrategy(i);
+				final TypeComparatorFactory<T> compFactory = config.getOutputComparator(i, cl);
+
+				final ChannelSelector<SerializationDelegate<T>> oe;
+				if (compFactory == null) {
+					oe = new OutputEmitter<T>(strategy);
+				}
+				else {
+					final DataDistribution dataDist = config.getOutputDataDistribution(i, cl);
+					final Partitioner<?> partitioner = config.getOutputPartitioner(i, cl);
+
+					final TypeComparator<T> comparator = compFactory.createComparator();
+					oe = new OutputEmitter<T>(strategy, comparator, partitioner, dataDist);
+				}
+
+				final RecordWriter<SerializationDelegate<T>> recordWriter =
+						new RecordWriter<SerializationDelegate<T>>(task.getEnvironment().getWriter(outputOffset + i), oe);
+
+				// setup live accumulator counters
+				recordWriter.setReporter(reporter);
+
+				writers.add(recordWriter);
+			}
+			if (eventualOutputs != null) {
+				eventualOutputs.addAll(writers);
+			}
+			return new OutputCollector<T>(writers, serializerFactory.getSerializer());
+		}
+	}
+
+	/**
+	 * Creates a writer for each output. Creates an OutputCollector which forwards its input to all writers.
+	 * The output collector applies the configured shipping strategy.
+	 */
+	@SuppressWarnings("unchecked")
+	public static <T> Collector<T> initOutputs(AbstractInvokable nepheleTask, ClassLoader cl, TaskConfig config,
+										List<ChainedDriver<?, ?>> chainedTasksTarget,
+										List<RecordWriter<?>> eventualOutputs,
+										ExecutionConfig executionConfig,
+										AccumulatorRegistry.Reporter reporter,
+										Map<String, Accumulator<?,?>> accumulatorMap)
+	throws Exception
+	{
+		final int numOutputs = config.getNumOutputs();
+
+		// check whether we got any chained tasks
+		final int numChained = config.getNumberOfChainedStubs();
+		if (numChained > 0) {
+			// got chained stubs. that means that this one may only have a single forward connection
+			if (numOutputs != 1 || config.getOutputShipStrategy(0) != ShipStrategyType.FORWARD) {
+				throw new RuntimeException("Plan Generation Bug: Found a chained stub that is not connected via an only forward connection.");
+			}
+
+			// instantiate each task
+			@SuppressWarnings("rawtypes")
+			Collector previous = null;
+			for (int i = numChained - 1; i >= 0; --i)
+			{
+				// get the task first
+				final ChainedDriver<?, ?> ct;
+				try {
+					Class<? extends ChainedDriver<?, ?>> ctc = config.getChainedTask(i);
+					ct = ctc.newInstance();
+				}
+				catch (Exception ex) {
+					throw new RuntimeException("Could not instantiate chained task driver.", ex);
+				}
+
+				// get the configuration for the task
+				final TaskConfig chainedStubConf = config.getChainedStubConfig(i);
+				final String taskName = config.getChainedTaskName(i);
+
+				if (i == numChained - 1) {
+					// last in chain, instantiate the output collector for this task
+					previous = getOutputCollector(nepheleTask, chainedStubConf, cl, eventualOutputs, 0, chainedStubConf.getNumOutputs(), reporter);
+				}
+
+				ct.setup(chainedStubConf, taskName, previous, nepheleTask, cl, executionConfig, accumulatorMap);
+				chainedTasksTarget.add(0, ct);
+
+				previous = ct;
+			}
+			// the collector of the first in the chain is the collector for the nephele task
+			return (Collector<T>) previous;
+		}
+		// else
+
+		// instantiate the output collector the default way from this configuration
+		return getOutputCollector(nepheleTask , config, cl, eventualOutputs, 0, numOutputs, reporter);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                                  User Code LifeCycle
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Opens the given stub using its {@link org.apache.flink.api.common.functions.RichFunction#open(Configuration)} method. If the open call produces
+	 * an exception, a new exception with a standard error message is created, using the encountered exception
+	 * as its cause.
+	 * 
+	 * @param stub The user code instance to be opened.
+	 * @param parameters The parameters supplied to the user code.
+	 * 
+	 * @throws Exception Thrown, if the user code's open method produces an exception.
+	 */
+	public static void openUserCode(Function stub, Configuration parameters) throws Exception {
+		try {
+			FunctionUtils.openFunction(stub, parameters);
+		} catch (Throwable t) {
+			throw new Exception("The user defined 'open(Configuration)' method in " + stub.getClass().toString() + " caused an exception: " + t.getMessage(), t);
+		}
+	}
+	
+	/**
+	 * Closes the given stub using its {@link org.apache.flink.api.common.functions.RichFunction#close()} method. If the close call produces
+	 * an exception, a new exception with a standard error message is created, using the encountered exception
+	 * as its cause.
+	 * 
+	 * @param stub The user code instance to be closed.
+	 * 
+	 * @throws Exception Thrown, if the user code's close method produces an exception.
+	 */
+	public static void closeUserCode(Function stub) throws Exception {
+		try {
+			FunctionUtils.closeFunction(stub);
+		} catch (Throwable t) {
+			throw new Exception("The user defined 'close()' method caused an exception: " + t.getMessage(), t);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                               Chained Task LifeCycle
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Opens all chained tasks, in the order as they are stored in the array. The opening process
+	 * creates a standardized log info message.
+	 * 
+	 * @param tasks The tasks to be opened.
+	 * @param parent The parent task, used to obtain parameters to include in the log message.
+	 * @throws Exception Thrown, if the opening encounters an exception.
+	 */
+	public static void openChainedTasks(List<ChainedDriver<?, ?>> tasks, AbstractInvokable parent) throws Exception {
+		// start all chained tasks
+		for (int i = 0; i < tasks.size(); i++) {
+			final ChainedDriver<?, ?> task = tasks.get(i);
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(constructLogString("Start task code", task.getTaskName(), parent));
+			}
+			task.openTask();
+		}
+	}
+	
+	/**
+	 * Closes all chained tasks, in the order as they are stored in the array. The closing process
+	 * creates a standardized log info message.
+	 * 
+	 * @param tasks The tasks to be closed.
+	 * @param parent The parent task, used to obtain parameters to include in the log message.
+	 * @throws Exception Thrown, if the closing encounters an exception.
+	 */
+	public static void closeChainedTasks(List<ChainedDriver<?, ?>> tasks, AbstractInvokable parent) throws Exception {
+		for (int i = 0; i < tasks.size(); i++) {
+			final ChainedDriver<?, ?> task = tasks.get(i);
+			task.closeTask();
+			
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(constructLogString("Finished task code", task.getTaskName(), parent));
+			}
+		}
+	}
+	
+	/**
+	 * Cancels all tasks via their {@link ChainedDriver#cancelTask()} method. Any occurring exception
+	 * and error is suppressed, such that the canceling method of every task is invoked in all cases.
+	 * 
+	 * @param tasks The tasks to be canceled.
+	 */
+	public static void cancelChainedTasks(List<ChainedDriver<?, ?>> tasks) {
+		for (int i = 0; i < tasks.size(); i++) {
+			try {
+				tasks.get(i).cancelTask();
+			} catch (Throwable t) {
+				// do nothing
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                                     Miscellaneous Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Instantiates a user code class from is definition in the task configuration.
+	 * The class is instantiated without arguments using the null-ary constructor. Instantiation
+	 * will fail if this constructor does not exist or is not public.
+	 * 
+	 * @param <T> The generic type of the user code class.
+	 * @param config The task configuration containing the class description.
+	 * @param cl The class loader to be used to load the class.
+	 * @param superClass The super class that the user code class extends or implements, for type checking.
+	 * 
+	 * @return An instance of the user code class.
+	 */
+	public static <T> T instantiateUserCode(TaskConfig config, ClassLoader cl, Class<? super T> superClass) {
+		try {
+			T stub = config.<T>getStubWrapper(cl).getUserCodeObject(superClass, cl);
+			// check if the class is a subclass, if the check is required
+			if (superClass != null && !superClass.isAssignableFrom(stub.getClass())) {
+				throw new RuntimeException("The class '" + stub.getClass().getName() + "' is not a subclass of '" + 
+						superClass.getName() + "' as is required.");
+			}
+			return stub;
+		}
+		catch (ClassCastException ccex) {
+			throw new RuntimeException("The UDF class is not a proper subclass of " + superClass.getName(), ccex);
+		}
+	}
+	
+	private static int[] asArray(List<Integer> list) {
+		int[] a = new int[list.size()];
+		
+		int i = 0;
+		for (int val : list) {
+			a[i++] = val;
+		}
+		return a;
+	}
+
+	public static void clearWriters(List<RecordWriter<?>> writers) {
+		for (RecordWriter<?> writer : writers) {
+			writer.clearBuffers();
+		}
+	}
+
+	public static void clearReaders(MutableReader<?>[] readers) {
+		for (MutableReader<?> reader : readers) {
+			reader.clearBuffers();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java
index dfc4e91..665ab0e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupDriver.java
@@ -43,12 +43,12 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @see org.apache.flink.api.common.functions.CoGroupFunction
  */
-public class CoGroupDriver<IT1, IT2, OT> implements PactDriver<CoGroupFunction<IT1, IT2, OT>, OT> {
+public class CoGroupDriver<IT1, IT2, OT> implements Driver<CoGroupFunction<IT1, IT2, OT>, OT> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(CoGroupDriver.class);
 	
 	
-	private PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
+	private TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	private CoGroupTaskIterator<IT1, IT2> coGroupIterator;				// the iterator that does the actual cogroup
 	
@@ -60,7 +60,7 @@ public class CoGroupDriver<IT1, IT2, OT> implements PactDriver<CoGroupFunction<I
 
 
 	@Override
-	public void setup(PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupRawDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupRawDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupRawDriver.java
index 7abad5d..0487904 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupRawDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupRawDriver.java
@@ -26,17 +26,17 @@ import org.apache.flink.util.MutableObjectIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class CoGroupRawDriver<IT1, IT2, OT> implements PactDriver<CoGroupFunction<IT1, IT2, OT>, OT> {
+public class CoGroupRawDriver<IT1, IT2, OT> implements Driver<CoGroupFunction<IT1, IT2, OT>, OT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(CoGroupRawDriver.class);
 
-	private PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
+	private TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
 
 	private SimpleIterable<IT1> coGroupIterator1;
 	private SimpleIterable<IT2> coGroupIterator2;
 
 	@Override
-	public void setup(PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
index 97d6e51..07fb352 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
@@ -29,7 +29,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
-import org.apache.flink.runtime.iterative.task.AbstractIterativePactTask;
+import org.apache.flink.runtime.iterative.task.AbstractIterativeTask;
 import org.apache.flink.runtime.operators.hash.CompactingHashTable;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator;
@@ -37,9 +37,9 @@ import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
 import org.apache.flink.runtime.util.SingleElementIterator;
 import org.apache.flink.util.Collector;
 
-public class CoGroupWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettablePactDriver<CoGroupFunction<IT1, IT2, OT>, OT> {
+public class CoGroupWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettableDriver<CoGroupFunction<IT1, IT2, OT>, OT> {
 	
-	private PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
+	private TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	private CompactingHashTable<IT1> hashTable;
 	
@@ -63,7 +63,7 @@ public class CoGroupWithSolutionSetFirstDriver<IT1, IT2, OT> implements Resettab
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
-	public void setup(PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}
@@ -106,8 +106,8 @@ public class CoGroupWithSolutionSetFirstDriver<IT1, IT2, OT> implements Resettab
 		final TypeComparator<IT1> solutionSetComparator;
 		
 		// grab a handle to the hash table from the iteration broker
-		if (taskContext instanceof AbstractIterativePactTask) {
-			AbstractIterativePactTask<?, ?> iterativeTaskContext = (AbstractIterativePactTask<?, ?>) taskContext;
+		if (taskContext instanceof AbstractIterativeTask) {
+			AbstractIterativeTask<?, ?> iterativeTaskContext = (AbstractIterativeTask<?, ?>) taskContext;
 			String identifier = iterativeTaskContext.brokerKey();
 			
 			Object table = SolutionSetBroker.instance().get(identifier);

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
index 9e8a81c..eec6597 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
@@ -27,7 +27,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
-import org.apache.flink.runtime.iterative.task.AbstractIterativePactTask;
+import org.apache.flink.runtime.iterative.task.AbstractIterativeTask;
 import org.apache.flink.runtime.operators.hash.CompactingHashTable;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.util.EmptyIterator;
@@ -36,9 +36,9 @@ import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
 import org.apache.flink.runtime.util.SingleElementIterator;
 import org.apache.flink.util.Collector;
 
-public class CoGroupWithSolutionSetSecondDriver<IT1, IT2, OT> implements ResettablePactDriver<CoGroupFunction<IT1, IT2, OT>, OT> {
+public class CoGroupWithSolutionSetSecondDriver<IT1, IT2, OT> implements ResettableDriver<CoGroupFunction<IT1, IT2, OT>, OT> {
 	
-	private PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
+	private TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	private CompactingHashTable<IT2> hashTable;
 	
@@ -61,7 +61,7 @@ public class CoGroupWithSolutionSetSecondDriver<IT1, IT2, OT> implements Resetta
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
-	public void setup(PactTaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<CoGroupFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}
@@ -104,8 +104,8 @@ public class CoGroupWithSolutionSetSecondDriver<IT1, IT2, OT> implements Resetta
 		final TypeComparator<IT2> solutionSetComparator;
 		
 		// grab a handle to the hash table from the iteration broker
-		if (taskContext instanceof AbstractIterativePactTask) {
-			AbstractIterativePactTask<?, ?> iterativeTaskContext = (AbstractIterativePactTask<?, ?>) taskContext;
+		if (taskContext instanceof AbstractIterativeTask) {
+			AbstractIterativeTask<?, ?> iterativeTaskContext = (AbstractIterativeTask<?, ?>) taskContext;
 			String identifier = iterativeTaskContext.brokerKey();
 			Object table = SolutionSetBroker.instance().get(identifier);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
index 7e6376f..60bc798 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
@@ -40,19 +40,19 @@ import org.slf4j.LoggerFactory;
  * @param <OT> The mapper's output data type.
  */
 @SuppressWarnings("deprecation")
-public class CollectorMapDriver<IT, OT> implements PactDriver<GenericCollectorMap<IT, OT>, OT> {
+public class CollectorMapDriver<IT, OT> implements Driver<GenericCollectorMap<IT, OT>, OT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(CollectorMapDriver.class);
 
 
-	private PactTaskContext<GenericCollectorMap<IT, OT>, OT> taskContext;
+	private TaskContext<GenericCollectorMap<IT, OT>, OT> taskContext;
 	
 	private volatile boolean running;
 
 	private boolean objectReuseEnabled = false;
 
 	@Override
-	public void setup(PactTaskContext<GenericCollectorMap<IT, OT>, OT> context) {
+	public void setup(TaskContext<GenericCollectorMap<IT, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
index 011c980..c9d84b1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
@@ -41,12 +41,12 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @see org.apache.flink.api.common.functions.CrossFunction
  */
-public class CrossDriver<T1, T2, OT> implements PactDriver<CrossFunction<T1, T2, OT>, OT> {
+public class CrossDriver<T1, T2, OT> implements Driver<CrossFunction<T1, T2, OT>, OT> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(CrossDriver.class);
 	
 	
-	private PactTaskContext<CrossFunction<T1, T2, OT>, OT> taskContext;
+	private TaskContext<CrossFunction<T1, T2, OT>, OT> taskContext;
 	
 	private MemoryManager memManager;
 	
@@ -70,7 +70,7 @@ public class CrossDriver<T1, T2, OT> implements PactDriver<CrossFunction<T1, T2,
 
 
 	@Override
-	public void setup(PactTaskContext<CrossFunction<T1, T2, OT>, OT> context) {
+	public void setup(TaskContext<CrossFunction<T1, T2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java
index 1002bae..d20bb89 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java
@@ -84,9 +84,8 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 
 	@Override
 	public void registerInputOutput() {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Start registering input and output"));
-		}
+
+		LOG.debug(getLogString("Start registering input and output"));
 
 		// initialize OutputFormat
 		initOutputFormat();
@@ -99,24 +98,18 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 					(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
 		}
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Finished registering input and output"));
-		}
+		LOG.debug(getLogString("Finished registering input and output"));
 	}
 
 
 	@Override
 	public void invoke() throws Exception
 	{
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Starting data sink operator"));
-		}
+		LOG.debug(getLogString("Starting data sink operator"));
 
 		if(RichOutputFormat.class.isAssignableFrom(this.format.getClass())){
 			((RichOutputFormat) this.format).setRuntimeContext(createRuntimeContext());
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Rich Sink detected. Initializing runtime context."));
-			}
+			LOG.debug(getLogString("Rich Sink detected. Initializing runtime context."));
 		}
 
 		ExecutionConfig executionConfig;
@@ -131,9 +124,7 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 				LOG.warn("The execution config returned by the configuration was null");
 				executionConfig = new ExecutionConfig();
 			}
-		} catch (IOException e) {
-			throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e);
-		} catch (ClassNotFoundException e) {
+		} catch (IOException | ClassNotFoundException e) {
 			throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e);
 		}
 		boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled();
@@ -190,9 +181,7 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 				return;
 			}
 
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Starting to produce output"));
-			}
+			LOG.debug(getLogString("Starting to produce output"));
 
 			// open
 			format.open(this.getEnvironment().getIndexInSubtaskGroup(), this.getEnvironment().getNumberOfSubtasks());
@@ -268,18 +257,14 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 				}
 			}
 
-			RegularPactTask.clearReaders(new MutableReader[]{inputReader});
+			BatchTask.clearReaders(new MutableReader[]{inputReader});
 		}
 
 		if (!this.taskCanceled) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Finished data sink operator"));
-			}
+			LOG.debug(getLogString("Finished data sink operator"));
 		}
 		else {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Data sink operator cancelled"));
-			}
+			LOG.debug(getLogString("Data sink operator cancelled"));
 		}
 	}
 
@@ -304,9 +289,7 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 			}
 		}
 		
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Cancelling data sink operator"));
-		}
+		LOG.debug(getLogString("Cancelling data sink operator"));
 	}
 	
 	/**
@@ -402,7 +385,7 @@ public class DataSinkTask<IT> extends AbstractInvokable {
 	 * @return The string ready for logging.
 	 */
 	private String getLogString(String message) {
-		return RegularPactTask.constructLogString(message, this.getEnvironment().getTaskName(), this);
+		return BatchTask.constructLogString(message, this.getEnvironment().getTaskName(), this);
 	}
 
 	public DistributedRuntimeUDFContext createRuntimeContext() {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
index 5806e77..801e1a1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
@@ -80,9 +80,7 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 	public void registerInputOutput() {
 		initInputFormat();
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Start registering input and output"));
-		}
+		LOG.debug(getLogString("Start registering input and output"));
 
 		try {
 			initOutputs(getUserCodeClassLoader());
@@ -91,24 +89,18 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 				ex.getMessage(), ex);
 		}
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Finished registering input and output"));
-		}
+		LOG.debug(getLogString("Finished registering input and output"));
 	}
 
 
 	@Override
 	public void invoke() throws Exception {
 		
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Starting data source operator"));
-		}
+		LOG.debug(getLogString("Starting data source operator"));
 
 		if(RichInputFormat.class.isAssignableFrom(this.format.getClass())){
 			((RichInputFormat) this.format).setRuntimeContext(createRuntimeContext());
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Rich Source detected. Initializing runtime context."));
-			}
+			LOG.debug(getLogString("Rich Source detected. Initializing runtime context."));
 		}
 
 		ExecutionConfig executionConfig;
@@ -123,23 +115,19 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 				LOG.warn("ExecutionConfig from job configuration is null. Creating empty config");
 				executionConfig = new ExecutionConfig();
 			}
-		} catch (IOException e) {
-			throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: ", e);
-		} catch (ClassNotFoundException e) {
+		} catch (IOException | ClassNotFoundException e) {
 			throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: ", e);
 		}
 
 		boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled();
 
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("DataSourceTask object reuse: " + (objectReuseEnabled ? "ENABLED" : "DISABLED") + ".");
-		}
+		LOG.debug("DataSourceTask object reuse: " + (objectReuseEnabled ? "ENABLED" : "DISABLED") + ".");
 		
 		final TypeSerializer<OT> serializer = this.serializerFactory.getSerializer();
 		
 		try {
 			// start all chained tasks
-			RegularPactTask.openChainedTasks(this.chainedTasks, this);
+			BatchTask.openChainedTasks(this.chainedTasks, this);
 			
 			// get input splits to read
 			final Iterator<InputSplit> splitIterator = getInputSplits();
@@ -150,18 +138,14 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 				// get start and end
 				final InputSplit split = splitIterator.next();
 
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(getLogString("Opening input split " + split.toString()));
-				}
+				LOG.debug(getLogString("Opening input split " + split.toString()));
 				
 				final InputFormat<OT, InputSplit> format = this.format;
 			
 				// open input format
 				format.open(split);
 	
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(getLogString("Starting to read input from split " + split.toString()));
-				}
+				LOG.debug(getLogString("Starting to read input from split " + split.toString()));
 				
 				try {
 					final Collector<OT> output = this.output;
@@ -201,7 +185,7 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 			this.output.close();
 
 			// close all chained tasks letting them report failure
-			RegularPactTask.closeChainedTasks(this.chainedTasks, this);
+			BatchTask.closeChainedTasks(this.chainedTasks, this);
 
 		}
 		catch (Exception ex) {
@@ -210,7 +194,7 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 				this.format.close();
 			} catch (Throwable ignored) {}
 
-			RegularPactTask.cancelChainedTasks(this.chainedTasks);
+			BatchTask.cancelChainedTasks(this.chainedTasks);
 
 			ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
 
@@ -220,30 +204,24 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 			}
 			else if (!this.taskCanceled) {
 				// drop exception, if the task was canceled
-				RegularPactTask.logAndThrowException(ex, this);
+				BatchTask.logAndThrowException(ex, this);
 			}
 		} finally {
-			RegularPactTask.clearWriters(eventualOutputs);
+			BatchTask.clearWriters(eventualOutputs);
 		}
 
 		if (!this.taskCanceled) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Finished data source operator"));
-			}
+			LOG.debug(getLogString("Finished data source operator"));
 		}
 		else {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug(getLogString("Data source operator cancelled"));
-			}
+			LOG.debug(getLogString("Data source operator cancelled"));
 		}
 	}
 
 	@Override
 	public void cancel() throws Exception {
 		this.taskCanceled = true;
-		if (LOG.isDebugEnabled()) {
-			LOG.debug(getLogString("Cancelling data source operator"));
-		}
+		LOG.debug(getLogString("Cancelling data source operator"));
 	}
 	
 	/**
@@ -303,7 +281,7 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 		final AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
 		final AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter();
 
-		this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs,
+		this.output = BatchTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs,
 				getExecutionConfig(), reporter, getEnvironment().getAccumulatorRegistry().getUserMap());
 	}
 
@@ -331,7 +309,7 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 	 * @return The string ready for logging.
 	 */
 	private String getLogString(String message, String taskName) {
-		return RegularPactTask.constructLogString(message, taskName, this);
+		return BatchTask.constructLogString(message, taskName, this);
 	}
 	
 	private Iterator<InputSplit> getInputSplits() {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/Driver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/Driver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/Driver.java
new file mode 100644
index 0000000..8e8795e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/Driver.java
@@ -0,0 +1,90 @@
+/*
+ * 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.runtime.operators;
+
+import org.apache.flink.api.common.functions.Function;
+
+/**
+ * The interface to be implemented by all drivers that run alone (or as the primary driver) in a task.
+ * A driver implements the actual code to perform a batch operation, like <i>map()</i>,
+ * <i>reduce()</i>, <i>join()</i>, or <i>coGroup()</i>.
+ *
+ * @see TaskContext
+ * 
+ * @param <S> The type of stub driven by this driver.
+ * @param <OT> The data type of the records produced by this driver.
+ */
+public interface Driver<S extends Function, OT> {
+	
+	void setup(TaskContext<S, OT> context);
+	
+	/**
+	 * Gets the number of inputs that the task has.
+	 * 
+	 * @return The number of inputs.
+	 */
+	int getNumberOfInputs();
+	
+	/**
+	 * Gets the number of comparators required for this driver.
+	 * 
+	 * @return The number of comparators required for this driver.
+	 */
+	int getNumberOfDriverComparators();
+	
+	/**
+	 * Gets the class of the stub type that is run by this task. For example, a <tt>MapTask</tt> should return
+	 * <code>MapFunction.class</code>.   
+	 * 
+	 * @return The class of the stub type run by the task.
+	 */
+	Class<S> getStubType();
+	
+	/**
+	 * This method is called before the user code is opened. An exception thrown by this method
+	 * signals failure of the task.
+	 * 
+	 * @throws Exception Exceptions may be forwarded and signal task failure.
+	 */
+	void prepare() throws Exception;
+	
+	/**
+	 * The main operation method of the task. It should call the user code with the data subsets until
+	 * the input is depleted.
+	 * 
+	 * @throws Exception Any exception thrown by this method signals task failure. Because exceptions in the user
+	 *                   code typically signal situations where this instance in unable to proceed, exceptions
+	 *                   from the user code should be forwarded.
+	 */
+	void run() throws Exception;
+	
+	/**
+	 * This method is invoked in any case (clean termination and exception) at the end of the tasks operation.
+	 * 
+	 * @throws Exception Exceptions may be forwarded.
+	 */
+	void cleanup() throws Exception;
+	
+	/**
+	 * This method is invoked when the driver must aborted in mid processing. It is invoked asynchronously by a different thread.
+	 * 
+	 * @throws Exception Exceptions may be forwarded.
+	 */
+	void cancel() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
index 0aceae2..bc7bee5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DriverStrategy.java
@@ -115,7 +115,7 @@ public enum DriverStrategy {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	private final Class<? extends PactDriver<?, ?>> driverClass;
+	private final Class<? extends Driver<?, ?>> driverClass;
 	
 	private final Class<? extends ChainedDriver<?, ?>> pushChainDriver;
 	
@@ -129,11 +129,11 @@ public enum DriverStrategy {
 
 	@SuppressWarnings("unchecked")
 	private DriverStrategy(
-			@SuppressWarnings("rawtypes") Class<? extends PactDriver> driverClass, 
+			@SuppressWarnings("rawtypes") Class<? extends Driver> driverClass,
 			@SuppressWarnings("rawtypes") Class<? extends ChainedDriver> pushChainDriverClass, 
 			DamBehavior dam, int numComparator)
 	{
-		this.driverClass = (Class<? extends PactDriver<?, ?>>) driverClass;
+		this.driverClass = (Class<? extends Driver<?, ?>>) driverClass;
 		this.pushChainDriver = (Class<? extends ChainedDriver<?, ?>>) pushChainDriverClass;
 		this.numInputs = 1;
 		this.dam1 = dam;
@@ -143,11 +143,11 @@ public enum DriverStrategy {
 	
 	@SuppressWarnings("unchecked")
 	private DriverStrategy(
-			@SuppressWarnings("rawtypes") Class<? extends PactDriver> driverClass, 
+			@SuppressWarnings("rawtypes") Class<? extends Driver> driverClass,
 			@SuppressWarnings("rawtypes") Class<? extends ChainedDriver> pushChainDriverClass, 
 			DamBehavior firstDam, DamBehavior secondDam, int numComparator)
 	{
-		this.driverClass = (Class<? extends PactDriver<?, ?>>) driverClass;
+		this.driverClass = (Class<? extends Driver<?, ?>>) driverClass;
 		this.pushChainDriver = (Class<? extends ChainedDriver<?, ?>>) pushChainDriverClass;
 		this.numInputs = 2;
 		this.dam1 = firstDam;
@@ -157,7 +157,7 @@ public enum DriverStrategy {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public Class<? extends PactDriver<?, ?>> getDriverClass() {
+	public Class<? extends Driver<?, ?>> getDriverClass() {
 		return this.driverClass;
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java
index fd245bd..c29923b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FlatMapDriver.java
@@ -39,18 +39,18 @@ import org.slf4j.LoggerFactory;
  * @param <IT> The mapper's input data type.
  * @param <OT> The mapper's output data type.
  */
-public class FlatMapDriver<IT, OT> implements PactDriver<FlatMapFunction<IT, OT>, OT> {
+public class FlatMapDriver<IT, OT> implements Driver<FlatMapFunction<IT, OT>, OT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(FlatMapDriver.class);
 
-	private PactTaskContext<FlatMapFunction<IT, OT>, OT> taskContext;
+	private TaskContext<FlatMapFunction<IT, OT>, OT> taskContext;
 	
 	private volatile boolean running;
 
 	private boolean objectReuseEnabled = false;
 
 	@Override
-	public void setup(PactTaskContext<FlatMapFunction<IT, OT>, OT> context) {
+	public void setup(TaskContext<FlatMapFunction<IT, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}


[4/5] flink git commit: [FLINK-2815] [REFACTOR] Remove Pact from class and file names since it is no longer valid reference

Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
deleted file mode 100644
index 9cb045f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadPactTask.java
+++ /dev/null
@@ -1,440 +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.runtime.iterative.task;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.operators.util.JoinHashMap;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.disk.InputViewIterator;
-import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel;
-import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannelBroker;
-import org.apache.flink.runtime.iterative.concurrent.Broker;
-import org.apache.flink.runtime.iterative.concurrent.IterationAggregatorBroker;
-import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
-import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrier;
-import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepBarrier;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatchBroker;
-import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent;
-import org.apache.flink.runtime.iterative.event.TerminationEvent;
-import org.apache.flink.runtime.iterative.event.WorkerDoneEvent;
-import org.apache.flink.runtime.iterative.io.SerializedUpdateBuffer;
-import org.apache.flink.runtime.operators.RegularPactTask;
-import org.apache.flink.runtime.operators.hash.CompactingHashTable;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.MutableObjectIterator;
-
-/**
- * The head is responsible for coordinating an iteration and can run a
- * {@link org.apache.flink.runtime.operators.PactDriver} inside. It will read
- * the initial input and establish a {@link BlockingBackChannel} to the iteration's tail. After successfully processing
- * the input, it will send EndOfSuperstep events to its outputs. It must also be connected to a
- * synchronization task and after each superstep, it will wait
- * until it receives an {@link AllWorkersDoneEvent} from the sync, which signals that all other heads have also finished
- * their iteration. Starting with
- * the second iteration, the input for the head is the output of the tail, transmitted through the backchannel. Once the
- * iteration is done, the head
- * will send a {@link TerminationEvent} to all it's connected tasks, signaling them to shutdown.
- * <p>
- * Assumption on the ordering of the outputs: - The first n output gates write to channels that go to the tasks of the
- * step function. - The next m output gates to to the tasks that consume the final solution. - The last output gate
- * connects to the synchronization task.
- * 
- * @param <X>
- *        The type of the bulk partial solution / solution set and the final output.
- * @param <Y>
- *        The type of the feed-back data set (bulk partial solution / workset). For bulk iterations, {@code Y} is the
- *        same as {@code X}
- */
-public class IterationHeadPactTask<X, Y, S extends Function, OT> extends AbstractIterativePactTask<S, OT> {
-
-	private static final Logger log = LoggerFactory.getLogger(IterationHeadPactTask.class);
-
-	private Collector<X> finalOutputCollector;
-
-	private TypeSerializerFactory<Y> feedbackTypeSerializer;
-
-	private TypeSerializerFactory<X> solutionTypeSerializer;
-
-	private ResultPartitionWriter toSync;
-
-	private int feedbackDataInput; // workset or bulk partial solution
-
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	protected int getNumTaskInputs() {
-		// this task has an additional input in the workset case for the initial solution set
-		boolean isWorkset = config.getIsWorksetIteration();
-		return driver.getNumberOfInputs() + (isWorkset ? 1 : 0);
-	}
-
-	@Override
-	protected void initOutputs() throws Exception {
-		// initialize the regular outputs first (the ones into the step function).
-		super.initOutputs();
-
-		// at this time, the outputs to the step function are created
-		// add the outputs for the final solution
-		List<RecordWriter<?>> finalOutputWriters = new ArrayList<RecordWriter<?>>();
-		final TaskConfig finalOutConfig = this.config.getIterationHeadFinalOutputConfig();
-		final ClassLoader userCodeClassLoader = getUserCodeClassLoader();
-		AccumulatorRegistry.Reporter reporter = getEnvironment().getAccumulatorRegistry().getReadWriteReporter();
-		this.finalOutputCollector = RegularPactTask.getOutputCollector(this, finalOutConfig,
-			userCodeClassLoader, finalOutputWriters, config.getNumOutputs(), finalOutConfig.getNumOutputs(), reporter);
-
-		// sanity check the setup
-		final int writersIntoStepFunction = this.eventualOutputs.size();
-		final int writersIntoFinalResult = finalOutputWriters.size();
-		final int syncGateIndex = this.config.getIterationHeadIndexOfSyncOutput();
-
-		if (writersIntoStepFunction + writersIntoFinalResult != syncGateIndex) {
-			throw new Exception("Error: Inconsistent head task setup - wrong mapping of output gates.");
-		}
-		// now, we can instantiate the sync gate
-		this.toSync = getEnvironment().getWriter(syncGateIndex);
-	}
-
-	/**
-	 * the iteration head prepares the backchannel: it allocates memory, instantiates a {@link BlockingBackChannel} and
-	 * hands it to the iteration tail via a {@link Broker} singleton
-	 **/
-	private BlockingBackChannel initBackChannel() throws Exception {
-
-		/* get the size of the memory available to the backchannel */
-		int backChannelMemoryPages = getMemoryManager().computeNumberOfPages(this.config.getRelativeBackChannelMemory());
-
-		/* allocate the memory available to the backchannel */
-		List<MemorySegment> segments = new ArrayList<MemorySegment>();
-		int segmentSize = getMemoryManager().getPageSize();
-		getMemoryManager().allocatePages(this, segments, backChannelMemoryPages);
-
-		/* instantiate the backchannel */
-		BlockingBackChannel backChannel = new BlockingBackChannel(new SerializedUpdateBuffer(segments, segmentSize,
-			getIOManager()));
-
-		/* hand the backchannel over to the iteration tail */
-		Broker<BlockingBackChannel> broker = BlockingBackChannelBroker.instance();
-		broker.handIn(brokerKey(), backChannel);
-
-		return backChannel;
-	}
-	
-	private <BT> CompactingHashTable<BT> initCompactingHashTable() throws Exception {
-		// get some memory
-		double hashjoinMemorySize = config.getRelativeSolutionSetMemory();
-		final ClassLoader userCodeClassLoader = getUserCodeClassLoader();
-
-		TypeSerializerFactory<BT> solutionTypeSerializerFactory = config.getSolutionSetSerializer(userCodeClassLoader);
-		TypeComparatorFactory<BT> solutionTypeComparatorFactory = config.getSolutionSetComparator(userCodeClassLoader);
-	
-		TypeSerializer<BT> solutionTypeSerializer = solutionTypeSerializerFactory.getSerializer();
-		TypeComparator<BT> solutionTypeComparator = solutionTypeComparatorFactory.createComparator();
-
-		CompactingHashTable<BT> hashTable = null;
-		List<MemorySegment> memSegments = null;
-		boolean success = false;
-		try {
-			int numPages = getMemoryManager().computeNumberOfPages(hashjoinMemorySize);
-			memSegments = getMemoryManager().allocatePages(getOwningNepheleTask(), numPages);
-			hashTable = new CompactingHashTable<BT>(solutionTypeSerializer, solutionTypeComparator, memSegments);
-			success = true;
-			return hashTable;
-		} finally {
-			if (!success) {
-				if (hashTable != null) {
-					try {
-						hashTable.close();
-					} catch (Throwable t) {
-						log.error("Error closing the solution set hash table after unsuccessful creation.", t);
-					}
-				}
-				if (memSegments != null) {
-					try {
-						getMemoryManager().release(memSegments);
-					} catch (Throwable t) {
-						log.error("Error freeing memory after error during solution set hash table creation.", t);
-					}
-				}
-			}
-		}
-	}
-	
-	private <BT> JoinHashMap<BT> initJoinHashMap() {
-		TypeSerializerFactory<BT> solutionTypeSerializerFactory = config.getSolutionSetSerializer
-				(getUserCodeClassLoader());
-		TypeComparatorFactory<BT> solutionTypeComparatorFactory = config.getSolutionSetComparator
-				(getUserCodeClassLoader());
-	
-		TypeSerializer<BT> solutionTypeSerializer = solutionTypeSerializerFactory.getSerializer();
-		TypeComparator<BT> solutionTypeComparator = solutionTypeComparatorFactory.createComparator();
-		
-		return new JoinHashMap<BT>(solutionTypeSerializer, solutionTypeComparator);
-	}
-	
-	private void readInitialSolutionSet(CompactingHashTable<X> solutionSet, MutableObjectIterator<X> solutionSetInput) throws IOException {
-		solutionSet.open();
-		solutionSet.buildTableWithUniqueKey(solutionSetInput);
-	}
-	
-	private void readInitialSolutionSet(JoinHashMap<X> solutionSet, MutableObjectIterator<X> solutionSetInput) throws IOException {
-		TypeSerializer<X> serializer = solutionTypeSerializer.getSerializer();
-		
-		X next;
-		while ((next = solutionSetInput.next(serializer.createInstance())) != null) {
-			solutionSet.insertOrReplace(next);
-		}
-	}
-
-	private SuperstepBarrier initSuperstepBarrier() {
-		SuperstepBarrier barrier = new SuperstepBarrier(getUserCodeClassLoader());
-		this.toSync.subscribeToEvent(barrier, AllWorkersDoneEvent.class);
-		this.toSync.subscribeToEvent(barrier, TerminationEvent.class);
-		return barrier;
-	}
-
-	@Override
-	public void run() throws Exception {
-		final String brokerKey = brokerKey();
-		final int workerIndex = getEnvironment().getIndexInSubtaskGroup();
-		
-		final boolean objectSolutionSet = config.isSolutionSetUnmanaged();
-
-		CompactingHashTable<X> solutionSet = null; // if workset iteration
-		JoinHashMap<X> solutionSetObjectMap = null; // if workset iteration with unmanaged solution set
-		
-		boolean waitForSolutionSetUpdate = config.getWaitForSolutionSetUpdate();
-		boolean isWorksetIteration = config.getIsWorksetIteration();
-
-		try {
-			/* used for receiving the current iteration result from iteration tail */
-			SuperstepKickoffLatch nextStepKickoff = new SuperstepKickoffLatch();
-			SuperstepKickoffLatchBroker.instance().handIn(brokerKey, nextStepKickoff);
-			
-			BlockingBackChannel backChannel = initBackChannel();
-			SuperstepBarrier barrier = initSuperstepBarrier();
-			SolutionSetUpdateBarrier solutionSetUpdateBarrier = null;
-
-			feedbackDataInput = config.getIterationHeadPartialSolutionOrWorksetInputIndex();
-			feedbackTypeSerializer = this.getInputSerializer(feedbackDataInput);
-			excludeFromReset(feedbackDataInput);
-
-			int initialSolutionSetInput;
-			if (isWorksetIteration) {
-				initialSolutionSetInput = config.getIterationHeadSolutionSetInputIndex();
-				solutionTypeSerializer = config.getSolutionSetSerializer(getUserCodeClassLoader());
-
-				// setup the index for the solution set
-				@SuppressWarnings("unchecked")
-				MutableObjectIterator<X> solutionSetInput = (MutableObjectIterator<X>) createInputIterator(inputReaders[initialSolutionSetInput], solutionTypeSerializer);
-				
-				// read the initial solution set
-				if (objectSolutionSet) {
-					solutionSetObjectMap = initJoinHashMap();
-					readInitialSolutionSet(solutionSetObjectMap, solutionSetInput);
-					SolutionSetBroker.instance().handIn(brokerKey, solutionSetObjectMap);
-				} else {
-					solutionSet = initCompactingHashTable();
-					readInitialSolutionSet(solutionSet, solutionSetInput);
-					SolutionSetBroker.instance().handIn(brokerKey, solutionSet);
-				}
-
-				if (waitForSolutionSetUpdate) {
-					solutionSetUpdateBarrier = new SolutionSetUpdateBarrier();
-					SolutionSetUpdateBarrierBroker.instance().handIn(brokerKey, solutionSetUpdateBarrier);
-				}
-			}
-			else {
-				// bulk iteration case
-				@SuppressWarnings("unchecked")
-				TypeSerializerFactory<X> solSer = (TypeSerializerFactory<X>) feedbackTypeSerializer;
-				solutionTypeSerializer = solSer;
-				
-				// = termination Criterion tail
-				if (waitForSolutionSetUpdate) {
-					solutionSetUpdateBarrier = new SolutionSetUpdateBarrier();
-					SolutionSetUpdateBarrierBroker.instance().handIn(brokerKey, solutionSetUpdateBarrier);
-				}
-			}
-
-			// instantiate all aggregators and register them at the iteration global registry
-			RuntimeAggregatorRegistry aggregatorRegistry = new RuntimeAggregatorRegistry(config.getIterationAggregators
-					(getUserCodeClassLoader()));
-			IterationAggregatorBroker.instance().handIn(brokerKey, aggregatorRegistry);
-
-			DataInputView superstepResult = null;
-
-			while (this.running && !terminationRequested()) {
-
-				if (log.isInfoEnabled()) {
-					log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
-				}
-
-				barrier.setup();
-
-				if (waitForSolutionSetUpdate) {
-					solutionSetUpdateBarrier.setup();
-				}
-
-				if (!inFirstIteration()) {
-					feedBackSuperstepResult(superstepResult);
-				}
-
-				super.run();
-
-				// signal to connected tasks that we are done with the superstep
-				sendEndOfSuperstepToAllIterationOutputs();
-
-				if (waitForSolutionSetUpdate) {
-					solutionSetUpdateBarrier.waitForSolutionSetUpdate();
-				}
-
-				// blocking call to wait for the result
-				superstepResult = backChannel.getReadEndAfterSuperstepEnded();
-				if (log.isInfoEnabled()) {
-					log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
-				}
-
-				sendEventToSync(new WorkerDoneEvent(workerIndex, aggregatorRegistry.getAllAggregators()));
-
-				if (log.isInfoEnabled()) {
-					log.info(formatLogString("waiting for other workers in iteration [" + currentIteration() + "]"));
-				}
-
-				barrier.waitForOtherWorkers();
-
-				if (barrier.terminationSignaled()) {
-					if (log.isInfoEnabled()) {
-						log.info(formatLogString("head received termination request in iteration ["
-							+ currentIteration()
-							+ "]"));
-					}
-					requestTermination();
-					nextStepKickoff.signalTermination();
-				} else {
-					incrementIterationCounter();
-
-					String[] globalAggregateNames = barrier.getAggregatorNames();
-					Value[] globalAggregates = barrier.getAggregates();
-					aggregatorRegistry.updateGlobalAggregatesAndReset(globalAggregateNames, globalAggregates);
-					
-					nextStepKickoff.triggerNextSuperstep();
-				}
-			}
-
-			if (log.isInfoEnabled()) {
-				log.info(formatLogString("streaming out final result after [" + currentIteration() + "] iterations"));
-			}
-
-			if (isWorksetIteration) {
-				if (objectSolutionSet) {
-					streamSolutionSetToFinalOutput(solutionSetObjectMap);
-				} else {
-					streamSolutionSetToFinalOutput(solutionSet);
-				}
-			} else {
-				streamOutFinalOutputBulk(new InputViewIterator<X>(superstepResult, this.solutionTypeSerializer.getSerializer()));
-			}
-
-			this.finalOutputCollector.close();
-
-		} finally {
-			// make sure we unregister everything from the broker:
-			// - backchannel
-			// - aggregator registry
-			// - solution set index
-			IterationAggregatorBroker.instance().remove(brokerKey);
-			BlockingBackChannelBroker.instance().remove(brokerKey);
-			SuperstepKickoffLatchBroker.instance().remove(brokerKey);
-			SolutionSetBroker.instance().remove(brokerKey);
-			SolutionSetUpdateBarrierBroker.instance().remove(brokerKey);
-
-			if (solutionSet != null) {
-				solutionSet.close();
-			}
-		}
-	}
-
-	private void streamOutFinalOutputBulk(MutableObjectIterator<X> results) throws IOException {
-		final Collector<X> out = this.finalOutputCollector;
-		X record = this.solutionTypeSerializer.getSerializer().createInstance();
-
-		while ((record = results.next(record)) != null) {
-			out.collect(record);
-		}
-	}
-	
-	private void streamSolutionSetToFinalOutput(CompactingHashTable<X> hashTable) throws IOException {
-		final MutableObjectIterator<X> results = hashTable.getEntryIterator();
-		final Collector<X> output = this.finalOutputCollector;
-		X record = solutionTypeSerializer.getSerializer().createInstance();
-
-		while ((record = results.next(record)) != null) {
-			output.collect(record);
-		}
-	}
-	
-	@SuppressWarnings("unchecked")
-	private void streamSolutionSetToFinalOutput(JoinHashMap<X> soluionSet) throws IOException {
-		final Collector<X> output = this.finalOutputCollector;
-		for (Object e : soluionSet.values()) {
-			output.collect((X) e);
-		}
-	}
-
-	private void feedBackSuperstepResult(DataInputView superstepResult) {
-		this.inputs[this.feedbackDataInput] =
-			new InputViewIterator<Y>(superstepResult, this.feedbackTypeSerializer.getSerializer());
-	}
-
-	private void sendEndOfSuperstepToAllIterationOutputs() throws IOException, InterruptedException {
-		if (log.isDebugEnabled()) {
-			log.debug(formatLogString("Sending end-of-superstep to all iteration outputs."));
-		}
-
-		for (RecordWriter<?> eventualOutput : this.eventualOutputs) {
-			eventualOutput.sendEndOfSuperstep();
-		}
-	}
-
-	private void sendEventToSync(WorkerDoneEvent event) throws IOException, InterruptedException {
-		if (log.isInfoEnabled()) {
-			log.info(formatLogString("sending " + WorkerDoneEvent.class.getSimpleName() + " to sync"));
-		}
-		this.toSync.writeEventToAllChannels(event);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java
new file mode 100644
index 0000000..c6268f4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java
@@ -0,0 +1,441 @@
+/*
+ * 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.runtime.iterative.task;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.operators.Driver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.operators.util.JoinHashMap;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.InputViewIterator;
+import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel;
+import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannelBroker;
+import org.apache.flink.runtime.iterative.concurrent.Broker;
+import org.apache.flink.runtime.iterative.concurrent.IterationAggregatorBroker;
+import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
+import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrier;
+import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepBarrier;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatchBroker;
+import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent;
+import org.apache.flink.runtime.iterative.event.TerminationEvent;
+import org.apache.flink.runtime.iterative.event.WorkerDoneEvent;
+import org.apache.flink.runtime.iterative.io.SerializedUpdateBuffer;
+import org.apache.flink.runtime.operators.BatchTask;
+import org.apache.flink.runtime.operators.hash.CompactingHashTable;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+/**
+ * The head is responsible for coordinating an iteration and can run a
+ * {@link Driver} inside. It will read
+ * the initial input and establish a {@link BlockingBackChannel} to the iteration's tail. After successfully processing
+ * the input, it will send EndOfSuperstep events to its outputs. It must also be connected to a
+ * synchronization task and after each superstep, it will wait
+ * until it receives an {@link AllWorkersDoneEvent} from the sync, which signals that all other heads have also finished
+ * their iteration. Starting with
+ * the second iteration, the input for the head is the output of the tail, transmitted through the backchannel. Once the
+ * iteration is done, the head
+ * will send a {@link TerminationEvent} to all it's connected tasks, signaling them to shutdown.
+ * <p>
+ * Assumption on the ordering of the outputs: - The first n output gates write to channels that go to the tasks of the
+ * step function. - The next m output gates to to the tasks that consume the final solution. - The last output gate
+ * connects to the synchronization task.
+ * 
+ * @param <X>
+ *        The type of the bulk partial solution / solution set and the final output.
+ * @param <Y>
+ *        The type of the feed-back data set (bulk partial solution / workset). For bulk iterations, {@code Y} is the
+ *        same as {@code X}
+ */
+public class IterationHeadTask<X, Y, S extends Function, OT> extends AbstractIterativeTask<S, OT> {
+
+	private static final Logger log = LoggerFactory.getLogger(IterationHeadTask.class);
+
+	private Collector<X> finalOutputCollector;
+
+	private TypeSerializerFactory<Y> feedbackTypeSerializer;
+
+	private TypeSerializerFactory<X> solutionTypeSerializer;
+
+	private ResultPartitionWriter toSync;
+
+	private int feedbackDataInput; // workset or bulk partial solution
+
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	protected int getNumTaskInputs() {
+		// this task has an additional input in the workset case for the initial solution set
+		boolean isWorkset = config.getIsWorksetIteration();
+		return driver.getNumberOfInputs() + (isWorkset ? 1 : 0);
+	}
+
+	@Override
+	protected void initOutputs() throws Exception {
+		// initialize the regular outputs first (the ones into the step function).
+		super.initOutputs();
+
+		// at this time, the outputs to the step function are created
+		// add the outputs for the final solution
+		List<RecordWriter<?>> finalOutputWriters = new ArrayList<RecordWriter<?>>();
+		final TaskConfig finalOutConfig = this.config.getIterationHeadFinalOutputConfig();
+		final ClassLoader userCodeClassLoader = getUserCodeClassLoader();
+		AccumulatorRegistry.Reporter reporter = getEnvironment().getAccumulatorRegistry().getReadWriteReporter();
+		this.finalOutputCollector = BatchTask.getOutputCollector(this, finalOutConfig,
+				userCodeClassLoader, finalOutputWriters, config.getNumOutputs(), finalOutConfig.getNumOutputs(), reporter);
+
+		// sanity check the setup
+		final int writersIntoStepFunction = this.eventualOutputs.size();
+		final int writersIntoFinalResult = finalOutputWriters.size();
+		final int syncGateIndex = this.config.getIterationHeadIndexOfSyncOutput();
+
+		if (writersIntoStepFunction + writersIntoFinalResult != syncGateIndex) {
+			throw new Exception("Error: Inconsistent head task setup - wrong mapping of output gates.");
+		}
+		// now, we can instantiate the sync gate
+		this.toSync = getEnvironment().getWriter(syncGateIndex);
+	}
+
+	/**
+	 * the iteration head prepares the backchannel: it allocates memory, instantiates a {@link BlockingBackChannel} and
+	 * hands it to the iteration tail via a {@link Broker} singleton
+	 **/
+	private BlockingBackChannel initBackChannel() throws Exception {
+
+		/* get the size of the memory available to the backchannel */
+		int backChannelMemoryPages = getMemoryManager().computeNumberOfPages(this.config.getRelativeBackChannelMemory());
+
+		/* allocate the memory available to the backchannel */
+		List<MemorySegment> segments = new ArrayList<MemorySegment>();
+		int segmentSize = getMemoryManager().getPageSize();
+		getMemoryManager().allocatePages(this, segments, backChannelMemoryPages);
+
+		/* instantiate the backchannel */
+		BlockingBackChannel backChannel = new BlockingBackChannel(new SerializedUpdateBuffer(segments, segmentSize,
+			getIOManager()));
+
+		/* hand the backchannel over to the iteration tail */
+		Broker<BlockingBackChannel> broker = BlockingBackChannelBroker.instance();
+		broker.handIn(brokerKey(), backChannel);
+
+		return backChannel;
+	}
+	
+	private <BT> CompactingHashTable<BT> initCompactingHashTable() throws Exception {
+		// get some memory
+		double hashjoinMemorySize = config.getRelativeSolutionSetMemory();
+		final ClassLoader userCodeClassLoader = getUserCodeClassLoader();
+
+		TypeSerializerFactory<BT> solutionTypeSerializerFactory = config.getSolutionSetSerializer(userCodeClassLoader);
+		TypeComparatorFactory<BT> solutionTypeComparatorFactory = config.getSolutionSetComparator(userCodeClassLoader);
+	
+		TypeSerializer<BT> solutionTypeSerializer = solutionTypeSerializerFactory.getSerializer();
+		TypeComparator<BT> solutionTypeComparator = solutionTypeComparatorFactory.createComparator();
+
+		CompactingHashTable<BT> hashTable = null;
+		List<MemorySegment> memSegments = null;
+		boolean success = false;
+		try {
+			int numPages = getMemoryManager().computeNumberOfPages(hashjoinMemorySize);
+			memSegments = getMemoryManager().allocatePages(getOwningNepheleTask(), numPages);
+			hashTable = new CompactingHashTable<BT>(solutionTypeSerializer, solutionTypeComparator, memSegments);
+			success = true;
+			return hashTable;
+		} finally {
+			if (!success) {
+				if (hashTable != null) {
+					try {
+						hashTable.close();
+					} catch (Throwable t) {
+						log.error("Error closing the solution set hash table after unsuccessful creation.", t);
+					}
+				}
+				if (memSegments != null) {
+					try {
+						getMemoryManager().release(memSegments);
+					} catch (Throwable t) {
+						log.error("Error freeing memory after error during solution set hash table creation.", t);
+					}
+				}
+			}
+		}
+	}
+	
+	private <BT> JoinHashMap<BT> initJoinHashMap() {
+		TypeSerializerFactory<BT> solutionTypeSerializerFactory = config.getSolutionSetSerializer
+				(getUserCodeClassLoader());
+		TypeComparatorFactory<BT> solutionTypeComparatorFactory = config.getSolutionSetComparator
+				(getUserCodeClassLoader());
+	
+		TypeSerializer<BT> solutionTypeSerializer = solutionTypeSerializerFactory.getSerializer();
+		TypeComparator<BT> solutionTypeComparator = solutionTypeComparatorFactory.createComparator();
+		
+		return new JoinHashMap<BT>(solutionTypeSerializer, solutionTypeComparator);
+	}
+	
+	private void readInitialSolutionSet(CompactingHashTable<X> solutionSet, MutableObjectIterator<X> solutionSetInput) throws IOException {
+		solutionSet.open();
+		solutionSet.buildTableWithUniqueKey(solutionSetInput);
+	}
+	
+	private void readInitialSolutionSet(JoinHashMap<X> solutionSet, MutableObjectIterator<X> solutionSetInput) throws IOException {
+		TypeSerializer<X> serializer = solutionTypeSerializer.getSerializer();
+		
+		X next;
+		while ((next = solutionSetInput.next(serializer.createInstance())) != null) {
+			solutionSet.insertOrReplace(next);
+		}
+	}
+
+	private SuperstepBarrier initSuperstepBarrier() {
+		SuperstepBarrier barrier = new SuperstepBarrier(getUserCodeClassLoader());
+		this.toSync.subscribeToEvent(barrier, AllWorkersDoneEvent.class);
+		this.toSync.subscribeToEvent(barrier, TerminationEvent.class);
+		return barrier;
+	}
+
+	@Override
+	public void run() throws Exception {
+		final String brokerKey = brokerKey();
+		final int workerIndex = getEnvironment().getIndexInSubtaskGroup();
+		
+		final boolean objectSolutionSet = config.isSolutionSetUnmanaged();
+
+		CompactingHashTable<X> solutionSet = null; // if workset iteration
+		JoinHashMap<X> solutionSetObjectMap = null; // if workset iteration with unmanaged solution set
+		
+		boolean waitForSolutionSetUpdate = config.getWaitForSolutionSetUpdate();
+		boolean isWorksetIteration = config.getIsWorksetIteration();
+
+		try {
+			/* used for receiving the current iteration result from iteration tail */
+			SuperstepKickoffLatch nextStepKickoff = new SuperstepKickoffLatch();
+			SuperstepKickoffLatchBroker.instance().handIn(brokerKey, nextStepKickoff);
+			
+			BlockingBackChannel backChannel = initBackChannel();
+			SuperstepBarrier barrier = initSuperstepBarrier();
+			SolutionSetUpdateBarrier solutionSetUpdateBarrier = null;
+
+			feedbackDataInput = config.getIterationHeadPartialSolutionOrWorksetInputIndex();
+			feedbackTypeSerializer = this.getInputSerializer(feedbackDataInput);
+			excludeFromReset(feedbackDataInput);
+
+			int initialSolutionSetInput;
+			if (isWorksetIteration) {
+				initialSolutionSetInput = config.getIterationHeadSolutionSetInputIndex();
+				solutionTypeSerializer = config.getSolutionSetSerializer(getUserCodeClassLoader());
+
+				// setup the index for the solution set
+				@SuppressWarnings("unchecked")
+				MutableObjectIterator<X> solutionSetInput = (MutableObjectIterator<X>) createInputIterator(inputReaders[initialSolutionSetInput], solutionTypeSerializer);
+				
+				// read the initial solution set
+				if (objectSolutionSet) {
+					solutionSetObjectMap = initJoinHashMap();
+					readInitialSolutionSet(solutionSetObjectMap, solutionSetInput);
+					SolutionSetBroker.instance().handIn(brokerKey, solutionSetObjectMap);
+				} else {
+					solutionSet = initCompactingHashTable();
+					readInitialSolutionSet(solutionSet, solutionSetInput);
+					SolutionSetBroker.instance().handIn(brokerKey, solutionSet);
+				}
+
+				if (waitForSolutionSetUpdate) {
+					solutionSetUpdateBarrier = new SolutionSetUpdateBarrier();
+					SolutionSetUpdateBarrierBroker.instance().handIn(brokerKey, solutionSetUpdateBarrier);
+				}
+			}
+			else {
+				// bulk iteration case
+				@SuppressWarnings("unchecked")
+				TypeSerializerFactory<X> solSer = (TypeSerializerFactory<X>) feedbackTypeSerializer;
+				solutionTypeSerializer = solSer;
+				
+				// = termination Criterion tail
+				if (waitForSolutionSetUpdate) {
+					solutionSetUpdateBarrier = new SolutionSetUpdateBarrier();
+					SolutionSetUpdateBarrierBroker.instance().handIn(brokerKey, solutionSetUpdateBarrier);
+				}
+			}
+
+			// instantiate all aggregators and register them at the iteration global registry
+			RuntimeAggregatorRegistry aggregatorRegistry = new RuntimeAggregatorRegistry(config.getIterationAggregators
+					(getUserCodeClassLoader()));
+			IterationAggregatorBroker.instance().handIn(brokerKey, aggregatorRegistry);
+
+			DataInputView superstepResult = null;
+
+			while (this.running && !terminationRequested()) {
+
+				if (log.isInfoEnabled()) {
+					log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
+				}
+
+				barrier.setup();
+
+				if (waitForSolutionSetUpdate) {
+					solutionSetUpdateBarrier.setup();
+				}
+
+				if (!inFirstIteration()) {
+					feedBackSuperstepResult(superstepResult);
+				}
+
+				super.run();
+
+				// signal to connected tasks that we are done with the superstep
+				sendEndOfSuperstepToAllIterationOutputs();
+
+				if (waitForSolutionSetUpdate) {
+					solutionSetUpdateBarrier.waitForSolutionSetUpdate();
+				}
+
+				// blocking call to wait for the result
+				superstepResult = backChannel.getReadEndAfterSuperstepEnded();
+				if (log.isInfoEnabled()) {
+					log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
+				}
+
+				sendEventToSync(new WorkerDoneEvent(workerIndex, aggregatorRegistry.getAllAggregators()));
+
+				if (log.isInfoEnabled()) {
+					log.info(formatLogString("waiting for other workers in iteration [" + currentIteration() + "]"));
+				}
+
+				barrier.waitForOtherWorkers();
+
+				if (barrier.terminationSignaled()) {
+					if (log.isInfoEnabled()) {
+						log.info(formatLogString("head received termination request in iteration ["
+							+ currentIteration()
+							+ "]"));
+					}
+					requestTermination();
+					nextStepKickoff.signalTermination();
+				} else {
+					incrementIterationCounter();
+
+					String[] globalAggregateNames = barrier.getAggregatorNames();
+					Value[] globalAggregates = barrier.getAggregates();
+					aggregatorRegistry.updateGlobalAggregatesAndReset(globalAggregateNames, globalAggregates);
+					
+					nextStepKickoff.triggerNextSuperstep();
+				}
+			}
+
+			if (log.isInfoEnabled()) {
+				log.info(formatLogString("streaming out final result after [" + currentIteration() + "] iterations"));
+			}
+
+			if (isWorksetIteration) {
+				if (objectSolutionSet) {
+					streamSolutionSetToFinalOutput(solutionSetObjectMap);
+				} else {
+					streamSolutionSetToFinalOutput(solutionSet);
+				}
+			} else {
+				streamOutFinalOutputBulk(new InputViewIterator<X>(superstepResult, this.solutionTypeSerializer.getSerializer()));
+			}
+
+			this.finalOutputCollector.close();
+
+		} finally {
+			// make sure we unregister everything from the broker:
+			// - backchannel
+			// - aggregator registry
+			// - solution set index
+			IterationAggregatorBroker.instance().remove(brokerKey);
+			BlockingBackChannelBroker.instance().remove(brokerKey);
+			SuperstepKickoffLatchBroker.instance().remove(brokerKey);
+			SolutionSetBroker.instance().remove(brokerKey);
+			SolutionSetUpdateBarrierBroker.instance().remove(brokerKey);
+
+			if (solutionSet != null) {
+				solutionSet.close();
+			}
+		}
+	}
+
+	private void streamOutFinalOutputBulk(MutableObjectIterator<X> results) throws IOException {
+		final Collector<X> out = this.finalOutputCollector;
+		X record = this.solutionTypeSerializer.getSerializer().createInstance();
+
+		while ((record = results.next(record)) != null) {
+			out.collect(record);
+		}
+	}
+	
+	private void streamSolutionSetToFinalOutput(CompactingHashTable<X> hashTable) throws IOException {
+		final MutableObjectIterator<X> results = hashTable.getEntryIterator();
+		final Collector<X> output = this.finalOutputCollector;
+		X record = solutionTypeSerializer.getSerializer().createInstance();
+
+		while ((record = results.next(record)) != null) {
+			output.collect(record);
+		}
+	}
+	
+	@SuppressWarnings("unchecked")
+	private void streamSolutionSetToFinalOutput(JoinHashMap<X> soluionSet) throws IOException {
+		final Collector<X> output = this.finalOutputCollector;
+		for (Object e : soluionSet.values()) {
+			output.collect((X) e);
+		}
+	}
+
+	private void feedBackSuperstepResult(DataInputView superstepResult) {
+		this.inputs[this.feedbackDataInput] =
+			new InputViewIterator<Y>(superstepResult, this.feedbackTypeSerializer.getSerializer());
+	}
+
+	private void sendEndOfSuperstepToAllIterationOutputs() throws IOException, InterruptedException {
+		if (log.isDebugEnabled()) {
+			log.debug(formatLogString("Sending end-of-superstep to all iteration outputs."));
+		}
+
+		for (RecordWriter<?> eventualOutput : this.eventualOutputs) {
+			eventualOutput.sendEndOfSuperstep();
+		}
+	}
+
+	private void sendEventToSync(WorkerDoneEvent event) throws IOException, InterruptedException {
+		if (log.isInfoEnabled()) {
+			log.info(formatLogString("sending " + WorkerDoneEvent.class.getSimpleName() + " to sync"));
+		}
+		this.toSync.writeEventToAllChannels(event);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java
deleted file mode 100644
index e7801e4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediatePactTask.java
+++ /dev/null
@@ -1,131 +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.runtime.iterative.task;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
-import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
-import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatchBroker;
-import org.apache.flink.runtime.iterative.event.TerminationEvent;
-import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector;
-import org.apache.flink.util.Collector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * An intermediate iteration task, which runs a Driver}inside.
- * <p>
- * It will propagate {@link EndOfSuperstepEvent}s and {@link TerminationEvent}s to it's connected tasks. Furthermore
- * intermediate tasks can also update the iteration state, either the workset or the solution set.
- * <p>
- * If the iteration state is updated, the output of this task will be send back to the {@link IterationHeadPactTask} via
- * a {@link BlockingBackChannel} for the workset -XOR- a eHashTable for the solution set. In this case
- * this task must be scheduled on the same instance as the head.
- */
-public class IterationIntermediatePactTask<S extends Function, OT> extends AbstractIterativePactTask<S, OT> {
-
-	private static final Logger log = LoggerFactory.getLogger(IterationIntermediatePactTask.class);
-
-	private WorksetUpdateOutputCollector<OT> worksetUpdateOutputCollector;
-
-	@Override
-	protected void initialize() throws Exception {
-		super.initialize();
-
-		// set the last output collector of this task to reflect the iteration intermediate state update
-		// a) workset update
-		// b) solution set update
-		// c) none
-
-		Collector<OT> delegate = getLastOutputCollector();
-		if (isWorksetUpdate) {
-			// sanity check: we should not have a solution set and workset update at the same time
-			// in an intermediate task
-			if (isSolutionSetUpdate) {
-				throw new IllegalStateException("Plan bug: Intermediate task performs workset and solutions set update.");
-			}
-			
-			Collector<OT> outputCollector = createWorksetUpdateOutputCollector(delegate);
-
-			// we need the WorksetUpdateOutputCollector separately to count the collected elements
-			if (isWorksetIteration) {
-				worksetUpdateOutputCollector = (WorksetUpdateOutputCollector<OT>) outputCollector;
-			}
-
-			setLastOutputCollector(outputCollector);
-		} else if (isSolutionSetUpdate) {
-			setLastOutputCollector(createSolutionSetUpdateOutputCollector(delegate));
-		}
-	}
-
-	@Override
-	public void run() throws Exception {
-		
-		SuperstepKickoffLatch nextSuperstepLatch = SuperstepKickoffLatchBroker.instance().get(brokerKey());
-
-		while (this.running && !terminationRequested()) {
-
-			if (log.isInfoEnabled()) {
-				log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
-			}
-
-			super.run();
-
-			// check if termination was requested
-			verifyEndOfSuperstepState();
-
-			if (isWorksetUpdate && isWorksetIteration) {
-				long numCollected = worksetUpdateOutputCollector.getElementsCollectedAndReset();
-				worksetAggregator.aggregate(numCollected);
-			}
-			
-			if (log.isInfoEnabled()) {
-				log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
-			}
-			
-			// let the successors know that the end of this superstep data is reached
-			sendEndOfSuperstep();
-			
-			if (isWorksetUpdate) {
-				// notify iteration head if responsible for workset update
-				worksetBackChannel.notifyOfEndOfSuperstep();
-			}
-			
-			boolean terminated = nextSuperstepLatch.awaitStartOfSuperstepOrTermination(currentIteration() + 1);
-
-			if (terminated) {
-				requestTermination();
-			}
-			else {
-				incrementIterationCounter();
-			}
-		}
-	}
-
-	private void sendEndOfSuperstep() throws IOException, InterruptedException {
-		for (RecordWriter eventualOutput : this.eventualOutputs) {
-			eventualOutput.sendEndOfSuperstep();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java
new file mode 100644
index 0000000..60f0dcf
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.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.runtime.iterative.task;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
+import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatchBroker;
+import org.apache.flink.runtime.iterative.event.TerminationEvent;
+import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector;
+import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * An intermediate iteration task, which runs a Driver}inside.
+ * <p>
+ * It will propagate {@link EndOfSuperstepEvent}s and {@link TerminationEvent}s to it's connected tasks. Furthermore
+ * intermediate tasks can also update the iteration state, either the workset or the solution set.
+ * <p>
+ * If the iteration state is updated, the output of this task will be send back to the {@link IterationHeadTask} via
+ * a {@link BlockingBackChannel} for the workset -XOR- a eHashTable for the solution set. In this case
+ * this task must be scheduled on the same instance as the head.
+ */
+public class IterationIntermediateTask<S extends Function, OT> extends AbstractIterativeTask<S, OT> {
+
+	private static final Logger log = LoggerFactory.getLogger(IterationIntermediateTask.class);
+
+	private WorksetUpdateOutputCollector<OT> worksetUpdateOutputCollector;
+
+	@Override
+	protected void initialize() throws Exception {
+		super.initialize();
+
+		// set the last output collector of this task to reflect the iteration intermediate state update
+		// a) workset update
+		// b) solution set update
+		// c) none
+
+		Collector<OT> delegate = getLastOutputCollector();
+		if (isWorksetUpdate) {
+			// sanity check: we should not have a solution set and workset update at the same time
+			// in an intermediate task
+			if (isSolutionSetUpdate) {
+				throw new IllegalStateException("Plan bug: Intermediate task performs workset and solutions set update.");
+			}
+			
+			Collector<OT> outputCollector = createWorksetUpdateOutputCollector(delegate);
+
+			// we need the WorksetUpdateOutputCollector separately to count the collected elements
+			if (isWorksetIteration) {
+				worksetUpdateOutputCollector = (WorksetUpdateOutputCollector<OT>) outputCollector;
+			}
+
+			setLastOutputCollector(outputCollector);
+		} else if (isSolutionSetUpdate) {
+			setLastOutputCollector(createSolutionSetUpdateOutputCollector(delegate));
+		}
+	}
+
+	@Override
+	public void run() throws Exception {
+		
+		SuperstepKickoffLatch nextSuperstepLatch = SuperstepKickoffLatchBroker.instance().get(brokerKey());
+
+		while (this.running && !terminationRequested()) {
+
+			if (log.isInfoEnabled()) {
+				log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
+			}
+
+			super.run();
+
+			// check if termination was requested
+			verifyEndOfSuperstepState();
+
+			if (isWorksetUpdate && isWorksetIteration) {
+				long numCollected = worksetUpdateOutputCollector.getElementsCollectedAndReset();
+				worksetAggregator.aggregate(numCollected);
+			}
+			
+			if (log.isInfoEnabled()) {
+				log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
+			}
+			
+			// let the successors know that the end of this superstep data is reached
+			sendEndOfSuperstep();
+			
+			if (isWorksetUpdate) {
+				// notify iteration head if responsible for workset update
+				worksetBackChannel.notifyOfEndOfSuperstep();
+			}
+			
+			boolean terminated = nextSuperstepLatch.awaitStartOfSuperstepOrTermination(currentIteration() + 1);
+
+			if (terminated) {
+				requestTermination();
+			}
+			else {
+				incrementIterationCounter();
+			}
+		}
+	}
+
+	private void sendEndOfSuperstep() throws IOException, InterruptedException {
+		for (RecordWriter eventualOutput : this.eventualOutputs) {
+			eventualOutput.sendEndOfSuperstep();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java
index fed0a17..a85e662 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationSynchronizationSinkTask.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.types.IntValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,7 +37,6 @@ import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent;
 import org.apache.flink.runtime.iterative.event.TerminationEvent;
 import org.apache.flink.runtime.iterative.event.WorkerDoneEvent;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.operators.RegularPactTask;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.types.Value;
 
@@ -204,7 +204,7 @@ public class IterationSynchronizationSinkTask extends AbstractInvokable implemen
 	}
 
 	private String formatLogString(String message) {
-		return RegularPactTask.constructLogString(message, getEnvironment().getTaskName(), this);
+		return BatchTask.constructLogString(message, getEnvironment().getTaskName(), this);
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java
deleted file mode 100644
index 159d3f2..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailPactTask.java
+++ /dev/null
@@ -1,140 +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.runtime.iterative.task;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrier;
-import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch;
-import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatchBroker;
-import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector;
-import org.apache.flink.util.Collector;
-
-/**
- * An iteration tail, which runs a driver inside.
- * <p>
- * If the iteration state is updated, the output of this task will be send back to the {@link IterationHeadPactTask} via
- * a BackChannel for the workset -OR- a HashTable for the solution set. Therefore this
- * task must be scheduled on the same instance as the head. It's also possible for the tail to update *both* the workset
- * and the solution set.
- * <p>
- * If there is a separate solution set tail, the iteration head has to make sure to wait for it to finish.
- */
-public class IterationTailPactTask<S extends Function, OT> extends AbstractIterativePactTask<S, OT> {
-
-	private static final Logger log = LoggerFactory.getLogger(IterationTailPactTask.class);
-
-	private SolutionSetUpdateBarrier solutionSetUpdateBarrier;
-
-	private WorksetUpdateOutputCollector<OT> worksetUpdateOutputCollector;
-	
-
-	@Override
-	protected void initialize() throws Exception {
-		super.initialize();
-
-		// sanity check: the tail has to update either the workset or the solution set
-		if (!isWorksetUpdate && !isSolutionSetUpdate) {
-			throw new RuntimeException("The iteration tail doesn't update workset or the solution set.");
-		}
-
-		// set the last output collector of this task to reflect the iteration tail state update:
-		// a) workset update,
-		// b) solution set update, or
-		// c) merged workset and solution set update
-
-		Collector<OT> outputCollector = null;
-		if (isWorksetUpdate) {
-			outputCollector = createWorksetUpdateOutputCollector();
-
-			// we need the WorksetUpdateOutputCollector separately to count the collected elements
-			if (isWorksetIteration) {
-				worksetUpdateOutputCollector = (WorksetUpdateOutputCollector<OT>) outputCollector;
-			}
-		}
-
-		if (isSolutionSetUpdate) {
-			if (isWorksetIteration) {
-				outputCollector = createSolutionSetUpdateOutputCollector(outputCollector);
-			}
-			// Bulk iteration with termination criterion
-			else {
-				outputCollector = new Collector<OT>() {
-					@Override
-					public void collect(OT record) {}
-					@Override
-					public void close() {}
-				};
-			}
-
-			if (!isWorksetUpdate) {
-				solutionSetUpdateBarrier = SolutionSetUpdateBarrierBroker.instance().get(brokerKey());
-			}
-		}
-
-		setLastOutputCollector(outputCollector);
-	}
-
-	@Override
-	public void run() throws Exception {
-		
-		SuperstepKickoffLatch nextSuperStepLatch = SuperstepKickoffLatchBroker.instance().get(brokerKey());
-		
-		while (this.running && !terminationRequested()) {
-
-			if (log.isInfoEnabled()) {
-				log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
-			}
-
-			super.run();
-
-			// check if termination was requested
-			verifyEndOfSuperstepState();
-
-			if (isWorksetUpdate && isWorksetIteration) {
-				// aggregate workset update element count
-				long numCollected = worksetUpdateOutputCollector.getElementsCollectedAndReset();
-				worksetAggregator.aggregate(numCollected);
-
-			}
-
-			if (log.isInfoEnabled()) {
-				log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
-			}
-			
-			if (isWorksetUpdate) {
-				// notify iteration head if responsible for workset update
-				worksetBackChannel.notifyOfEndOfSuperstep();
-			} else if (isSolutionSetUpdate) {
-				// notify iteration head if responsible for solution set update
-				solutionSetUpdateBarrier.notifySolutionSetUpdate();
-			}
-
-			boolean terminate = nextSuperStepLatch.awaitStartOfSuperstepOrTermination(currentIteration() + 1);
-			if (terminate) {
-				requestTermination();
-			}
-			else {
-				incrementIterationCounter();
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java
new file mode 100644
index 0000000..9e0b560
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationTailTask.java
@@ -0,0 +1,140 @@
+/*
+ * 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.runtime.iterative.task;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrier;
+import org.apache.flink.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatch;
+import org.apache.flink.runtime.iterative.concurrent.SuperstepKickoffLatchBroker;
+import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector;
+import org.apache.flink.util.Collector;
+
+/**
+ * An iteration tail, which runs a driver inside.
+ * <p>
+ * If the iteration state is updated, the output of this task will be send back to the {@link IterationHeadTask} via
+ * a BackChannel for the workset -OR- a HashTable for the solution set. Therefore this
+ * task must be scheduled on the same instance as the head. It's also possible for the tail to update *both* the workset
+ * and the solution set.
+ * <p>
+ * If there is a separate solution set tail, the iteration head has to make sure to wait for it to finish.
+ */
+public class IterationTailTask<S extends Function, OT> extends AbstractIterativeTask<S, OT> {
+
+	private static final Logger log = LoggerFactory.getLogger(IterationTailTask.class);
+
+	private SolutionSetUpdateBarrier solutionSetUpdateBarrier;
+
+	private WorksetUpdateOutputCollector<OT> worksetUpdateOutputCollector;
+	
+
+	@Override
+	protected void initialize() throws Exception {
+		super.initialize();
+
+		// sanity check: the tail has to update either the workset or the solution set
+		if (!isWorksetUpdate && !isSolutionSetUpdate) {
+			throw new RuntimeException("The iteration tail doesn't update workset or the solution set.");
+		}
+
+		// set the last output collector of this task to reflect the iteration tail state update:
+		// a) workset update,
+		// b) solution set update, or
+		// c) merged workset and solution set update
+
+		Collector<OT> outputCollector = null;
+		if (isWorksetUpdate) {
+			outputCollector = createWorksetUpdateOutputCollector();
+
+			// we need the WorksetUpdateOutputCollector separately to count the collected elements
+			if (isWorksetIteration) {
+				worksetUpdateOutputCollector = (WorksetUpdateOutputCollector<OT>) outputCollector;
+			}
+		}
+
+		if (isSolutionSetUpdate) {
+			if (isWorksetIteration) {
+				outputCollector = createSolutionSetUpdateOutputCollector(outputCollector);
+			}
+			// Bulk iteration with termination criterion
+			else {
+				outputCollector = new Collector<OT>() {
+					@Override
+					public void collect(OT record) {}
+					@Override
+					public void close() {}
+				};
+			}
+
+			if (!isWorksetUpdate) {
+				solutionSetUpdateBarrier = SolutionSetUpdateBarrierBroker.instance().get(brokerKey());
+			}
+		}
+
+		setLastOutputCollector(outputCollector);
+	}
+
+	@Override
+	public void run() throws Exception {
+		
+		SuperstepKickoffLatch nextSuperStepLatch = SuperstepKickoffLatchBroker.instance().get(brokerKey());
+		
+		while (this.running && !terminationRequested()) {
+
+			if (log.isInfoEnabled()) {
+				log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
+			}
+
+			super.run();
+
+			// check if termination was requested
+			verifyEndOfSuperstepState();
+
+			if (isWorksetUpdate && isWorksetIteration) {
+				// aggregate workset update element count
+				long numCollected = worksetUpdateOutputCollector.getElementsCollectedAndReset();
+				worksetAggregator.aggregate(numCollected);
+
+			}
+
+			if (log.isInfoEnabled()) {
+				log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
+			}
+			
+			if (isWorksetUpdate) {
+				// notify iteration head if responsible for workset update
+				worksetBackChannel.notifyOfEndOfSuperstep();
+			} else if (isSolutionSetUpdate) {
+				// notify iteration head if responsible for solution set update
+				solutionSetUpdateBarrier.notifySolutionSetUpdate();
+			}
+
+			boolean terminate = nextSuperStepLatch.awaitStartOfSuperstepOrTermination(currentIteration() + 1);
+			if (terminate) {
+				requestTermination();
+			}
+			else {
+				incrementIterationCounter();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
index 0e0bd26..4923b3b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobgraph.tasks;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.util.InstantiationUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,7 +29,7 @@ import org.slf4j.LoggerFactory;
 /**
  * This is the abstract base class for every task that can be executed by a TaskManager.
  * Concrete tasks like the stream vertices of the batch tasks
- * (see {@link org.apache.flink.runtime.operators.RegularPactTask}) inherit from this class.
+ * (see {@link BatchTask}) inherit from this class.
  *
  * The TaskManager invokes the methods {@link #registerInputOutput()} and {@link #invoke()} in
  * this order when executing a task. The first method is responsible for setting up input and

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
index 4096f0c..8f72754 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractCachedBuildSideJoinDriver.java
@@ -34,7 +34,7 @@ import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
-public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends JoinDriver<IT1, IT2, OT> implements ResettablePactDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
+public abstract class AbstractCachedBuildSideJoinDriver<IT1, IT2, OT> extends JoinDriver<IT1, IT2, OT> implements ResettableDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
 
 	private volatile JoinTaskIterator<IT1, IT2, OT> matchIterator;
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
index 38c74e0..8c964d4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
@@ -38,11 +38,11 @@ import org.slf4j.LoggerFactory;
  *
  * @see FlatJoinFunction
  */
-public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements PactDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
+public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements Driver<FlatJoinFunction<IT1, IT2, OT>, OT> {
 	
 	protected static final Logger LOG = LoggerFactory.getLogger(AbstractOuterJoinDriver.class);
 	
-	protected PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
+	protected TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> taskContext;
 	
 	protected volatile JoinTaskIterator<IT1, IT2, OT> outerJoinIterator; // the iterator that does the actual outer join
 	protected volatile boolean running;
@@ -50,7 +50,7 @@ public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements PactDrive
 	// ------------------------------------------------------------------------
 	
 	@Override
-	public void setup(PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
+	public void setup(TaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> context) {
 		this.taskContext = context;
 		this.running = true;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupCombineDriver.java
index 7b279ee..0c8dc34 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupCombineDriver.java
@@ -37,18 +37,18 @@ import org.slf4j.LoggerFactory;
 *
 * @see GroupCombineFunction
 */
-public class AllGroupCombineDriver<IN, OUT> implements PactDriver<GroupCombineFunction<IN, OUT>, OUT> {
+public class AllGroupCombineDriver<IN, OUT> implements Driver<GroupCombineFunction<IN, OUT>, OUT> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(AllGroupCombineDriver.class);
 
-	private PactTaskContext<GroupCombineFunction<IN, OUT>, OUT> taskContext;
+	private TaskContext<GroupCombineFunction<IN, OUT>, OUT> taskContext;
 
 	private boolean objectReuseEnabled = false;
 
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<GroupCombineFunction<IN, OUT>, OUT> context) {
+	public void setup(TaskContext<GroupCombineFunction<IN, OUT>, OUT> context) {
 		this.taskContext = context;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java
index a20fddf..255c57c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllGroupReduceDriver.java
@@ -45,11 +45,11 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @see org.apache.flink.api.common.functions.GroupReduceFunction
  */
-public class AllGroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunction<IT, OT>, OT> {
+public class AllGroupReduceDriver<IT, OT> implements Driver<GroupReduceFunction<IT, OT>, OT> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(AllGroupReduceDriver.class);
 
-	private PactTaskContext<GroupReduceFunction<IT, OT>, OT> taskContext;
+	private TaskContext<GroupReduceFunction<IT, OT>, OT> taskContext;
 	
 	private MutableObjectIterator<IT> input;
 
@@ -62,7 +62,7 @@ public class AllGroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunct
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<GroupReduceFunction<IT, OT>, OT> context) {
+	public void setup(TaskContext<GroupReduceFunction<IT, OT>, OT> context) {
 		this.taskContext = context;
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java
index 1f58c1b..f27ae34 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AllReduceDriver.java
@@ -38,11 +38,11 @@ import org.apache.flink.util.MutableObjectIterator;
  * 
  * @see org.apache.flink.api.common.functions.ReduceFunction
  */
-public class AllReduceDriver<T> implements PactDriver<ReduceFunction<T>, T> {
+public class AllReduceDriver<T> implements Driver<ReduceFunction<T>, T> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(AllReduceDriver.class);
 
-	private PactTaskContext<ReduceFunction<T>, T> taskContext;
+	private TaskContext<ReduceFunction<T>, T> taskContext;
 	
 	private MutableObjectIterator<T> input;
 
@@ -55,7 +55,7 @@ public class AllReduceDriver<T> implements PactDriver<ReduceFunction<T>, T> {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void setup(PactTaskContext<ReduceFunction<T>, T> context) {
+	public void setup(TaskContext<ReduceFunction<T>, T> context) {
 		this.taskContext = context;
 		this.running = true;
 	}


[5/5] flink git commit: [FLINK-2815] [REFACTOR] Remove Pact from class and file names since it is no longer valid reference

Posted by hs...@apache.org.
[FLINK-2815] [REFACTOR] Remove Pact from class and file names since it is no longer valid reference

Remove Pact word from class and file names in Apache Flink.
Pact was the name used in Stratosphere time to refer to concept of distributed datasets (similar to Flink Dataset). It was used when Pact and Nephele still separate concept.

As part of 0.10.0 release cleanup effort, let's remove the Pact names to avoid confusion.

The PR also contains small cleanups (sorry):
1. Small refactor DataSinkTask and DataSourceTask to follow Java7 generic convention creation new collection. Remove LOG.isDebugEnabled check.
2. Simple cleanup to update MapValue and TypeInformation with Java7 generic convention creation new collection.
3. Combine several exceptions that have same catch operation.

Apologize for the extra changes with PR. But I separated them into different commits for easier review.

Author: hsaputra <hs...@apache.org>

Closes #1218 from hsaputra/remove_pact_name and squashes the following commits:

b3c55b4 [hsaputra] Rename RegularTask to BatchTask per review.
e278fac [hsaputra] Address review comments from chiwanpark (good catch).
9f92f33 [hsaputra] Remove Pact from the file names of teh flink-runtime and flink-clients modules.
dbb2175 [hsaputra] Simple cleanup to update MapValue with Java7 generic for new collection. Remove unused imports in CollectionsDataTypeTest.
df2f553 [hsaputra] Use Java7 style of type resolution for new collection.
6403d44 [hsaputra] Remove the word Pact from the Javadoc for ChainedDriver.
0c562f4 [hsaputra] Small refactor on DataSinkTask and DataSourceTask classes to keep up with modern Java practice.


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

Branch: refs/heads/master
Commit: b08669abf282c52b54c395b85e992edb8ca621d4
Parents: e494c27
Author: hsaputra <hs...@apache.org>
Authored: Tue Oct 6 08:14:06 2015 -0700
Committer: hsaputra <hs...@apache.org>
Committed: Tue Oct 6 08:14:06 2015 -0700

----------------------------------------------------------------------
 .../apache/flink/client/web/JobJSONServlet.java |  163 ++
 .../flink/client/web/PactJobJSONServlet.java    |  163 --
 .../flink/client/web/WebInterfaceServer.java    |    2 +-
 .../api/common/typeinfo/TypeInformation.java    |    2 +-
 .../java/org/apache/flink/types/MapValue.java   |   18 +-
 .../flink/types/CollectionsDataTypeTest.java    |    6 -
 .../plantranslate/JobGraphGenerator.java        |   30 +-
 .../src/test/resources/logback-test.xml         |    2 +-
 .../broadcast/BroadcastVariableManager.java     |   10 +-
 .../BroadcastVariableMaterialization.java       |   12 +-
 .../concurrent/SolutionSetUpdateBarrier.java    |    7 +-
 .../SolutionSetUpdateBarrierBroker.java         |    7 +-
 .../task/AbstractIterativePactTask.java         |  395 -----
 .../iterative/task/AbstractIterativeTask.java   |  395 +++++
 .../iterative/task/IterationHeadPactTask.java   |  440 -----
 .../iterative/task/IterationHeadTask.java       |  441 ++++++
 .../task/IterationIntermediatePactTask.java     |  131 --
 .../task/IterationIntermediateTask.java         |  131 ++
 .../task/IterationSynchronizationSinkTask.java  |    4 +-
 .../iterative/task/IterationTailPactTask.java   |  140 --
 .../iterative/task/IterationTailTask.java       |  140 ++
 .../jobgraph/tasks/AbstractInvokable.java       |    3 +-
 .../AbstractCachedBuildSideJoinDriver.java      |    2 +-
 .../operators/AbstractOuterJoinDriver.java      |    6 +-
 .../operators/AllGroupCombineDriver.java        |    6 +-
 .../runtime/operators/AllGroupReduceDriver.java |    6 +-
 .../runtime/operators/AllReduceDriver.java      |    6 +-
 .../flink/runtime/operators/BatchTask.java      | 1499 ++++++++++++++++++
 .../flink/runtime/operators/CoGroupDriver.java  |    6 +-
 .../runtime/operators/CoGroupRawDriver.java     |    6 +-
 .../CoGroupWithSolutionSetFirstDriver.java      |   12 +-
 .../CoGroupWithSolutionSetSecondDriver.java     |   12 +-
 .../runtime/operators/CollectorMapDriver.java   |    6 +-
 .../flink/runtime/operators/CrossDriver.java    |    6 +-
 .../flink/runtime/operators/DataSinkTask.java   |   41 +-
 .../flink/runtime/operators/DataSourceTask.java |   58 +-
 .../apache/flink/runtime/operators/Driver.java  |   90 ++
 .../flink/runtime/operators/DriverStrategy.java |   12 +-
 .../flink/runtime/operators/FlatMapDriver.java  |    6 +-
 .../operators/GroupReduceCombineDriver.java     |    6 +-
 .../runtime/operators/GroupReduceDriver.java    |    6 +-
 .../flink/runtime/operators/JoinDriver.java     |    6 +-
 .../JoinWithSolutionSetFirstDriver.java         |   12 +-
 .../JoinWithSolutionSetSecondDriver.java        |   12 +-
 .../flink/runtime/operators/MapDriver.java      |    6 +-
 .../runtime/operators/MapPartitionDriver.java   |    6 +-
 .../flink/runtime/operators/NoOpDriver.java     |    6 +-
 .../flink/runtime/operators/PactDriver.java     |   90 --
 .../runtime/operators/PactTaskContext.java      |   70 -
 .../runtime/operators/ReduceCombineDriver.java  |    6 +-
 .../flink/runtime/operators/ReduceDriver.java   |    6 +-
 .../runtime/operators/RegularPactTask.java      | 1499 ------------------
 .../runtime/operators/ResettableDriver.java     |   44 +
 .../runtime/operators/ResettablePactDriver.java |   44 -
 .../flink/runtime/operators/TaskContext.java    |   70 +
 .../operators/UnionWithTempOperator.java        |    6 +-
 .../chaining/ChainedAllReduceDriver.java        |    8 +-
 .../chaining/ChainedCollectorMapDriver.java     |    8 +-
 .../operators/chaining/ChainedDriver.java       |    8 +-
 .../chaining/ChainedFlatMapDriver.java          |    8 +-
 .../operators/chaining/ChainedMapDriver.java    |    8 +-
 .../chaining/GroupCombineChainedDriver.java     |    8 +-
 .../SynchronousChainedCombineDriver.java        |    8 +-
 .../runtime/operators/util/TaskConfig.java      |    8 +-
 .../TaskDeploymentDescriptorTest.java           |    4 +-
 .../ExecutionGraphDeploymentTest.java           |   16 +-
 .../operators/chaining/ChainTaskTest.java       |   10 +-
 .../operators/drivers/TestTaskContext.java      |    4 +-
 .../testutils/BinaryOperatorTestBase.java       |   20 +-
 .../operators/testutils/DriverTestBase.java     |   20 +-
 .../operators/testutils/TaskTestBase.java       |    4 +-
 .../testutils/UnaryOperatorTestBase.java        |   20 +-
 .../src/test/resources/logback-test.xml         |    2 +-
 .../src/test/resources/logback-test.xml         |    2 +-
 .../flink/tez/runtime/RegularProcessor.java     |    6 +-
 .../org/apache/flink/tez/runtime/TezTask.java   |   18 +-
 .../src/test/resources/logback-test.xml         |    2 +-
 flink-tests/src/test/resources/logback-test.xml |    2 +-
 78 files changed, 3236 insertions(+), 3275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-clients/src/main/java/org/apache/flink/client/web/JobJSONServlet.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/web/JobJSONServlet.java b/flink-clients/src/main/java/org/apache/flink/client/web/JobJSONServlet.java
new file mode 100644
index 0000000..77250d3
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/web/JobJSONServlet.java
@@ -0,0 +1,163 @@
+/*
+ * 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.client.web;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.flink.client.program.PackagedProgram;
+
+
+public class JobJSONServlet extends HttpServlet {
+	
+	/** Serial UID for serialization interoperability. */
+	private static final long serialVersionUID = 558077298726449201L;
+	
+	private static final Logger LOG = LoggerFactory.getLogger(JobJSONServlet.class);
+
+	// ------------------------------------------------------------------------
+
+	private static final String JOB_PARAM_NAME = "job";
+
+	private static final String CLASS_PARAM_NAME = "assemblerClass";
+
+	// ------------------------------------------------------------------------
+
+	private final File jobStoreDirectory; // the directory in which the jobs are stored
+
+	public JobJSONServlet(File jobStoreDirectory) {
+		this.jobStoreDirectory = jobStoreDirectory;
+	}
+
+	@Override
+	protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
+		resp.setContentType("application/json");
+
+		String jobName = req.getParameter(JOB_PARAM_NAME);
+		if (jobName == null) {
+			LOG.warn("Received request without job parameter name.");
+			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+			return;
+		}
+
+		// check, if the jar exists
+		File jarFile = new File(jobStoreDirectory, jobName);
+		if (!jarFile.exists()) {
+			LOG.warn("Received request for non-existing jar file.");
+			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+			return;
+		}
+
+		// create the pact plan
+		PackagedProgram pactProgram;
+		try {
+			pactProgram = new PackagedProgram(jarFile, req.getParameter(CLASS_PARAM_NAME), new String[0]);
+		}
+		catch (Throwable t) {
+			LOG.info("Instantiating the PactProgram for '" + jarFile.getName() + "' failed.", t);
+			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+			resp.getWriter().print(t.getMessage());
+			return;
+		}
+		
+		String jsonPlan = null;
+		String programDescription = null;
+		
+		try {
+			jsonPlan = pactProgram.getPreviewPlan();
+		}
+		catch (Throwable t) {
+			LOG.error("Failed to create json dump of pact program.", t);
+		}
+		
+		try {
+			programDescription = pactProgram.getDescription();
+		}
+		catch (Throwable t) {
+			LOG.error("Failed to create description of pact program.", t);
+		}
+			
+		if (jsonPlan == null && programDescription == null) {
+			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+			return;
+		} else {
+			resp.setStatus(HttpServletResponse.SC_OK);
+			PrintWriter wrt = resp.getWriter();
+			wrt.print("{ \"jobname\": \"");
+			wrt.print(jobName);
+			if (jsonPlan != null) {
+				wrt.print("\", \"plan\": ");
+				wrt.println(jsonPlan);
+			}
+			if (programDescription != null) {
+				wrt.print(", \"description\": \"");
+				wrt.print(escapeString(programDescription));
+				wrt.print("\"");
+			}
+			
+			wrt.println("}");
+		}
+	}
+
+	protected String escapeString(String str) {
+		int len = str.length();
+		char[] s = str.toCharArray();
+		StringBuilder sb = new StringBuilder();
+
+		for (int i = 0; i < len; i += 1) {
+			char c = s[i];
+			if ((c == '\\') || (c == '"') || (c == '/')) {
+				sb.append('\\');
+				sb.append(c);
+			}
+			else if (c == '\b') {
+				sb.append("\\b");
+			} else if (c == '\t') {
+				sb.append("\\t");
+			} else if (c == '\n') {
+				sb.append("<br>");
+			} else if (c == '\f') {
+				sb.append("\\f");
+			} else if (c == '\r') {
+				sb.append("\\r");
+			} else if (c == '>') {
+				sb.append("&gt;");
+			} else if (c == '<') {
+				sb.append("&lt;");
+			} else {
+				if (c < ' ') {
+					// Unreadable throw away
+				} else {
+					sb.append(c);
+				}
+			}
+		}
+
+		return sb.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-clients/src/main/java/org/apache/flink/client/web/PactJobJSONServlet.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/web/PactJobJSONServlet.java b/flink-clients/src/main/java/org/apache/flink/client/web/PactJobJSONServlet.java
deleted file mode 100644
index 019fc50..0000000
--- a/flink-clients/src/main/java/org/apache/flink/client/web/PactJobJSONServlet.java
+++ /dev/null
@@ -1,163 +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.client.web;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintWriter;
-
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.client.program.PackagedProgram;
-
-
-public class PactJobJSONServlet extends HttpServlet {
-	
-	/** Serial UID for serialization interoperability. */
-	private static final long serialVersionUID = 558077298726449201L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(PactJobJSONServlet.class);
-
-	// ------------------------------------------------------------------------
-
-	private static final String JOB_PARAM_NAME = "job";
-
-	private static final String CLASS_PARAM_NAME = "assemblerClass";
-
-	// ------------------------------------------------------------------------
-
-	private final File jobStoreDirectory; // the directory in which the jobs are stored
-
-	public PactJobJSONServlet(File jobStoreDirectory) {
-		this.jobStoreDirectory = jobStoreDirectory;
-	}
-
-	@Override
-	protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException {
-		resp.setContentType("application/json");
-
-		String jobName = req.getParameter(JOB_PARAM_NAME);
-		if (jobName == null) {
-			LOG.warn("Received request without job parameter name.");
-			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-			return;
-		}
-
-		// check, if the jar exists
-		File jarFile = new File(jobStoreDirectory, jobName);
-		if (!jarFile.exists()) {
-			LOG.warn("Received request for non-existing jar file.");
-			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-			return;
-		}
-
-		// create the pact plan
-		PackagedProgram pactProgram;
-		try {
-			pactProgram = new PackagedProgram(jarFile, req.getParameter(CLASS_PARAM_NAME), new String[0]);
-		}
-		catch (Throwable t) {
-			LOG.info("Instantiating the PactProgram for '" + jarFile.getName() + "' failed.", t);
-			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-			resp.getWriter().print(t.getMessage());
-			return;
-		}
-		
-		String jsonPlan = null;
-		String programDescription = null;
-		
-		try {
-			jsonPlan = pactProgram.getPreviewPlan();
-		}
-		catch (Throwable t) {
-			LOG.error("Failed to create json dump of pact program.", t);
-		}
-		
-		try {
-			programDescription = pactProgram.getDescription();
-		}
-		catch (Throwable t) {
-			LOG.error("Failed to create description of pact program.", t);
-		}
-			
-		if (jsonPlan == null && programDescription == null) {
-			resp.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-			return;
-		} else {
-			resp.setStatus(HttpServletResponse.SC_OK);
-			PrintWriter wrt = resp.getWriter();
-			wrt.print("{ \"jobname\": \"");
-			wrt.print(jobName);
-			if (jsonPlan != null) {
-				wrt.print("\", \"plan\": ");
-				wrt.println(jsonPlan);
-			}
-			if (programDescription != null) {
-				wrt.print(", \"description\": \"");
-				wrt.print(escapeString(programDescription));
-				wrt.print("\"");
-			}
-			
-			wrt.println("}");
-		}
-	}
-
-	protected String escapeString(String str) {
-		int len = str.length();
-		char[] s = str.toCharArray();
-		StringBuilder sb = new StringBuilder();
-
-		for (int i = 0; i < len; i += 1) {
-			char c = s[i];
-			if ((c == '\\') || (c == '"') || (c == '/')) {
-				sb.append('\\');
-				sb.append(c);
-			}
-			else if (c == '\b') {
-				sb.append("\\b");
-			} else if (c == '\t') {
-				sb.append("\\t");
-			} else if (c == '\n') {
-				sb.append("<br>");
-			} else if (c == '\f') {
-				sb.append("\\f");
-			} else if (c == '\r') {
-				sb.append("\\r");
-			} else if (c == '>') {
-				sb.append("&gt;");
-			} else if (c == '<') {
-				sb.append("&lt;");
-			} else {
-				if (c < ' ') {
-					// Unreadable throw away
-				} else {
-					sb.append(c);
-				}
-			}
-		}
-
-		return sb.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java b/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
index bab3cf7..68a0706 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
@@ -144,7 +144,7 @@ public class WebInterfaceServer {
 		CliFrontend cli = new CliFrontend(configDir);
 		ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.SESSIONS);
 		servletContext.setContextPath("/");
-		servletContext.addServlet(new ServletHolder(new PactJobJSONServlet(uploadDir)), "/pactPlan");
+		servletContext.addServlet(new ServletHolder(new JobJSONServlet(uploadDir)), "/pactPlan");
 		servletContext.addServlet(new ServletHolder(new PlanDisplayServlet(jobManagerWebPort)), "/showPlan");
 		servletContext.addServlet(new ServletHolder(new JobsServlet(uploadDir, tmpDir, "launch.html")), "/jobs");
 		servletContext.addServlet(new ServletHolder(new JobSubmissionServlet(cli, uploadDir, planDumpDir)), "/runJob");

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
index 309c968..07d8544 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
@@ -119,7 +119,7 @@ public abstract class TypeInformation<T> implements Serializable {
 	 */
 	public List<TypeInformation<?>> getGenericParameters() {
 		// Return an empty list as the default implementation
-		return new LinkedList<TypeInformation<?>>();
+		return new LinkedList<>();
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-core/src/main/java/org/apache/flink/types/MapValue.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/types/MapValue.java b/flink-core/src/main/java/org/apache/flink/types/MapValue.java
index ed5b4e1..a6cafd2 100644
--- a/flink-core/src/main/java/org/apache/flink/types/MapValue.java
+++ b/flink-core/src/main/java/org/apache/flink/types/MapValue.java
@@ -31,7 +31,7 @@ import org.apache.flink.util.ReflectionUtil;
 
 /**
  * Generic map base type for PACT programs that implements the Value and Map interfaces.
- * PactMap encapsulates a Java HashMap object.
+ * The {@link MapValue} encapsulates a Java {@link HashMap} object.
  * 
  * @see org.apache.flink.types.Value
  * @see java.util.Map
@@ -56,10 +56,10 @@ public abstract class MapValue<K extends Value, V extends Value> implements Valu
 	 * Initializes the encapsulated map with an empty HashMap.
 	 */
 	public MapValue() {
-		this.keyClass = ReflectionUtil.<K> getTemplateType1(this.getClass());
-		this.valueClass = ReflectionUtil.<V> getTemplateType2(this.getClass());
+		this.keyClass = ReflectionUtil.getTemplateType1(this.getClass());
+		this.valueClass = ReflectionUtil.getTemplateType2(this.getClass());
 
-		this.map = new HashMap<K, V>();
+		this.map = new HashMap<>();
 	}
 
 	/**
@@ -68,10 +68,10 @@ public abstract class MapValue<K extends Value, V extends Value> implements Valu
 	 * @param map Map holding all entries with which the new encapsulated map is filled.
 	 */
 	public MapValue(Map<K, V> map) {
-		this.keyClass = ReflectionUtil.<K> getTemplateType1(this.getClass());
-		this.valueClass = ReflectionUtil.<V> getTemplateType2(this.getClass());
+		this.keyClass = ReflectionUtil.getTemplateType1(this.getClass());
+		this.valueClass = ReflectionUtil.getTemplateType2(this.getClass());
 
-		this.map = new HashMap<K, V>(map);
+		this.map = new HashMap<>(map);
 	}
 
 	@Override
@@ -87,9 +87,7 @@ public abstract class MapValue<K extends Value, V extends Value> implements Valu
 				val.read(in);
 				this.map.put(key, val);
 			}
-		} catch (final InstantiationException e) {
-			throw new RuntimeException(e);
-		} catch (final IllegalAccessException e) {
+		} catch (final InstantiationException | IllegalAccessException e) {
 			throw new RuntimeException(e);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-core/src/test/java/org/apache/flink/types/CollectionsDataTypeTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/CollectionsDataTypeTest.java b/flink-core/src/test/java/org/apache/flink/types/CollectionsDataTypeTest.java
index b631799..5c81e4a 100644
--- a/flink-core/src/test/java/org/apache/flink/types/CollectionsDataTypeTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/CollectionsDataTypeTest.java
@@ -32,12 +32,6 @@ import org.junit.Assert;
 
 import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
 import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.ListValue;
-import org.apache.flink.types.MapValue;
-import org.apache.flink.types.Pair;
-import org.apache.flink.types.StringValue;
 import org.junit.Before;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
index 3567fad..c15e47a 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java
@@ -55,10 +55,10 @@ import org.apache.flink.optimizer.util.Utils;
 import org.apache.flink.runtime.io.network.DataExchangeMode;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
-import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
-import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
+import org.apache.flink.runtime.iterative.task.IterationHeadTask;
+import org.apache.flink.runtime.iterative.task.IterationIntermediateTask;
 import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask;
-import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.iterative.task.IterationTailTask;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
@@ -66,17 +66,17 @@ import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetFirstDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetSecondDriver;
 import org.apache.flink.runtime.operators.DataSinkTask;
 import org.apache.flink.runtime.operators.DataSourceTask;
 import org.apache.flink.runtime.operators.DriverStrategy;
+import org.apache.flink.runtime.operators.JoinDriver;
 import org.apache.flink.runtime.operators.JoinWithSolutionSetFirstDriver;
 import org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver;
-import org.apache.flink.runtime.operators.JoinDriver;
 import org.apache.flink.runtime.operators.NoOpDriver;
-import org.apache.flink.runtime.operators.RegularPactTask;
 import org.apache.flink.runtime.operators.chaining.ChainedDriver;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
@@ -829,7 +829,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 		} else {
 			// create task vertex
 			vertex = new JobVertex(taskName);
-			vertex.setInvokableClass((this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
+			vertex.setInvokableClass((this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediateTask.class : BatchTask.class);
 			
 			config = new TaskConfig(vertex.getConfiguration());
 			config.setDriver(ds.getDriverClass());
@@ -854,7 +854,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 		final DriverStrategy ds = node.getDriverStrategy();
 		final JobVertex vertex = new JobVertex(taskName);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
-		vertex.setInvokableClass( (this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
+		vertex.setInvokableClass( (this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediateTask.class : BatchTask.class);
 		
 		// set user code
 		config.setStubWrapper(node.getProgramOperator().getUserCodeWrapper());
@@ -951,7 +951,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 			}
 			
 			// reset the vertex type to iteration head
-			headVertex.setInvokableClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			toReturn = null;
 		} else {
@@ -959,7 +959,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 			// everything else happens in the post visit, after the input (the initial partial solution)
 			// is connected.
 			headVertex = new JobVertex("PartialSolution ("+iteration.getNodeName()+")");
-			headVertex.setInvokableClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			headConfig.setDriver(NoOpDriver.class);
 			toReturn = headVertex;
@@ -1019,7 +1019,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 			}
 			
 			// reset the vertex type to iteration head
-			headVertex.setInvokableClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			toReturn = null;
 		} else {
@@ -1027,7 +1027,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 			// everything else happens in the post visit, after the input (the initial partial solution)
 			// is connected.
 			headVertex = new JobVertex("IterationHead("+iteration.getNodeName()+")");
-			headVertex.setInvokableClass(IterationHeadPactTask.class);
+			headVertex.setInvokableClass(IterationHeadTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			headConfig.setDriver(NoOpDriver.class);
 			toReturn = headVertex;
@@ -1310,7 +1310,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 		// No following termination criterion
 		if (rootOfStepFunction.getOutgoingChannels().isEmpty()) {
 			
-			rootOfStepFunctionVertex.setInvokableClass(IterationTailPactTask.class);
+			rootOfStepFunctionVertex.setInvokableClass(IterationTailTask.class);
 			
 			tailConfig.setOutputSerializer(bulkNode.getSerializerForIterationChannel());
 		}
@@ -1337,7 +1337,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 				tailConfigOfTerminationCriterion = new TaskConfig(rootOfTerminationCriterionVertex.getConfiguration());
 			}
 			
-			rootOfTerminationCriterionVertex.setInvokableClass(IterationTailPactTask.class);
+			rootOfTerminationCriterionVertex.setInvokableClass(IterationTailTask.class);
 			// Hack
 			tailConfigOfTerminationCriterion.setIsSolutionSetUpdate();
 			tailConfigOfTerminationCriterion.setOutputSerializer(bulkNode.getSerializerForIterationChannel());
@@ -1457,7 +1457,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 				worksetTailConfig.setIsWorksetUpdate();
 				
 				if (hasWorksetTail) {
-					nextWorksetVertex.setInvokableClass(IterationTailPactTask.class);
+					nextWorksetVertex.setInvokableClass(IterationTailTask.class);
 					
 					worksetTailConfig.setOutputSerializer(iterNode.getWorksetSerializer());
 				}
@@ -1481,7 +1481,7 @@ public class JobGraphGenerator implements Visitor<PlanNode> {
 				solutionDeltaConfig.setIsSolutionSetUpdate();
 				
 				if (hasSolutionSetTail) {
-					solutionDeltaVertex.setInvokableClass(IterationTailPactTask.class);
+					solutionDeltaVertex.setInvokableClass(IterationTailTask.class);
 					
 					solutionDeltaConfig.setOutputSerializer(iterNode.getSolutionSetSerializer());
 					

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime-web/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/resources/logback-test.xml b/flink-runtime-web/src/test/resources/logback-test.xml
index 9d4f644..2235251 100644
--- a/flink-runtime-web/src/test/resources/logback-test.xml
+++ b/flink-runtime-web/src/test/resources/logback-test.xml
@@ -31,7 +31,7 @@
          throw error to test failing scenarios. Logging those would overflow the log. -->
          <!---->
     <logger name="org.apache.flink.runtime.operators.DataSinkTask" level="OFF"/>
-    <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
+    <logger name="org.apache.flink.runtime.operators.BatchTask" level="OFF"/>
     <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
index 660a62c..7d0454e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableManager.java
@@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.runtime.io.network.api.reader.MutableReader;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 
 public class BroadcastVariableManager {
 	
@@ -33,7 +33,7 @@ public class BroadcastVariableManager {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public <T> BroadcastVariableMaterialization<T, ?> materializeBroadcastVariable(String name, int superstep, RegularPactTask<?, ?> holder, 
+	public <T> BroadcastVariableMaterialization<T, ?> materializeBroadcastVariable(String name, int superstep, BatchTask<?, ?> holder,
 			MutableReader<?> reader, TypeSerializerFactory<T> serializerFactory) throws IOException
 	{
 		final BroadcastVariableKey key = new BroadcastVariableKey(holder.getEnvironment().getJobVertexId(), name, superstep);
@@ -77,12 +77,12 @@ public class BroadcastVariableManager {
 	}
 	
 	
-	public void releaseReference(String name, int superstep, RegularPactTask<?, ?> referenceHolder) {
+	public void releaseReference(String name, int superstep, BatchTask<?, ?> referenceHolder) {
 		BroadcastVariableKey key = new BroadcastVariableKey(referenceHolder.getEnvironment().getJobVertexId(), name, superstep);
 		releaseReference(key, referenceHolder);
 	}
 	
-	public void releaseReference(BroadcastVariableKey key, RegularPactTask<?, ?> referenceHolder) {
+	public void releaseReference(BroadcastVariableKey key, BatchTask<?, ?> referenceHolder) {
 		BroadcastVariableMaterialization<?, ?> mat = variables.get(key);
 		
 		// release this reference
@@ -93,7 +93,7 @@ public class BroadcastVariableManager {
 	}
 	
 	
-	public void releaseAllReferencesFromTask(RegularPactTask<?, ?> referenceHolder) {
+	public void releaseAllReferencesFromTask(BatchTask<?, ?> referenceHolder) {
 		// go through all registered variables 
 		for (Map.Entry<BroadcastVariableKey, BroadcastVariableMaterialization<?, ?>> entry : variables.entrySet()) {
 			BroadcastVariableMaterialization<?, ?> mat = entry.getValue();

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
index c4dd8a9..86e0111 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/broadcast/BroadcastVariableMaterialization.java
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.runtime.io.network.api.reader.MutableReader;
-import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.operators.BatchTask;
 import org.apache.flink.runtime.operators.util.ReaderIterator;
 import org.apache.flink.runtime.plugable.DeserializationDelegate;
 import org.slf4j.Logger;
@@ -44,7 +44,7 @@ public class BroadcastVariableMaterialization<T, C> {
 	private static final Logger LOG = LoggerFactory.getLogger(BroadcastVariableMaterialization.class);
 	
 	
-	private final Set<RegularPactTask<?, ?>> references = new HashSet<RegularPactTask<?,?>>();
+	private final Set<BatchTask<?, ?>> references = new HashSet<BatchTask<?,?>>();
 	
 	private final Object materializationMonitor = new Object();
 	
@@ -65,7 +65,7 @@ public class BroadcastVariableMaterialization<T, C> {
 
 	// --------------------------------------------------------------------------------------------
 	
-	public void materializeVariable(MutableReader<?> reader, TypeSerializerFactory<?> serializerFactory, RegularPactTask<?, ?> referenceHolder)
+	public void materializeVariable(MutableReader<?> reader, TypeSerializerFactory<?> serializerFactory, BatchTask<?, ?> referenceHolder)
 			throws MaterializationExpiredException, IOException
 	{
 		Preconditions.checkNotNull(reader);
@@ -156,15 +156,15 @@ public class BroadcastVariableMaterialization<T, C> {
 		}
 	}
 	
-	public boolean decrementReference(RegularPactTask<?, ?> referenceHolder) {
+	public boolean decrementReference(BatchTask<?, ?> referenceHolder) {
 		return decrementReferenceInternal(referenceHolder, true);
 	}
 	
-	public boolean decrementReferenceIfHeld(RegularPactTask<?, ?> referenceHolder) {
+	public boolean decrementReferenceIfHeld(BatchTask<?, ?> referenceHolder) {
 		return decrementReferenceInternal(referenceHolder, false);
 	}
 	
-	private boolean decrementReferenceInternal(RegularPactTask<?, ?> referenceHolder, boolean errorIfNoReference) {
+	private boolean decrementReferenceInternal(BatchTask<?, ?> referenceHolder, boolean errorIfNoReference) {
 		synchronized (references) {
 			if (disposed || references.isEmpty()) {
 				if (errorIfNoReference) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
index bc11d3f..ffc74d7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
@@ -18,12 +18,15 @@
 
 package org.apache.flink.runtime.iterative.concurrent;
 
+import org.apache.flink.runtime.iterative.task.IterationHeadTask;
+import org.apache.flink.runtime.iterative.task.IterationTailTask;
+
 import java.util.concurrent.CountDownLatch;
 
 /**
  * Resettable barrier to synchronize the
- * {@link org.apache.flink.runtime.iterative.task.IterationHeadPactTask} and
- * the {@link org.apache.flink.runtime.iterative.task.IterationTailPactTask} in case of
+ * {@link IterationHeadTask} and
+ * the {@link IterationTailTask} in case of
  * iterations that contain a separate solution set tail.
  */
 public class SolutionSetUpdateBarrier {

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
index abbecde..352a262 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
@@ -18,10 +18,13 @@
 
 package org.apache.flink.runtime.iterative.concurrent;
 
+import org.apache.flink.runtime.iterative.task.IterationHeadTask;
+import org.apache.flink.runtime.iterative.task.IterationTailTask;
+
 /**
  * Broker to hand over {@link SolutionSetUpdateBarrier} from 
- * {@link org.apache.flink.runtime.iterative.task.IterationHeadPactTask} to
- * {@link org.apache.flink.runtime.iterative.task.IterationTailPactTask}.
+ * {@link IterationHeadTask} to
+ * {@link IterationTailTask}.
  */
 public class SolutionSetUpdateBarrierBroker extends Broker<SolutionSetUpdateBarrier> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java
deleted file mode 100644
index efe74f9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java
+++ /dev/null
@@ -1,395 +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.runtime.iterative.task;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.core.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.api.common.operators.util.JoinHashMap;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.io.network.api.reader.MutableReader;
-import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel;
-import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannelBroker;
-import org.apache.flink.runtime.iterative.concurrent.Broker;
-import org.apache.flink.runtime.iterative.concurrent.IterationAggregatorBroker;
-import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
-import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
-import org.apache.flink.runtime.iterative.io.SolutionSetObjectsUpdateOutputCollector;
-import org.apache.flink.runtime.iterative.io.SolutionSetUpdateOutputCollector;
-import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector;
-import org.apache.flink.runtime.operators.PactDriver;
-import org.apache.flink.runtime.operators.RegularPactTask;
-import org.apache.flink.runtime.operators.ResettablePactDriver;
-import org.apache.flink.runtime.operators.hash.CompactingHashTable;
-import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.MutableObjectIterator;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Map;
-import java.util.concurrent.Future;
-
-/**
- * The abstract base class for all tasks able to participate in an iteration.
- */
-public abstract class AbstractIterativePactTask<S extends Function, OT> extends RegularPactTask<S, OT>
-		implements Terminable
-{
-	private static final Logger log = LoggerFactory.getLogger(AbstractIterativePactTask.class);
-	
-	protected LongSumAggregator worksetAggregator;
-
-	protected BlockingBackChannel worksetBackChannel;
-
-	protected boolean isWorksetIteration;
-
-	protected boolean isWorksetUpdate;
-
-	protected boolean isSolutionSetUpdate;
-	
-
-	private RuntimeAggregatorRegistry iterationAggregators;
-
-	private String brokerKey;
-
-	private int superstepNum = 1;
-	
-	private volatile boolean terminationRequested;
-
-	// --------------------------------------------------------------------------------------------
-	// Main life cycle methods that implement the iterative behavior
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	protected void initialize() throws Exception {
-		super.initialize();
-
-		// check if the driver is resettable
-		if (this.driver instanceof ResettablePactDriver) {
-			final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
-			// make sure that the according inputs are not reseted
-			for (int i = 0; i < resDriver.getNumberOfInputs(); i++) {
-				if (resDriver.isInputResettable(i)) {
-					excludeFromReset(i);
-				}
-			}
-		}
-		
-		TaskConfig config = getLastTasksConfig();
-		isWorksetIteration = config.getIsWorksetIteration();
-		isWorksetUpdate = config.getIsWorksetUpdate();
-		isSolutionSetUpdate = config.getIsSolutionSetUpdate();
-
-		if (isWorksetUpdate) {
-			worksetBackChannel = BlockingBackChannelBroker.instance().getAndRemove(brokerKey());
-
-			if (isWorksetIteration) {
-				worksetAggregator = getIterationAggregators().getAggregator(WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME);
-
-				if (worksetAggregator == null) {
-					throw new RuntimeException("Missing workset elements count aggregator.");
-				}
-			}
-		}
-	}
-
-	@Override
-	public void run() throws Exception {
-		if (inFirstIteration()) {
-			if (this.driver instanceof ResettablePactDriver) {
-				// initialize the repeatable driver
-				((ResettablePactDriver<?, ?>) this.driver).initialize();
-			}
-		} else {
-			reinstantiateDriver();
-			resetAllInputs();
-			
-			// re-read the iterative broadcast variables
-			for (int i : this.iterativeBroadcastInputs) {
-				final String name = getTaskConfig().getBroadcastInputName(i);
-				readAndSetBroadcastInput(i, name, this.runtimeUdfContext, superstepNum);
-			}
-		}
-
-		// call the parent to execute the superstep
-		super.run();
-		
-		// release the iterative broadcast variables
-		for (int i : this.iterativeBroadcastInputs) {
-			final String name = getTaskConfig().getBroadcastInputName(i);
-			releaseBroadcastVariables(name, superstepNum, this.runtimeUdfContext);
-		}
-	}
-
-	@Override
-	protected void closeLocalStrategiesAndCaches() {
-		try {
-			super.closeLocalStrategiesAndCaches();
-		}
-		finally {
-			if (this.driver instanceof ResettablePactDriver) {
-				final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
-				try {
-					resDriver.teardown();
-				} catch (Throwable t) {
-					log.error("Error while shutting down an iterative operator.", t);
-				}
-			}
-		}
-	}
-
-	@Override
-	public DistributedRuntimeUDFContext createRuntimeContext(String taskName) {
-		Environment env = getEnvironment();
-		return new IterativeRuntimeUdfContext(taskName, env.getNumberOfSubtasks(),
-				env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(),
-				env.getDistributedCacheEntries(), this.accumulatorMap);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// Utility Methods for Iteration Handling
-	// --------------------------------------------------------------------------------------------
-
-	protected boolean inFirstIteration() {
-		return this.superstepNum == 1;
-	}
-
-	protected int currentIteration() {
-		return this.superstepNum;
-	}
-
-	protected void incrementIterationCounter() {
-		this.superstepNum++;
-	}
-
-	public String brokerKey() {
-		if (brokerKey == null) {
-			int iterationId = config.getIterationId();
-			brokerKey = getEnvironment().getJobID().toString() + '#' + iterationId + '#' +
-					getEnvironment().getIndexInSubtaskGroup();
-		}
-		return brokerKey;
-	}
-
-	private void reinstantiateDriver() throws Exception {
-		if (this.driver instanceof ResettablePactDriver) {
-			final ResettablePactDriver<?, ?> resDriver = (ResettablePactDriver<?, ?>) this.driver;
-			resDriver.reset();
-		} else {
-			Class<? extends PactDriver<S, OT>> driverClass = this.config.getDriver();
-			this.driver = InstantiationUtil.instantiate(driverClass, PactDriver.class);
-
-			try {
-				this.driver.setup(this);
-			}
-			catch (Throwable t) {
-				throw new Exception("The pact driver setup for '" + this.getEnvironment().getTaskName() +
-						"' , caused an error: " + t.getMessage(), t);
-			}
-		}
-	}
-
-	public RuntimeAggregatorRegistry getIterationAggregators() {
-		if (this.iterationAggregators == null) {
-			this.iterationAggregators = IterationAggregatorBroker.instance().get(brokerKey());
-		}
-		return this.iterationAggregators;
-	}
-
-	protected void verifyEndOfSuperstepState() throws IOException {
-		// sanity check that there is at least one iterative input reader
-		if (this.iterativeInputs.length == 0 && this.iterativeBroadcastInputs.length == 0) {
-			throw new IllegalStateException("Error: Iterative task without a single iterative input.");
-		}
-
-		for (int inputNum : this.iterativeInputs) {
-			MutableReader<?> reader = this.inputReaders[inputNum];
-
-			if (!reader.isFinished()) {
-				if (reader.hasReachedEndOfSuperstep()) {
-					reader.startNextSuperstep();
-				}
-				else {
-					// need to read and drop all non-consumed data until we reach the end-of-superstep
-					@SuppressWarnings("unchecked")
-					MutableObjectIterator<Object> inIter = (MutableObjectIterator<Object>) this.inputIterators[inputNum];
-					Object o = this.inputSerializers[inputNum].getSerializer().createInstance();
-					while ((o = inIter.next(o)) != null);
-					
-					if (!reader.isFinished()) {
-						// also reset the end-of-superstep state
-						reader.startNextSuperstep();
-					}
-				}
-			}
-		}
-		
-		for (int inputNum : this.iterativeBroadcastInputs) {
-			MutableReader<?> reader = this.broadcastInputReaders[inputNum];
-
-			if (!reader.isFinished()) {
-				
-				// sanity check that the BC input is at the end of the superstep
-				if (!reader.hasReachedEndOfSuperstep()) {
-					throw new IllegalStateException("An iterative broadcast input has not been fully consumed.");
-				}
-				
-				reader.startNextSuperstep();
-			}
-		}
-	}
-
-	@Override
-	public boolean terminationRequested() {
-		return this.terminationRequested;
-	}
-
-	@Override
-	public void requestTermination() {
-		this.terminationRequested = true;
-	}
-
-	@Override
-	public void cancel() throws Exception {
-		requestTermination();
-		super.cancel();
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// Iteration State Update Handling
-	// -----------------------------------------------------------------------------------------------------------------
-
-	/**
-	 * Creates a new {@link WorksetUpdateOutputCollector}.
-	 * <p>
-	 * This collector is used by {@link IterationIntermediatePactTask} or {@link IterationTailPactTask} to update the
-	 * workset.
-	 * <p>
-	 * If a non-null delegate is given, the new {@link Collector} will write to the solution set and also call
-	 * collect(T) of the delegate.
-	 *
-	 * @param delegate null -OR- the delegate on which to call collect() by the newly created collector
-	 * @return a new {@link WorksetUpdateOutputCollector}
-	 */
-	protected Collector<OT> createWorksetUpdateOutputCollector(Collector<OT> delegate) {
-		DataOutputView outputView = worksetBackChannel.getWriteEnd();
-		TypeSerializer<OT> serializer = getOutputSerializer();
-		return new WorksetUpdateOutputCollector<OT>(outputView, serializer, delegate);
-	}
-
-	protected Collector<OT> createWorksetUpdateOutputCollector() {
-		return createWorksetUpdateOutputCollector(null);
-	}
-
-	/**
-	 * Creates a new solution set update output collector.
-	 * <p>
-	 * This collector is used by {@link IterationIntermediatePactTask} or {@link IterationTailPactTask} to update the
-	 * solution set of workset iterations. Depending on the task configuration, either a fast (non-probing)
-	 * {@link org.apache.flink.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or normal (re-probing)
-	 * {@link SolutionSetUpdateOutputCollector} is created.
-	 * <p>
-	 * If a non-null delegate is given, the new {@link Collector} will write back to the solution set and also call
-	 * collect(T) of the delegate.
-	 *
-	 * @param delegate null -OR- a delegate collector to be called by the newly created collector
-	 * @return a new {@link org.apache.flink.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or
-	 * {@link SolutionSetUpdateOutputCollector}
-	 */
-	protected Collector<OT> createSolutionSetUpdateOutputCollector(Collector<OT> delegate) {
-		Broker<Object> solutionSetBroker = SolutionSetBroker.instance();
-		
-		Object ss = solutionSetBroker.get(brokerKey());
-		if (ss instanceof CompactingHashTable) {
-			@SuppressWarnings("unchecked")
-			CompactingHashTable<OT> solutionSet = (CompactingHashTable<OT>) ss;
-			return new SolutionSetUpdateOutputCollector<OT>(solutionSet, delegate);
-		}
-		else if (ss instanceof JoinHashMap) {
-			@SuppressWarnings("unchecked")
-			JoinHashMap<OT> map = (JoinHashMap<OT>) ss;
-			return new SolutionSetObjectsUpdateOutputCollector<OT>(map, delegate);
-		} else {
-			throw new RuntimeException("Unrecognized solution set handle: " + ss);
-		}
-	}
-
-	/**
-	 * @return output serializer of this task
-	 */
-	private TypeSerializer<OT> getOutputSerializer() {
-		TypeSerializerFactory<OT> serializerFactory;
-
-		if ((serializerFactory = getLastTasksConfig().getOutputSerializer(getUserCodeClassLoader())) ==
-				null) {
-			throw new RuntimeException("Missing output serializer for workset update.");
-		}
-
-		return serializerFactory.getSerializer();
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-
-	private class IterativeRuntimeUdfContext extends DistributedRuntimeUDFContext implements IterationRuntimeContext {
-
-		public IterativeRuntimeUdfContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader,
-										ExecutionConfig executionConfig, Map<String, Future<Path>> cpTasks,
-										Map<String, Accumulator<?,?>> accumulatorMap) {
-			super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig, cpTasks, accumulatorMap);
-		}
-
-		@Override
-		public int getSuperstepNumber() {
-			return AbstractIterativePactTask.this.superstepNum;
-		}
-
-		@Override
-		public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-			return getIterationAggregators().<T>getAggregator(name);
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public <T extends Value> T getPreviousIterationAggregate(String name) {
-			return (T) getIterationAggregators().getPreviousGlobalAggregate(name);
-		}
-
-		@Override
-		public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> newAccumulator) {
-			// only add accumulator on first iteration
-			if (inFirstIteration()) {
-				super.addAccumulator(name, newAccumulator);
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/b08669ab/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java
new file mode 100644
index 0000000..215111b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java
@@ -0,0 +1,395 @@
+/*
+ * 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.runtime.iterative.task;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.operators.BatchTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.aggregators.LongSumAggregator;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.common.operators.util.JoinHashMap;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.io.network.api.reader.MutableReader;
+import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannel;
+import org.apache.flink.runtime.iterative.concurrent.BlockingBackChannelBroker;
+import org.apache.flink.runtime.iterative.concurrent.Broker;
+import org.apache.flink.runtime.iterative.concurrent.IterationAggregatorBroker;
+import org.apache.flink.runtime.iterative.concurrent.SolutionSetBroker;
+import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
+import org.apache.flink.runtime.iterative.io.SolutionSetObjectsUpdateOutputCollector;
+import org.apache.flink.runtime.iterative.io.SolutionSetUpdateOutputCollector;
+import org.apache.flink.runtime.iterative.io.WorksetUpdateOutputCollector;
+import org.apache.flink.runtime.operators.Driver;
+import org.apache.flink.runtime.operators.ResettableDriver;
+import org.apache.flink.runtime.operators.hash.CompactingHashTable;
+import org.apache.flink.runtime.operators.util.DistributedRuntimeUDFContext;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+/**
+ * The abstract base class for all tasks able to participate in an iteration.
+ */
+public abstract class AbstractIterativeTask<S extends Function, OT> extends BatchTask<S, OT>
+		implements Terminable
+{
+	private static final Logger log = LoggerFactory.getLogger(AbstractIterativeTask.class);
+	
+	protected LongSumAggregator worksetAggregator;
+
+	protected BlockingBackChannel worksetBackChannel;
+
+	protected boolean isWorksetIteration;
+
+	protected boolean isWorksetUpdate;
+
+	protected boolean isSolutionSetUpdate;
+	
+
+	private RuntimeAggregatorRegistry iterationAggregators;
+
+	private String brokerKey;
+
+	private int superstepNum = 1;
+	
+	private volatile boolean terminationRequested;
+
+	// --------------------------------------------------------------------------------------------
+	// Main life cycle methods that implement the iterative behavior
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws Exception {
+		super.initialize();
+
+		// check if the driver is resettable
+		if (this.driver instanceof ResettableDriver) {
+			final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
+			// make sure that the according inputs are not reseted
+			for (int i = 0; i < resDriver.getNumberOfInputs(); i++) {
+				if (resDriver.isInputResettable(i)) {
+					excludeFromReset(i);
+				}
+			}
+		}
+		
+		TaskConfig config = getLastTasksConfig();
+		isWorksetIteration = config.getIsWorksetIteration();
+		isWorksetUpdate = config.getIsWorksetUpdate();
+		isSolutionSetUpdate = config.getIsSolutionSetUpdate();
+
+		if (isWorksetUpdate) {
+			worksetBackChannel = BlockingBackChannelBroker.instance().getAndRemove(brokerKey());
+
+			if (isWorksetIteration) {
+				worksetAggregator = getIterationAggregators().getAggregator(WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME);
+
+				if (worksetAggregator == null) {
+					throw new RuntimeException("Missing workset elements count aggregator.");
+				}
+			}
+		}
+	}
+
+	@Override
+	public void run() throws Exception {
+		if (inFirstIteration()) {
+			if (this.driver instanceof ResettableDriver) {
+				// initialize the repeatable driver
+				((ResettableDriver<?, ?>) this.driver).initialize();
+			}
+		} else {
+			reinstantiateDriver();
+			resetAllInputs();
+			
+			// re-read the iterative broadcast variables
+			for (int i : this.iterativeBroadcastInputs) {
+				final String name = getTaskConfig().getBroadcastInputName(i);
+				readAndSetBroadcastInput(i, name, this.runtimeUdfContext, superstepNum);
+			}
+		}
+
+		// call the parent to execute the superstep
+		super.run();
+		
+		// release the iterative broadcast variables
+		for (int i : this.iterativeBroadcastInputs) {
+			final String name = getTaskConfig().getBroadcastInputName(i);
+			releaseBroadcastVariables(name, superstepNum, this.runtimeUdfContext);
+		}
+	}
+
+	@Override
+	protected void closeLocalStrategiesAndCaches() {
+		try {
+			super.closeLocalStrategiesAndCaches();
+		}
+		finally {
+			if (this.driver instanceof ResettableDriver) {
+				final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
+				try {
+					resDriver.teardown();
+				} catch (Throwable t) {
+					log.error("Error while shutting down an iterative operator.", t);
+				}
+			}
+		}
+	}
+
+	@Override
+	public DistributedRuntimeUDFContext createRuntimeContext(String taskName) {
+		Environment env = getEnvironment();
+		return new IterativeRuntimeUdfContext(taskName, env.getNumberOfSubtasks(),
+				env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(),
+				env.getDistributedCacheEntries(), this.accumulatorMap);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Utility Methods for Iteration Handling
+	// --------------------------------------------------------------------------------------------
+
+	protected boolean inFirstIteration() {
+		return this.superstepNum == 1;
+	}
+
+	protected int currentIteration() {
+		return this.superstepNum;
+	}
+
+	protected void incrementIterationCounter() {
+		this.superstepNum++;
+	}
+
+	public String brokerKey() {
+		if (brokerKey == null) {
+			int iterationId = config.getIterationId();
+			brokerKey = getEnvironment().getJobID().toString() + '#' + iterationId + '#' +
+					getEnvironment().getIndexInSubtaskGroup();
+		}
+		return brokerKey;
+	}
+
+	private void reinstantiateDriver() throws Exception {
+		if (this.driver instanceof ResettableDriver) {
+			final ResettableDriver<?, ?> resDriver = (ResettableDriver<?, ?>) this.driver;
+			resDriver.reset();
+		} else {
+			Class<? extends Driver<S, OT>> driverClass = this.config.getDriver();
+			this.driver = InstantiationUtil.instantiate(driverClass, Driver.class);
+
+			try {
+				this.driver.setup(this);
+			}
+			catch (Throwable t) {
+				throw new Exception("The pact driver setup for '" + this.getEnvironment().getTaskName() +
+						"' , caused an error: " + t.getMessage(), t);
+			}
+		}
+	}
+
+	public RuntimeAggregatorRegistry getIterationAggregators() {
+		if (this.iterationAggregators == null) {
+			this.iterationAggregators = IterationAggregatorBroker.instance().get(brokerKey());
+		}
+		return this.iterationAggregators;
+	}
+
+	protected void verifyEndOfSuperstepState() throws IOException {
+		// sanity check that there is at least one iterative input reader
+		if (this.iterativeInputs.length == 0 && this.iterativeBroadcastInputs.length == 0) {
+			throw new IllegalStateException("Error: Iterative task without a single iterative input.");
+		}
+
+		for (int inputNum : this.iterativeInputs) {
+			MutableReader<?> reader = this.inputReaders[inputNum];
+
+			if (!reader.isFinished()) {
+				if (reader.hasReachedEndOfSuperstep()) {
+					reader.startNextSuperstep();
+				}
+				else {
+					// need to read and drop all non-consumed data until we reach the end-of-superstep
+					@SuppressWarnings("unchecked")
+					MutableObjectIterator<Object> inIter = (MutableObjectIterator<Object>) this.inputIterators[inputNum];
+					Object o = this.inputSerializers[inputNum].getSerializer().createInstance();
+					while ((o = inIter.next(o)) != null);
+					
+					if (!reader.isFinished()) {
+						// also reset the end-of-superstep state
+						reader.startNextSuperstep();
+					}
+				}
+			}
+		}
+		
+		for (int inputNum : this.iterativeBroadcastInputs) {
+			MutableReader<?> reader = this.broadcastInputReaders[inputNum];
+
+			if (!reader.isFinished()) {
+				
+				// sanity check that the BC input is at the end of the superstep
+				if (!reader.hasReachedEndOfSuperstep()) {
+					throw new IllegalStateException("An iterative broadcast input has not been fully consumed.");
+				}
+				
+				reader.startNextSuperstep();
+			}
+		}
+	}
+
+	@Override
+	public boolean terminationRequested() {
+		return this.terminationRequested;
+	}
+
+	@Override
+	public void requestTermination() {
+		this.terminationRequested = true;
+	}
+
+	@Override
+	public void cancel() throws Exception {
+		requestTermination();
+		super.cancel();
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+	// Iteration State Update Handling
+	// -----------------------------------------------------------------------------------------------------------------
+
+	/**
+	 * Creates a new {@link WorksetUpdateOutputCollector}.
+	 * <p>
+	 * This collector is used by {@link IterationIntermediateTask} or {@link IterationTailTask} to update the
+	 * workset.
+	 * <p>
+	 * If a non-null delegate is given, the new {@link Collector} will write to the solution set and also call
+	 * collect(T) of the delegate.
+	 *
+	 * @param delegate null -OR- the delegate on which to call collect() by the newly created collector
+	 * @return a new {@link WorksetUpdateOutputCollector}
+	 */
+	protected Collector<OT> createWorksetUpdateOutputCollector(Collector<OT> delegate) {
+		DataOutputView outputView = worksetBackChannel.getWriteEnd();
+		TypeSerializer<OT> serializer = getOutputSerializer();
+		return new WorksetUpdateOutputCollector<OT>(outputView, serializer, delegate);
+	}
+
+	protected Collector<OT> createWorksetUpdateOutputCollector() {
+		return createWorksetUpdateOutputCollector(null);
+	}
+
+	/**
+	 * Creates a new solution set update output collector.
+	 * <p>
+	 * This collector is used by {@link IterationIntermediateTask} or {@link IterationTailTask} to update the
+	 * solution set of workset iterations. Depending on the task configuration, either a fast (non-probing)
+	 * {@link org.apache.flink.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or normal (re-probing)
+	 * {@link SolutionSetUpdateOutputCollector} is created.
+	 * <p>
+	 * If a non-null delegate is given, the new {@link Collector} will write back to the solution set and also call
+	 * collect(T) of the delegate.
+	 *
+	 * @param delegate null -OR- a delegate collector to be called by the newly created collector
+	 * @return a new {@link org.apache.flink.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or
+	 * {@link SolutionSetUpdateOutputCollector}
+	 */
+	protected Collector<OT> createSolutionSetUpdateOutputCollector(Collector<OT> delegate) {
+		Broker<Object> solutionSetBroker = SolutionSetBroker.instance();
+		
+		Object ss = solutionSetBroker.get(brokerKey());
+		if (ss instanceof CompactingHashTable) {
+			@SuppressWarnings("unchecked")
+			CompactingHashTable<OT> solutionSet = (CompactingHashTable<OT>) ss;
+			return new SolutionSetUpdateOutputCollector<OT>(solutionSet, delegate);
+		}
+		else if (ss instanceof JoinHashMap) {
+			@SuppressWarnings("unchecked")
+			JoinHashMap<OT> map = (JoinHashMap<OT>) ss;
+			return new SolutionSetObjectsUpdateOutputCollector<OT>(map, delegate);
+		} else {
+			throw new RuntimeException("Unrecognized solution set handle: " + ss);
+		}
+	}
+
+	/**
+	 * @return output serializer of this task
+	 */
+	private TypeSerializer<OT> getOutputSerializer() {
+		TypeSerializerFactory<OT> serializerFactory;
+
+		if ((serializerFactory = getLastTasksConfig().getOutputSerializer(getUserCodeClassLoader())) ==
+				null) {
+			throw new RuntimeException("Missing output serializer for workset update.");
+		}
+
+		return serializerFactory.getSerializer();
+	}
+
+	// -----------------------------------------------------------------------------------------------------------------
+
+	private class IterativeRuntimeUdfContext extends DistributedRuntimeUDFContext implements IterationRuntimeContext {
+
+		public IterativeRuntimeUdfContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader,
+										ExecutionConfig executionConfig, Map<String, Future<Path>> cpTasks,
+										Map<String, Accumulator<?,?>> accumulatorMap) {
+			super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig, cpTasks, accumulatorMap);
+		}
+
+		@Override
+		public int getSuperstepNumber() {
+			return AbstractIterativeTask.this.superstepNum;
+		}
+
+		@Override
+		public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+			return getIterationAggregators().<T>getAggregator(name);
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public <T extends Value> T getPreviousIterationAggregate(String name) {
+			return (T) getIterationAggregators().getPreviousGlobalAggregate(name);
+		}
+
+		@Override
+		public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> newAccumulator) {
+			// only add accumulator on first iteration
+			if (inFirstIteration()) {
+				super.addAccumulator(name, newAccumulator);
+			}
+		}
+	}
+
+}