Refactored Hyracks into runtime/dataflow-common/dataflow-std/server

git-svn-id: https://hyracks.googlecode.com/svn/trunk/hyracks@38 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-dataflow-std/.classpath b/hyracks-dataflow-std/.classpath
new file mode 100644
index 0000000..1f3c1ff
--- /dev/null
+++ b/hyracks-dataflow-std/.classpath
@@ -0,0 +1,7 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<classpath>
+	<classpathentry kind="src" output="target/classes" path="src/main/java"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
+	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
+	<classpathentry kind="output" path="target/classes"/>
+</classpath>
diff --git a/hyracks-dataflow-std/.project b/hyracks-dataflow-std/.project
new file mode 100644
index 0000000..1101a65
--- /dev/null
+++ b/hyracks-dataflow-std/.project
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<projectDescription>
+	<name>hyracks-dataflow-std</name>
+	<comment></comment>
+	<projects>
+	</projects>
+	<buildSpec>
+		<buildCommand>
+			<name>org.eclipse.jdt.core.javabuilder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+		<buildCommand>
+			<name>org.maven.ide.eclipse.maven2Builder</name>
+			<arguments>
+			</arguments>
+		</buildCommand>
+	</buildSpec>
+	<natures>
+		<nature>org.maven.ide.eclipse.maven2Nature</nature>
+		<nature>org.eclipse.jdt.core.javanature</nature>
+	</natures>
+</projectDescription>
diff --git a/hyracks-dataflow-std/.settings/org.eclipse.jdt.core.prefs b/hyracks-dataflow-std/.settings/org.eclipse.jdt.core.prefs
new file mode 100644
index 0000000..1ca1bd4
--- /dev/null
+++ b/hyracks-dataflow-std/.settings/org.eclipse.jdt.core.prefs
@@ -0,0 +1,6 @@
+#Thu Jul 29 14:32:56 PDT 2010
+eclipse.preferences.version=1
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks-dataflow-std/.settings/org.maven.ide.eclipse.prefs b/hyracks-dataflow-std/.settings/org.maven.ide.eclipse.prefs
new file mode 100644
index 0000000..e96df89
--- /dev/null
+++ b/hyracks-dataflow-std/.settings/org.maven.ide.eclipse.prefs
@@ -0,0 +1,9 @@
+#Thu Jul 29 14:32:56 PDT 2010
+activeProfiles=
+eclipse.preferences.version=1
+fullBuildGoals=process-test-resources
+includeModules=false
+resolveWorkspaceProjects=true
+resourceFilterGoals=process-resources resources\:testResources
+skipCompilerPlugin=true
+version=1
diff --git a/hyracks-dataflow-std/pom.xml b/hyracks-dataflow-std/pom.xml
new file mode 100644
index 0000000..9d720b3
--- /dev/null
+++ b/hyracks-dataflow-std/pom.xml
@@ -0,0 +1,48 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>edu.uci.ics.hyracks</groupId>
+  <artifactId>hyracks-dataflow-std</artifactId>
+  <version>0.1.0</version>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-api</artifactId>
+  		<version>0.1.0</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-dataflow-common</artifactId>
+  		<version>0.1.0</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>org.apache.hadoop</groupId>
+  		<artifactId>hadoop-core</artifactId>
+  		<version>0.20.2</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.dcache</groupId>
+  		<artifactId>dcache-client</artifactId>
+  		<version>0.0.1</version>
+  		<scope>compile</scope>
+  	</dependency>
+  </dependencies>
+</project>
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ExternalSortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ExternalSortOperatorDescriptor.java
new file mode 100644
index 0000000..d4787c2
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ExternalSortOperatorDescriptor.java
@@ -0,0 +1,611 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
+
+public class ExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String IN_FRAMES = "inFrames";
+    private static final String TPOINTERS = "tPointers";
+    private static final String RUNS = "runs";
+
+    private static final long serialVersionUID = 1L;
+    private final int[] sortFields;
+    private IBinaryComparatorFactory[] comparatorFactories;
+    private final int framesLimit;
+
+    public ExternalSortOperatorDescriptor(JobSpecification spec, int framesLimit, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.framesLimit = framesLimit;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        if (framesLimit <= 1) {
+            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
+        }
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity();
+        MergeActivity ma = new MergeActivity();
+
+        builder.addTask(sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addTask(ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    private class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return ExternalSortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private final FrameTupleAccessor fta1 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                private final FrameTupleAccessor fta2 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                private List<ByteBuffer> inFrames;
+                private ByteBuffer outFrame;
+                private LinkedList<File> runs;
+                private int activeInFrame;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    inFrames = new ArrayList<ByteBuffer>();
+                    outFrame = ctx.getResourceManager().allocateFrame();
+                    runs = new LinkedList<File>();
+                    activeInFrame = 0;
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    if (activeInFrame + 1 >= framesLimit) { // + 1 outFrame.
+                        try {
+                            createRunFromInFrames(inFrames.size());
+                        } catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                    ByteBuffer copy;
+                    buffer.position(0);
+                    buffer.limit(buffer.capacity());
+                    if (runs.size() <= 0) {
+                        copy = ctx.getResourceManager().allocateFrame();
+                        copy.put(buffer);
+                        inFrames.add(copy);
+                    } else {
+                        copy = inFrames.get(activeInFrame);
+                        copy.put(buffer);
+                    }
+                    ++activeInFrame;
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(IN_FRAMES, inFrames);
+                    env.set(RUNS, runs);
+                    if (activeInFrame > 0) {
+                        if (runs.size() <= 0) {
+                            long[] tPointers = getSortedTPointers(activeInFrame);
+                            env.set(TPOINTERS, tPointers);
+                        } else {
+                            createRunFromInFrames(activeInFrame);
+                        }
+                    }
+                }
+
+                private void createRunFromInFrames(int nBuffers) throws HyracksDataException {
+                    File runFile;
+                    try {
+                        runFile = ctx.getResourceManager().createFile(
+                                ExternalSortOperatorDescriptor.class.getSimpleName(), ".run");
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                    RunFileWriter writer = new RunFileWriter(runFile);
+                    writer.open();
+                    try {
+                        flushFrames(ctx, inFrames, outFrame, getSortedTPointers(nBuffers), writer);
+                    } finally {
+                        writer.close();
+                    }
+                    runs.add(runFile);
+                    activeInFrame = 0;
+                }
+
+                private long[] getSortedTPointers(int nBuffers) {
+                    FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                    int totalTCount = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(inFrames.get(i));
+                        totalTCount += accessor.getTupleCount();
+                    }
+                    long[] tPointers = new long[totalTCount];
+                    int ptr = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(inFrames.get(i));
+                        int tCount = accessor.getTupleCount();
+                        for (int j = 0; j < tCount; ++j) {
+                            tPointers[ptr++] = (((long) i) << 32) + j;
+                        }
+                    }
+                    if (tPointers.length > 0) {
+                        sort(tPointers, 0, tPointers.length);
+                    }
+                    return tPointers;
+                }
+
+                private void sort(long[] tPointers, int offset, int length) {
+                    int m = offset + (length >> 1);
+                    long v = tPointers[m];
+
+                    int a = offset;
+                    int b = a;
+                    int c = offset + length - 1;
+                    int d = c;
+                    while (true) {
+                        while (b <= c && compare(tPointers[b], v) <= 0) {
+                            if (compare(tPointers[b], v) == 0) {
+                                swap(tPointers, a++, b);
+                            }
+                            ++b;
+                        }
+                        while (c >= b && compare(tPointers[c], v) >= 0) {
+                            if (compare(tPointers[c], v) == 0) {
+                                swap(tPointers, c, d--);
+                            }
+                            --c;
+                        }
+                        if (b > c)
+                            break;
+                        swap(tPointers, b++, c--);
+                    }
+
+                    int s;
+                    int n = offset + length;
+                    s = Math.min(a - offset, b - a);
+                    vecswap(tPointers, offset, b - s, s);
+                    s = Math.min(d - c, n - d - 1);
+                    vecswap(tPointers, b, n - s, s);
+
+                    if ((s = b - a) > 1) {
+                        sort(tPointers, offset, s);
+                    }
+                    if ((s = d - c) > 1) {
+                        sort(tPointers, n - s, s);
+                    }
+                }
+
+                private void swap(long x[], int a, int b) {
+                    long t = x[a];
+                    x[a] = x[b];
+                    x[b] = t;
+                }
+
+                private void vecswap(long x[], int a, int b, int n) {
+                    for (int i = 0; i < n; i++, a++, b++) {
+                        swap(x, a, b);
+                    }
+                }
+
+                private int compare(long tp1, long tp2) {
+                    int i1 = (int) ((tp1 >> 32) & 0xffffffff);
+                    int j1 = (int) (tp1 & 0xffffffff);
+                    int i2 = (int) ((tp2 >> 32) & 0xffffffff);
+                    int j2 = (int) (tp2 & 0xffffffff);
+                    ByteBuffer buf1 = inFrames.get(i1);
+                    ByteBuffer buf2 = inFrames.get(i2);
+                    byte[] b1 = buf1.array();
+                    byte[] b2 = buf2.array();
+                    fta1.reset(buf1);
+                    fta2.reset(buf2);
+                    for (int f = 0; f < sortFields.length; ++f) {
+                        int fIdx = sortFields[f];
+                        int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                                + fta1.getFieldStartOffset(j1, fIdx);
+                        int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                        int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                                + fta2.getFieldStartOffset(j2, fIdx);
+                        int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                        int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                        if (c != 0) {
+                            return c;
+                        }
+                    }
+                    return 0;
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+            };
+            return op;
+        }
+    }
+
+    private class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return ExternalSortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private IFrameWriter writer;
+                private List<ByteBuffer> inFrames;
+                private ByteBuffer outFrame;
+                LinkedList<File> runs;
+                private FrameTupleAppender outFrameAppender;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    inFrames = (List<ByteBuffer>) env.get(IN_FRAMES);
+                    outFrame = ctx.getResourceManager().allocateFrame();
+                    runs = (LinkedList<File>) env.get(RUNS);
+                    outFrameAppender = new FrameTupleAppender(ctx);
+                    outFrameAppender.reset(outFrame, true);
+                    writer.open();
+                    try {
+                        if (runs.size() <= 0) {
+                            long[] tPointers = (long[]) env.get(TPOINTERS);
+                            if (tPointers != null) {
+                                flushFrames(ctx, inFrames, outFrame, tPointers, writer);
+                                env.set(TPOINTERS, null);
+                            }
+                        } else {
+                            int passCount = 0;
+                            while (runs.size() > 0) {
+                                passCount++;
+                                try {
+                                    doPass(runs, passCount);
+                                } catch (Exception e) {
+                                    e.printStackTrace();
+                                }
+                            }
+                        }
+                    } finally {
+                        writer.close();
+                    }
+                    env.set(IN_FRAMES, null);
+                    env.set(RUNS, null);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    // do nothing
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+
+                // creates a new run from runs that can fit in memory.
+                private void doPass(LinkedList<File> runs, int passCount) throws ClassNotFoundException, Exception {
+                    File newRun = null;
+                    IFrameWriter writer = this.writer;
+                    boolean finalPass = false;
+                    if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
+                        finalPass = true;
+                        for (int i = inFrames.size() - 1; i >= runs.size(); i--) {
+                            inFrames.remove(i);
+                        }
+                    } else {
+                        newRun = ctx.getResourceManager().createFile(
+                                ExternalSortOperatorDescriptor.class.getSimpleName(), ".run");
+                        writer = new RunFileWriter(newRun);
+                    }
+                    RunFileReader[] runCursors = new RunFileReader[inFrames.size()];
+                    FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
+                    Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
+                    ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx, recordDescriptors[0],
+                            inFrames.size(), comparator);
+                    int[] tupleIndexes = new int[inFrames.size()];
+                    for (int i = 0; i < inFrames.size(); i++) {
+                        tupleIndexes[i] = 0;
+                        int runIndex = topTuples.peek().getRunid();
+                        runCursors[runIndex] = new RunFileReader(runs.get(runIndex));
+                        runCursors[runIndex].open();
+                        if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
+                            tupleAccessors[runIndex] = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                            tupleAccessors[runIndex].reset(inFrames.get(runIndex));
+                            setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+                        } else {
+                            closeRun(runIndex, runCursors, tupleAccessors);
+                        }
+                    }
+
+                    while (!topTuples.areRunsExhausted()) {
+                        ReferenceEntry top = topTuples.peek();
+                        int runIndex = top.getRunid();
+                        FrameTupleAccessor fta = top.getAccessor();
+                        int tupleIndex = top.getTupleIndex();
+
+                        if (!outFrameAppender.append(fta, tupleIndex)) {
+                            flushFrame(outFrame, writer);
+                            outFrameAppender.reset(outFrame, true);
+                            if (!outFrameAppender.append(fta, tupleIndex)) {
+                                throw new IllegalStateException();
+                            }
+                        }
+
+                        ++tupleIndexes[runIndex];
+                        setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+                    }
+                    if (outFrameAppender.getTupleCount() > 0) {
+                        flushFrame(outFrame, writer);
+                        outFrameAppender.reset(outFrame, true);
+                    }
+                    runs.subList(0, inFrames.size()).clear();
+                    if (!finalPass) {
+                        runs.add(0, newRun);
+                    }
+                }
+
+                private void setNextTopTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
+                        FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws IOException {
+                    boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+                    if (exists) {
+                        topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
+                    } else {
+                        topTuples.pop();
+                        closeRun(runIndex, runCursors, tupleAccessors);
+                    }
+                }
+
+                private boolean hasNextTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
+                        FrameTupleAccessor[] tupleAccessors) throws IOException {
+                    if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+                        return false;
+                    } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
+                        ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
+                        if (runCursors[runIndex].nextFrame(buf)) {
+                            tupleIndexes[runIndex] = 0;
+                            return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+                        } else {
+                            return false;
+                        }
+                    } else {
+                        return true;
+                    }
+                }
+
+                private void closeRun(int index, RunFileReader[] runCursors, IFrameTupleAccessor[] tupleAccessor) {
+                    runCursors[index] = null;
+                    tupleAccessor[index] = null;
+                }
+            };
+            return op;
+        }
+    }
+
+    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
+        return new Comparator<ReferenceEntry>() {
+            public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+                FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
+                FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+                int j1 = (Integer) tp1.getTupleIndex();
+                int j2 = (Integer) tp2.getTupleIndex();
+                byte[] b1 = fta1.getBuffer().array();
+                byte[] b2 = fta2.getBuffer().array();
+                for (int f = 0; f < sortFields.length; ++f) {
+                    int fIdx = sortFields[f];
+                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                            + fta1.getFieldStartOffset(j1, fIdx);
+                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                            + fta2.getFieldStartOffset(j2, fIdx);
+                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    if (c != 0) {
+                        return c;
+                    }
+                }
+                return 0;
+            }
+        };
+    }
+
+    private void flushFrames(IHyracksContext ctx, List<ByteBuffer> inFrames, ByteBuffer outFrame, long[] tPointers,
+            IFrameWriter writer) throws HyracksDataException {
+        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+        FrameTupleAppender outFrameAppender = new FrameTupleAppender(ctx);
+        for (ByteBuffer buf : inFrames) {
+            buf.position(0);
+            buf.limit(buf.capacity());
+        }
+        outFrameAppender.reset(outFrame, true);
+        for (int ptr = 0; ptr < tPointers.length; ++ptr) {
+            long tp = tPointers[ptr];
+            int i = (int) ((tp >> 32) & 0xffffffff);
+            int j = (int) (tp & 0xffffffff);
+            ByteBuffer buffer = inFrames.get(i);
+            accessor.reset(buffer);
+            if (!outFrameAppender.append(accessor, j)) {
+                flushFrame(outFrame, writer);
+                outFrameAppender.reset(outFrame, true);
+                if (!outFrameAppender.append(accessor, j)) {
+                    throw new IllegalStateException();
+                }
+            }
+        }
+        if (outFrameAppender.getTupleCount() > 0) {
+            flushFrame(outFrame, writer);
+            outFrame.position(0);
+            outFrame.limit(outFrame.capacity());
+        }
+    }
+
+    private void flushFrame(ByteBuffer frame, IFrameWriter writer) throws HyracksDataException {
+        frame.position(0);
+        frame.limit(frame.capacity());
+        writer.nextFrame(frame);
+    }
+
+    private class RunFileWriter implements IFrameWriter {
+        private final File file;
+        private FileChannel channel;
+
+        public RunFileWriter(File file) {
+            this.file = file;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            RandomAccessFile raf;
+            try {
+                raf = new RandomAccessFile(file, "rw");
+            } catch (FileNotFoundException e) {
+                throw new HyracksDataException(e);
+            }
+            channel = raf.getChannel();
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            int remain = buffer.capacity();
+            while (remain > 0) {
+                int len;
+                try {
+                    len = channel.write(buffer);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+                if (len < 0) {
+                    throw new HyracksDataException("Error writing data");
+                }
+                remain -= len;
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                channel.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    public static class RunFileReader implements IFrameReader {
+        private final File file;
+        private FileChannel channel;
+
+        public RunFileReader(File file) throws FileNotFoundException {
+            this.file = file;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            RandomAccessFile raf;
+            try {
+                raf = new RandomAccessFile(file, "r");
+            } catch (FileNotFoundException e) {
+                throw new HyracksDataException(e);
+            }
+            channel = raf.getChannel();
+        }
+
+        @Override
+        public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            buffer.clear();
+            int remain = buffer.capacity();
+            while (remain > 0) {
+                int len;
+                try {
+                    len = channel.read(buffer);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+                if (len < 0) {
+                    return false;
+                }
+                remain -= len;
+            }
+            return true;
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                channel.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/FieldHashPartitionComputerFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/FieldHashPartitionComputerFactory.java
new file mode 100644
index 0000000..952af68
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/FieldHashPartitionComputerFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+
+public class FieldHashPartitionComputerFactory implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final int[] hashFields;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+
+    public FieldHashPartitionComputerFactory(int[] hashFields, IBinaryHashFunctionFactory[] hashFunctionFactories) {
+        this.hashFields = hashFields;
+        this.hashFunctionFactories = hashFunctionFactories;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
+        for (int i = 0; i < hashFunctionFactories.length; ++i) {
+            hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();
+        }
+        return new ITuplePartitionComputer() {
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) {
+                int h = 0;
+                int startOffset = accessor.getTupleStartOffset(tIndex);
+                int slotLength = accessor.getFieldSlotsLength();
+                for (int j = 0; j < hashFields.length; ++j) {
+                    int fIdx = hashFields[j];
+                    IBinaryHashFunction hashFn = hashFunctions[j];
+                    int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
+                    int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
+                    int fh = hashFn
+                            .hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
+                    h += fh;
+                }
+                if (h < 0) {
+                    h = -h;
+                }
+                return h % nParts;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/HashDataWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/HashDataWriter.java
new file mode 100644
index 0000000..5e9eef3
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/HashDataWriter.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+
+public class HashDataWriter implements IFrameWriter {
+    private final int consumerPartitionCount;
+    private final IFrameWriter[] epWriters;
+    private final FrameTupleAppender[] appenders;
+    private final FrameTupleAccessor tupleAccessor;
+    private final ITuplePartitionComputer tpc;
+
+    public HashDataWriter(IHyracksContext ctx, int consumerPartitionCount, IEndpointDataWriterFactory edwFactory,
+            RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
+        this.consumerPartitionCount = consumerPartitionCount;
+        epWriters = new IFrameWriter[consumerPartitionCount];
+        appenders = new FrameTupleAppender[consumerPartitionCount];
+        for (int i = 0; i < consumerPartitionCount; ++i) {
+            try {
+                epWriters[i] = edwFactory.createFrameWriter(i);
+                appenders[i] = new FrameTupleAppender(ctx);
+                appenders[i].reset(ctx.getResourceManager().allocateFrame(), true);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        tupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        this.tpc = tpc;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < epWriters.length; ++i) {
+            if (appenders[i].getTupleCount() > 0) {
+                flushFrame(appenders[i].getBuffer(), epWriters[i]);
+            }
+            epWriters[i].close();
+        }
+    }
+
+    private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        frameWriter.nextFrame(buffer);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        for (int i = 0; i < epWriters.length; ++i) {
+            epWriters[i].open();
+            appenders[i].reset(appenders[i].getBuffer(), true);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        tupleAccessor.reset(buffer);
+        int tupleCount = tupleAccessor.getTupleCount();
+        for (int i = 0; i < tupleCount; ++i) {
+            int h = tpc.partition(tupleAccessor, i, consumerPartitionCount);
+            FrameTupleAppender appender = appenders[h];
+            if (!appender.append(tupleAccessor, i)) {
+                ByteBuffer appenderBuffer = appender.getBuffer();
+                flushFrame(appenderBuffer, epWriters[h]);
+                appender.reset(appenderBuffer, true);
+                if (!appender.append(tupleAccessor, i)) {
+                    throw new IllegalStateException();
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/IMapper.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/IMapper.java
new file mode 100644
index 0000000..af32e1d
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/IMapper.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMapper {
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException;
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/IMapperFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/IMapperFactory.java
new file mode 100644
index 0000000..05dc203
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/IMapperFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMapperFactory extends Serializable {
+    public IMapper createMapper() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ITupleAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ITupleAggregator.java
new file mode 100644
index 0000000..ca3090b
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ITupleAggregator.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleAggregator {
+    void add(Object[] data);
+
+    void init(Object[] data);
+
+    void write(IDataWriter<Object[]> writer) throws HyracksDataException;
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ITupleAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ITupleAggregatorFactory.java
new file mode 100644
index 0000000..ae74ce7
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/ITupleAggregatorFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleAggregatorFactory extends Serializable {
+    public ITupleAggregator createTupleAggregator() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/InMemorySortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/InMemorySortOperatorDescriptor.java
new file mode 100644
index 0000000..4e9fd2c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/InMemorySortOperatorDescriptor.java
@@ -0,0 +1,288 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String BUFFERS = "buffers";
+    private static final String TPOINTERS = "tpointers";
+
+    private static final long serialVersionUID = 1L;
+    private final int[] sortFields;
+    private IBinaryComparatorFactory[] comparatorFactories;
+
+    public InMemorySortOperatorDescriptor(JobSpecification spec, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity();
+        MergeActivity ma = new MergeActivity();
+
+        builder.addTask(sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addTask(ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    private class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemorySortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private List<ByteBuffer> buffers;
+
+                private final FrameTupleAccessor fta1 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                private final FrameTupleAccessor fta2 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+
+                @Override
+                public void open() throws HyracksDataException {
+                    buffers = new ArrayList<ByteBuffer>();
+                    env.set(BUFFERS, buffers);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copy = ctx.getResourceManager().allocateFrame();
+                    FrameUtils.copy(buffer, copy);
+                    buffers.add(copy);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                    int nBuffers = buffers.size();
+                    int totalTCount = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(buffers.get(i));
+                        totalTCount += accessor.getTupleCount();
+                    }
+                    long[] tPointers = new long[totalTCount];
+                    int ptr = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(buffers.get(i));
+                        int tCount = accessor.getTupleCount();
+                        for (int j = 0; j < tCount; ++j) {
+                            tPointers[ptr++] = (((long) i) << 32) + j;
+                        }
+                    }
+                    if (tPointers.length > 0) {
+                        sort(tPointers, 0, tPointers.length);
+                    }
+                    env.set(TPOINTERS, tPointers);
+                }
+
+                private void sort(long[] tPointers, int offset, int length) {
+                    int m = offset + (length >> 1);
+                    long v = tPointers[m];
+
+                    int a = offset;
+                    int b = a;
+                    int c = offset + length - 1;
+                    int d = c;
+                    while (true) {
+                        while (b <= c && compare(tPointers[b], v) <= 0) {
+                            if (compare(tPointers[b], v) == 0) {
+                                swap(tPointers, a++, b);
+                            }
+                            ++b;
+                        }
+                        while (c >= b && compare(tPointers[c], v) >= 0) {
+                            if (compare(tPointers[c], v) == 0) {
+                                swap(tPointers, c, d--);
+                            }
+                            --c;
+                        }
+                        if (b > c)
+                            break;
+                        swap(tPointers, b++, c--);
+                    }
+
+                    int s;
+                    int n = offset + length;
+                    s = Math.min(a - offset, b - a);
+                    vecswap(tPointers, offset, b - s, s);
+                    s = Math.min(d - c, n - d - 1);
+                    vecswap(tPointers, b, n - s, s);
+
+                    if ((s = b - a) > 1) {
+                        sort(tPointers, offset, s);
+                    }
+                    if ((s = d - c) > 1) {
+                        sort(tPointers, n - s, s);
+                    }
+                }
+
+                private void swap(long x[], int a, int b) {
+                    long t = x[a];
+                    x[a] = x[b];
+                    x[b] = t;
+                }
+
+                private void vecswap(long x[], int a, int b, int n) {
+                    for (int i = 0; i < n; i++, a++, b++) {
+                        swap(x, a, b);
+                    }
+                }
+
+                private int compare(long tp1, long tp2) {
+                    int i1 = (int) ((tp1 >> 32) & 0xffffffff);
+                    int j1 = (int) (tp1 & 0xffffffff);
+                    int i2 = (int) ((tp2 >> 32) & 0xffffffff);
+                    int j2 = (int) (tp2 & 0xffffffff);
+                    ByteBuffer buf1 = buffers.get(i1);
+                    ByteBuffer buf2 = buffers.get(i2);
+                    byte[] b1 = buf1.array();
+                    byte[] b2 = buf2.array();
+                    fta1.reset(buf1);
+                    fta2.reset(buf2);
+                    for (int f = 0; f < sortFields.length; ++f) {
+                        int fIdx = sortFields[f];
+                        int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                                + fta1.getFieldStartOffset(j1, fIdx);
+                        int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                        int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                                + fta2.getFieldStartOffset(j2, fIdx);
+                        int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                        int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                        if (c != 0) {
+                            return c;
+                        }
+                    }
+                    return 0;
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+            };
+            return op;
+        }
+    }
+
+    private class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemorySortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private IFrameWriter writer;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    List<ByteBuffer> buffers = (List<ByteBuffer>) env.get(BUFFERS);
+                    long[] tPointers = (long[]) env.get(TPOINTERS);
+                    FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                    FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                    ByteBuffer outFrame = ctx.getResourceManager().allocateFrame();
+                    writer.open();
+                    appender.reset(outFrame, true);
+                    for (int ptr = 0; ptr < tPointers.length; ++ptr) {
+                        long tp = tPointers[ptr];
+                        int i = (int) ((tp >> 32) & 0xffffffff);
+                        int j = (int) (tp & 0xffffffff);
+                        ByteBuffer buffer = buffers.get(i);
+                        accessor.reset(buffer);
+                        if (!appender.append(accessor, j)) {
+                            flushFrame(outFrame);
+                            appender.reset(outFrame, true);
+                            if (!appender.append(accessor, j)) {
+                                throw new IllegalStateException();
+                            }
+                        }
+                    }
+                    if (appender.getTupleCount() > 0) {
+                        flushFrame(outFrame);
+                    }
+                    writer.close();
+                    env.set(BUFFERS, null);
+                    env.set(TPOINTERS, null);
+                }
+
+                private void flushFrame(ByteBuffer frame) throws HyracksDataException {
+                    frame.position(0);
+                    frame.limit(frame.capacity());
+                    writer.nextFrame(frame);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    // do nothing
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNHashPartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNHashPartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..73a7b32
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNHashPartitioningConnectorDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+
+public class MToNHashPartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private ITuplePartitionComputerFactory tpcf;
+
+    public MToNHashPartitioningConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf) {
+        super(spec);
+        this.tpcf = tpcf;
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
+                tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNHashPartitioningMergingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNHashPartitioningMergingConnectorDescriptor.java
new file mode 100644
index 0000000..befdc72
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNHashPartitioningMergingConnectorDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.SortMergeFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+
+public class MToNHashPartitioningMergingConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITuplePartitionComputerFactory tpcf;
+    private final int[] sortFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    public MToNHashPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
+        super(spec);
+        this.tpcf = tpcf;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
+                tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        return new SortMergeFrameReader(ctx, demux, sortFields, comparators, recordDesc);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNRangePartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNRangePartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..ef82ab3
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNRangePartitioningConnectorDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+
+public class MToNRangePartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private class RangeDataWriter implements IFrameWriter {
+        private final IFrameWriter[] epWriters;
+        private final FrameTupleAppender[] appenders;
+        private final FrameTupleAccessor tupleAccessor;
+
+        public RangeDataWriter(IHyracksContext ctx, int consumerPartitionCount, IFrameWriter[] epWriters,
+                FrameTupleAppender[] appenders, RecordDescriptor recordDescriptor) {
+            this.epWriters = epWriters;
+            this.appenders = appenders;
+            tupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            for (int i = 0; i < epWriters.length; ++i) {
+                if (appenders[i].getTupleCount() > 0) {
+                    flushFrame(appenders[i].getBuffer(), epWriters[i]);
+                }
+                epWriters[i].close();
+            }
+        }
+
+        private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
+            buffer.position(0);
+            buffer.limit(buffer.capacity());
+            frameWriter.nextFrame(buffer);
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            if (true) {
+                throw new UnsupportedOperationException();
+            }
+            tupleAccessor.reset(buffer);
+            int slotLength = tupleAccessor.getFieldSlotsLength();
+            int tupleCount = tupleAccessor.getTupleCount();
+            for (int i = 0; i < tupleCount; ++i) {
+                int startOffset = tupleAccessor.getTupleStartOffset(i);
+                int fStart = tupleAccessor.getFieldStartOffset(i, partitioningField);
+                int fEnd = tupleAccessor.getFieldEndOffset(i, partitioningField);
+                int h = 0;
+                FrameTupleAppender appender = appenders[h];
+                if (!appender.append(tupleAccessor, i)) {
+                    ByteBuffer appenderBuffer = appender.getBuffer();
+                    flushFrame(appenderBuffer, epWriters[h]);
+                    appender.reset(appenderBuffer, true);
+                }
+            }
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            for (int i = 0; i < epWriters.length; ++i) {
+                epWriters[i].open();
+                appenders[i].reset(appenders[i].getBuffer(), true);
+            }
+        }
+    }
+
+    private final int partitioningField;
+    private final Object[] splitVector;
+
+    public MToNRangePartitioningConnectorDescriptor(JobSpecification spec, int partitioningField, Object[] splitVector) {
+        super(spec);
+        this.partitioningField = partitioningField;
+        this.splitVector = splitVector;
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
+        final FrameTupleAppender[] appenders = new FrameTupleAppender[nConsumerPartitions];
+        for (int i = 0; i < nConsumerPartitions; ++i) {
+            try {
+                epWriters[i] = edwFactory.createFrameWriter(i);
+                appenders[i] = new FrameTupleAppender(ctx);
+                appenders[i].reset(ctx.getResourceManager().allocateFrame(), true);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        final RangeDataWriter rangeWriter = new RangeDataWriter(ctx, nConsumerPartitions, epWriters, appenders,
+                recordDesc);
+        return rangeWriter;
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNReplicatingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNReplicatingConnectorDescriptor.java
new file mode 100644
index 0000000..48d598f
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MToNReplicatingConnectorDescriptor.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+
+public class MToNReplicatingConnectorDescriptor extends AbstractConnectorDescriptor {
+    public MToNReplicatingConnectorDescriptor(JobSpecification spec) {
+        super(spec);
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc, IEndpointDataWriterFactory edwFactory,
+        int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
+        for (int i = 0; i < nConsumerPartitions; ++i) {
+            epWriters[i] = edwFactory.createFrameWriter(i);
+        }
+        return new IFrameWriter() {
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                buffer.mark();
+                for (int i = 0; i < epWriters.length; ++i) {
+                    if (i != 0) {
+                        buffer.reset();
+                    }
+                    epWriters[i].nextFrame(buffer);
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].close();
+                }
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].open();
+                }
+            }
+        };
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc, IConnectionDemultiplexer demux,
+        int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MapperOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MapperOperatorDescriptor.java
new file mode 100644
index 0000000..f83763a
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MapperOperatorDescriptor.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class MapperOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private class MapperOperator implements IOpenableDataWriterOperator {
+        private IMapper mapper;
+        private IOpenableDataWriter<Object[]> writer;
+
+        @Override
+        public void close() throws HyracksDataException {
+            // writer.writeData(null);
+            writer.close();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            mapper = mapperFactory.createMapper();
+            writer.open();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IllegalArgumentException();
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            mapper.map(data, writer);
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final IMapperFactory mapperFactory;
+
+    public MapperOperatorDescriptor(JobSpecification spec, IMapperFactory mapperFactory,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.mapperFactory = mapperFactory;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
+                recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MaterializingOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MaterializingOperatorDescriptor.java
new file mode 100644
index 0000000..77aeed5
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/MaterializingOperatorDescriptor.java
@@ -0,0 +1,181 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class MaterializingOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected static final String MATERIALIZED_FILE = "materialized-file";
+    protected static final String FRAME_COUNT = "frame-count";
+
+    public MaterializingOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        MaterializerActivityNode ma = new MaterializerActivityNode();
+        ReaderActivityNode ra = new ReaderActivityNode();
+
+        builder.addTask(ma);
+        builder.addSourceEdge(0, ma, 0);
+
+        builder.addTask(ra);
+        builder.addTargetEdge(0, ra, 0);
+
+        builder.addBlockingEdge(ma, ra);
+    }
+
+    private final class MaterializerActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            return new IOperatorNodePushable() {
+                private FileChannel out;
+                private int frameCount;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    File outFile;
+                    try {
+                        outFile = ctx.getResourceManager().createFile("mat", ".dat");
+                        out = new RandomAccessFile(outFile, "rw").getChannel();
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                    env.set(MATERIALIZED_FILE, outFile.getAbsolutePath());
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ++frameCount;
+                    buffer.position(0);
+                    buffer.limit(buffer.capacity());
+                    int rem = buffer.capacity();
+                    while (rem > 0) {
+                        int c;
+                        try {
+                            c = out.write(buffer);
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                        rem -= c;
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    try {
+                        env.set(FRAME_COUNT, frameCount);
+                        out.close();
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return MaterializingOperatorDescriptor.this;
+        }
+    }
+
+    private final class ReaderActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            return new IOperatorNodePushable() {
+                private IFrameWriter writer;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    try {
+                        File inFile = new File((String) env.get(MATERIALIZED_FILE));
+                        int frameCount = (Integer) env.get(FRAME_COUNT);
+                        FileChannel in = new RandomAccessFile(inFile, "r").getChannel();
+                        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
+                        writer.open();
+                        for (int i = 0; i < frameCount; ++i) {
+                            frame.clear();
+                            int rem = frame.capacity();
+                            while (rem > 0) {
+                                int c = in.read(frame);
+                                rem -= c;
+                            }
+                            frame.flip();
+                            writer.nextFrame(frame);
+                        }
+                        writer.close();
+                        in.close();
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(MATERIALIZED_FILE, null);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return MaterializingOperatorDescriptor.this;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/OneToOneConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/OneToOneConnectorDescriptor.java
new file mode 100644
index 0000000..19e4510
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/OneToOneConnectorDescriptor.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+
+public class OneToOneConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public OneToOneConnectorDescriptor(JobSpecification spec) {
+        super(spec);
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return edwFactory.createFrameWriter(index);
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(IHyracksContext ctx, RecordDescriptor recordDesc,
+            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/PrinterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/PrinterOperatorDescriptor.java
new file mode 100644
index 0000000..b024e8a
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/PrinterOperatorDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class PrinterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public PrinterOperatorDescriptor(JobSpecification spec) {
+        super(spec, 1, 0);
+    }
+
+    private class PrinterOperator implements IOpenableDataWriterOperator {
+        @Override
+        public void open() throws HyracksDataException {
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            for (int i = 0; i < data.length; ++i) {
+                System.err.print(String.valueOf(data[i]));
+                System.err.print(", ");
+            }
+            System.err.println();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            throw new IllegalArgumentException();
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
+                recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/RepartitionComputerFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/RepartitionComputerFactory.java
new file mode 100644
index 0000000..90cf2cc
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/RepartitionComputerFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class RepartitionComputerFactory implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+
+    private int factor;
+    private ITuplePartitionComputerFactory delegateFactory;
+
+    public RepartitionComputerFactory(int factor, ITuplePartitionComputerFactory delegate) {
+        this.factor = factor;
+        this.delegateFactory = delegate;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        return new ITuplePartitionComputer() {
+            private ITuplePartitionComputer delegate = delegateFactory.createPartitioner();
+
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                return delegate.partition(accessor, tIndex, factor * nParts) / factor;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/SplitVectorOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/SplitVectorOperatorDescriptor.java
new file mode 100644
index 0000000..b95ee24
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/SplitVectorOperatorDescriptor.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class SplitVectorOperatorDescriptor extends AbstractOperatorDescriptor {
+    private class CollectActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public ActivityNodeId getActivityNodeId() {
+            return id;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return SplitVectorOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
+                private ArrayList<Object[]> buffer;
+
+                @Override
+                public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+                    throw new IllegalArgumentException();
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    buffer = new ArrayList<Object[]>();
+                    env.set(BUFFER, buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+
+                }
+
+                @Override
+                public void writeData(Object[] data) throws HyracksDataException {
+                    buffer.add(data);
+                }
+            };
+            return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
+                    getOperatorId(), 0));
+        }
+    }
+
+    private class SplitActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return SplitVectorOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
+                private IOpenableDataWriter<Object[]> writer;
+
+                @Override
+                public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                }
+
+                @Override
+                public void writeData(Object[] data) throws HyracksDataException {
+                    List<Object[]> buffer = (List<Object[]>) env.get(BUFFER);
+                    int n = buffer.size();
+                    int step = (int) Math.floor(n / (float) splits);
+                    writer.open();
+                    for (int i = 0; i < splits; ++i) {
+                        writer.writeData(buffer.get(step * (i + 1) - 1));
+                    }
+                    writer.close();
+                }
+            };
+            return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
+                    getOperatorId(), 0));
+        }
+    }
+
+    private static final String BUFFER = "buffer";
+
+    private static final long serialVersionUID = 1L;
+
+    private final int splits;
+
+    public SplitVectorOperatorDescriptor(JobSpecification spec, int splits, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.splits = splits;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        CollectActivity ca = new CollectActivity();
+        SplitActivity sa = new SplitActivity();
+
+        builder.addTask(ca);
+        builder.addSourceEdge(0, ca, 0);
+
+        builder.addTask(sa);
+        builder.addTargetEdge(0, sa, 0);
+
+        builder.addBlockingEdge(ca, sa);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ReflectionBasedTupleAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ReflectionBasedTupleAggregatorFactory.java
new file mode 100644
index 0000000..05f3b9f
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ReflectionBasedTupleAggregatorFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.aggregators;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.ITupleAggregator;
+import edu.uci.ics.hyracks.dataflow.std.ITupleAggregatorFactory;
+
+public class ReflectionBasedTupleAggregatorFactory implements
+        ITupleAggregatorFactory {
+
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    private final Class<? extends ITupleAggregator> aggregatorClass;
+
+    public ReflectionBasedTupleAggregatorFactory(
+            Class<? extends ITupleAggregator> aggregatorClass) {
+        this.aggregatorClass = aggregatorClass;
+    }
+
+    @Override
+    public ITupleAggregator createTupleAggregator() throws HyracksDataException {
+        try {
+            return aggregatorClass.newInstance();
+        } catch (InstantiationException e) {
+            throw new HyracksDataException(e);
+        } catch (IllegalAccessException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumGroupAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumGroupAggregator.java
new file mode 100644
index 0000000..88b38b3
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumGroupAggregator.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.aggregators;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.group.IGroupAggregator;
+
+public class SumGroupAggregator implements IGroupAggregator {
+    private static final long serialVersionUID = 1L;
+    private int keyIdx;
+
+    public SumGroupAggregator(int keyIdx) {
+        this.keyIdx = keyIdx;
+    }
+
+    @Override
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException {
+        String key = "";
+        Object[] data;
+        int count = 0;
+        while ((data = reader.readData()) != null) {
+            key = (String) data[keyIdx];
+            ++count;
+        }
+        writer.writeData(new Object[] { key, count });
+    }
+
+    @Override
+    public void close() throws Exception {
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumStringGroupAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumStringGroupAggregator.java
new file mode 100644
index 0000000..dd4e4eb
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumStringGroupAggregator.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.aggregators;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.group.IGroupAggregator;
+
+public class SumStringGroupAggregator implements IGroupAggregator {
+    private static final long serialVersionUID = 1L;
+    private int keyIdx;
+
+    public SumStringGroupAggregator(int keyIdx) {
+        this.keyIdx = keyIdx;
+    }
+
+    @Override
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException {
+        String key = "";
+        Object[] data;
+        int count = 0;
+        while ((data = reader.readData()) != null) {
+            key = (String) data[keyIdx];
+            ++count;
+        }
+        writer.writeData(new Object[] { key, String.valueOf(count) });
+    }
+
+    @Override
+    public void close() throws Exception {
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumTupleAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumTupleAggregator.java
new file mode 100644
index 0000000..87ab523
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumTupleAggregator.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.aggregators;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.ITupleAggregator;
+
+public class SumTupleAggregator implements ITupleAggregator {
+
+    private Object key;
+    private int count;
+
+    @Override
+    public void add(Object[] data) {
+        count++;
+    }
+
+    @Override
+    public void init(Object[] data) {
+        key = data[0];
+        count = 0;
+    }
+
+    @Override
+    public void write(IDataWriter<Object[]> writer) throws HyracksDataException {
+        writer.writeData(new Object[] { key, count });
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
new file mode 100644
index 0000000..4976230
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.base;
+
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+
+public abstract class AbstractActivityNode implements IActivityNode {
+    private static final long serialVersionUID = 1L;
+
+    protected final ActivityNodeId id;
+
+    public AbstractActivityNode() {
+        this.id = new ActivityNodeId(getOwner().getOperatorId(), UUID.randomUUID());
+    }
+
+    @Override
+    public ActivityNodeId getActivityNodeId() {
+        return id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
new file mode 100644
index 0000000..aaae486
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.base;
+
+import java.util.UUID;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected final ConnectorDescriptorId id;
+
+    public AbstractConnectorDescriptor(JobSpecification spec) {
+        this.id = new ConnectorDescriptorId(UUID.randomUUID());
+        spec.getConnectorMap().put(id, this);
+    }
+
+    public ConnectorDescriptorId getConnectorId() {
+        return id;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jconn = new JSONObject();
+
+        jconn.put("type", "connector");
+        jconn.put("id", getConnectorId().getId().toString());
+        jconn.put("java-class", getClass().getName());
+
+        return jconn;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
new file mode 100644
index 0000000..7d979f9
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.base;
+
+import java.util.UUID;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    protected final OperatorDescriptorId odId;
+
+    protected PartitionConstraint partitionConstraint;
+
+    protected String[] partitions;
+
+    protected final RecordDescriptor[] recordDescriptors;
+
+    protected final int inputArity;
+
+    protected final int outputArity;
+
+    public AbstractOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
+        odId = new OperatorDescriptorId(UUID.randomUUID());
+        this.inputArity = inputArity;
+        this.outputArity = outputArity;
+        recordDescriptors = new RecordDescriptor[outputArity];
+        spec.getOperatorMap().put(getOperatorId(), this);
+    }
+
+    @Override
+    public final OperatorDescriptorId getOperatorId() {
+        return odId;
+    }
+
+    @Override
+    public int getInputArity() {
+        return inputArity;
+    }
+
+    @Override
+    public int getOutputArity() {
+        return outputArity;
+    }
+
+    @Override
+    public PartitionConstraint getPartitionConstraint() {
+        return partitionConstraint;
+    }
+
+    @Override
+    public void setPartitionConstraint(PartitionConstraint partitionConstraint) {
+        this.partitionConstraint = partitionConstraint;
+    }
+
+    @Override
+    public RecordDescriptor[] getOutputRecordDescriptors() {
+        return recordDescriptors;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jop = new JSONObject();
+        jop.put("type", "operator");
+        jop.put("id", getOperatorId().getId().toString());
+        jop.put("java-class", getClass().getName());
+        jop.put("in-arity", getInputArity());
+        jop.put("out-arity", getOutputArity());
+        return jop;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
new file mode 100644
index 0000000..018b7b7
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.base;
+
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements
+        IActivityNode {
+    private static final long serialVersionUID = 1L;
+
+    protected final ActivityNodeId activityNodeId;
+
+    public AbstractSingleActivityOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
+        super(spec, inputArity, outputArity);
+        activityNodeId = new ActivityNodeId(odId, UUID.randomUUID());
+    }
+
+    @Override
+    public ActivityNodeId getActivityNodeId() {
+        return activityNodeId;
+    }
+
+    @Override
+    public final IOperatorDescriptor getOwner() {
+        return this;
+    }
+
+    @Override
+    public final void contributeTaskGraph(IActivityGraphBuilder builder) {
+        builder.addTask(this);
+        for (int i = 0; i < getInputArity(); ++i) {
+            builder.addSourceEdge(i, this, i);
+        }
+        for (int i = 0; i < getOutputArity(); ++i) {
+            builder.addTargetEdge(i, this, i);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/IOpenableDataWriterOperator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/IOpenableDataWriterOperator.java
new file mode 100644
index 0000000..b802c6f
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/IOpenableDataWriterOperator.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+
+public interface IOpenableDataWriterOperator extends IOpenableDataWriter<Object[]> {
+    public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer);
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatBinaryComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatBinaryComparatorFactory.java
new file mode 100644
index 0000000..047e7d2
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatBinaryComparatorFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class FloatBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatBinaryComparatorFactory INSTANCE = new FloatBinaryComparatorFactory();
+
+    private FloatBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return Float.compare(FloatSerializerDeserializer.getFloat(b1, s1), FloatSerializerDeserializer
+                        .getFloat(b2, s2));
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatBinaryHashFunctionFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..11aca03
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatBinaryHashFunctionFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class FloatBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatBinaryHashFunctionFactory INSTANCE = new FloatBinaryHashFunctionFactory();
+
+    private FloatBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                return FloatSerializerDeserializer.getIntBits(bytes, offset);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatComparatorFactory.java
new file mode 100644
index 0000000..c2a5b9e
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatComparatorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+
+public class FloatComparatorFactory implements IComparatorFactory<Float> {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatComparatorFactory INSTANCE = new FloatComparatorFactory();
+
+    private FloatComparatorFactory() {
+    }
+
+    @Override
+    public IComparator<Float> createComparator() {
+        return new IComparator<Float>() {
+            @Override
+            public int compare(Float o1, Float o2) {
+                return o1.compareTo(o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatHashFunctionFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatHashFunctionFactory.java
new file mode 100644
index 0000000..d45f056
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatHashFunctionFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FloatHashFunctionFactory implements IHashFunctionFactory<Float> {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatHashFunctionFactory INSTANCE = new FloatHashFunctionFactory();
+
+    private FloatHashFunctionFactory() {
+    }
+
+    @Override
+    public IHashFunction<Float> createHashFunction() {
+        return new IHashFunction<Float>() {
+            @Override
+            public int hash(Float o) throws HyracksDataException {
+                return o.hashCode();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatSerializerDeserializer.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatSerializerDeserializer.java
new file mode 100644
index 0000000..a432734
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/FloatSerializerDeserializer.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FloatSerializerDeserializer implements ISerializerDeserializer<Float> {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatSerializerDeserializer INSTANCE = new FloatSerializerDeserializer();
+
+    private FloatSerializerDeserializer() {
+    }
+
+    @Override
+    public Float deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readFloat();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(Float instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeFloat(instance.floatValue());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    static float getFloat(byte[] bytes, int offset) {
+        return Float.intBitsToFloat(getIntBits(bytes, offset));
+    }
+
+    public static int getIntBits(byte[] bytes, int offset) {
+        return IntegerSerializerDeserializer.getInt(bytes, offset);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntArraySerializerDeserializer.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntArraySerializerDeserializer.java
new file mode 100644
index 0000000..1c83e3d
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntArraySerializerDeserializer.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class IntArraySerializerDeserializer implements ISerializerDeserializer<int[]> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntArraySerializerDeserializer INSTANCE = new IntArraySerializerDeserializer();
+
+    private IntArraySerializerDeserializer() {
+    }
+
+    @Override
+    public int[] deserialize(DataInput in) throws HyracksDataException {
+        try {
+            int len = in.readInt();
+            int[] array = new int[len];
+            for (int i = 0; i < array.length; ++i) {
+                array[i] = in.readInt();
+            }
+            return array;
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(int[] instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeInt(instance.length);
+            for (int i = 0; i < instance.length; ++i) {
+                out.writeInt(instance[i]);
+            }
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerBinaryComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerBinaryComparatorFactory.java
new file mode 100644
index 0000000..77b8446
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerBinaryComparatorFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class IntegerBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerBinaryComparatorFactory INSTANCE = new IntegerBinaryComparatorFactory();
+
+    private IntegerBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int v1 = IntegerSerializerDeserializer.getInt(b1, s1);
+                int v2 = IntegerSerializerDeserializer.getInt(b2, s2);
+                return v1 < v2 ? -1 : (v1 > v2 ? 1 : 0);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerBinaryHashFunctionFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..50b26d2
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerBinaryHashFunctionFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class IntegerBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerBinaryHashFunctionFactory INSTANCE = new IntegerBinaryHashFunctionFactory();
+
+    private IntegerBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                return IntegerSerializerDeserializer.getInt(bytes, offset);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerComparatorFactory.java
new file mode 100644
index 0000000..11496e1
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerComparatorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+
+public class IntegerComparatorFactory implements IComparatorFactory<Integer> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerComparatorFactory INSTANCE = new IntegerComparatorFactory();
+
+    private IntegerComparatorFactory() {
+    }
+
+    @Override
+    public IComparator<Integer> createComparator() {
+        return new IComparator<Integer>() {
+            @Override
+            public int compare(Integer o1, Integer o2) {
+                return o1.compareTo(o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerHashFunctionFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerHashFunctionFactory.java
new file mode 100644
index 0000000..0623759
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerHashFunctionFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class IntegerHashFunctionFactory implements IHashFunctionFactory<Integer> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerHashFunctionFactory INSTANCE = new IntegerHashFunctionFactory();
+
+    private IntegerHashFunctionFactory() {
+    }
+
+    @Override
+    public IHashFunction<Integer> createHashFunction() {
+        return new IHashFunction<Integer>() {
+            @Override
+            public int hash(Integer o) throws HyracksDataException {
+                return o.hashCode();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerSerializerDeserializer.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerSerializerDeserializer.java
new file mode 100644
index 0000000..977d901
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/IntegerSerializerDeserializer.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class IntegerSerializerDeserializer implements ISerializerDeserializer<Integer> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerSerializerDeserializer INSTANCE = new IntegerSerializerDeserializer();
+
+    private IntegerSerializerDeserializer() {
+    }
+
+    @Override
+    public Integer deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readInt();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(Integer instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeInt(instance.intValue());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    static int getInt(byte[] bytes, int offset) {
+        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+                + ((bytes[offset + 3] & 0xff) << 0);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringBinaryComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringBinaryComparatorFactory.java
new file mode 100644
index 0000000..b5bd118
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringBinaryComparatorFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class StringBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final StringBinaryComparatorFactory INSTANCE = new StringBinaryComparatorFactory();
+
+    private StringBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int utflen1 = StringUtils.getUTFLen(b1, s1);
+                int utflen2 = StringUtils.getUTFLen(b2, s2);
+
+                int c1 = 0;
+                int c2 = 0;
+
+                int s1Start = s1 + 2;
+                int s2Start = s2 + 2;
+
+                while (c1 < utflen1 && c2 < utflen2) {
+                    char ch1 = StringUtils.charAt(b1, s1Start + c1);
+                    char ch2 = StringUtils.charAt(b2, s2Start + c2);
+
+                    if (ch1 != ch2) {
+                        return ch1 - ch2;
+                    }
+                    c1 += StringUtils.charSize(b1, s1Start + c1);
+                    c2 += StringUtils.charSize(b2, s2Start + c2);
+                }
+                return utflen1 - utflen2;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringBinaryHashFunctionFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..ff38ce9
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringBinaryHashFunctionFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class StringBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    public static final StringBinaryHashFunctionFactory INSTANCE = new StringBinaryHashFunctionFactory();
+
+    private static final long serialVersionUID = 1L;
+
+    private StringBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                int h = 0;
+                int utflen = StringUtils.getUTFLen(bytes, offset);
+                int sStart = offset + 2;
+                int c = 0;
+
+                while (c < utflen) {
+                    char ch = StringUtils.charAt(bytes, sStart + c);
+                    h = 31 * h + ch;
+                    c += StringUtils.charSize(bytes, sStart + c);
+                }
+                return h;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringComparatorFactory.java
new file mode 100644
index 0000000..ae8239c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringComparatorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+
+public class StringComparatorFactory implements IComparatorFactory<String> {
+    private static final long serialVersionUID = 1L;
+
+    public static final StringComparatorFactory INSTANCE = new StringComparatorFactory();
+
+    private StringComparatorFactory() {
+    }
+
+    @Override
+    public IComparator<String> createComparator() {
+        return new IComparator<String>() {
+            @Override
+            public int compare(String o1, String o2) {
+                return o1.compareTo(o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringHashFunctionFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringHashFunctionFactory.java
new file mode 100644
index 0000000..9803dff
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringHashFunctionFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class StringHashFunctionFactory implements IHashFunctionFactory<String> {
+    private static final long serialVersionUID = 1L;
+
+    public static final StringHashFunctionFactory INSTANCE = new StringHashFunctionFactory();
+
+    private StringHashFunctionFactory() {
+    }
+
+    @Override
+    public IHashFunction<String> createHashFunction() {
+        return new IHashFunction<String>() {
+            @Override
+            public int hash(String o) throws HyracksDataException {
+                return o.hashCode();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringSerializerDeserializer.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringSerializerDeserializer.java
new file mode 100644
index 0000000..f2db5c0
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringSerializerDeserializer.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class StringSerializerDeserializer implements ISerializerDeserializer<String> {
+    public static final StringSerializerDeserializer INSTANCE = new StringSerializerDeserializer();
+
+    private static final long serialVersionUID = 1L;
+
+    private StringSerializerDeserializer() {
+    }
+
+    @Override
+    public String deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readUTF();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(String instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeUTF(instance);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringUtils.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringUtils.java
new file mode 100644
index 0000000..1c6a964
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/data/StringUtils.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.data;
+
+public class StringUtils {
+    public static int charSize(byte[] b, int s) {
+        int c = (int) b[s] & 0xff;
+        switch (c >> 4) {
+            case 0:
+            case 1:
+            case 2:
+            case 3:
+            case 4:
+            case 5:
+            case 6:
+            case 7:
+                return 1;
+
+            case 12:
+            case 13:
+                return 2;
+
+            case 14:
+                return 3;
+        }
+        throw new IllegalStateException();
+    }
+
+    public static char charAt(byte[] b, int s) {
+        int c = (int) b[s] & 0xff;
+        switch (c >> 4) {
+            case 0:
+            case 1:
+            case 2:
+            case 3:
+            case 4:
+            case 5:
+            case 6:
+            case 7:
+                return (char) c;
+
+            case 12:
+            case 13:
+                return (char) (((c & 0x1F) << 6) | (((int) b[s + 1]) & 0x3F));
+
+            case 14:
+                return (char) (((c & 0x0F) << 12) | ((((int) b[s + 1]) & 0x3F) << 6) | ((((int) b[s + 2]) & 0x3F) << 0));
+
+            default:
+                throw new IllegalArgumentException();
+        }
+    }
+
+    public static int getUTFLen(byte[] b, int s) {
+        return ((b[s] & 0xff) << 8) + ((b[s + 1] & 0xff) << 0);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..43d821c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileScanOperatorDescriptor.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    protected FileSplit[] splits;
+
+    public AbstractFileScanOperatorDescriptor(JobSpecification spec, FileSplit[] splits,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 0, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.splits = splits;
+    }
+
+    protected abstract IRecordReader createRecordReader(File file, RecordDescriptor desc) throws Exception;
+
+    protected abstract void configure() throws Exception;
+
+    protected class FileScanOperator implements IOpenableDataWriterOperator {
+        private IOpenableDataWriter<Object[]> writer;
+        private int index;
+
+        FileScanOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IndexOutOfBoundsException("Invalid index: " + index);
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FileSplit split = splits[index];
+            RecordDescriptor desc = recordDescriptors[0];
+            try {
+                IRecordReader reader = createRecordReader(split.getLocalFile(), desc);
+                if (desc == null) {
+                    desc = recordDescriptors[0];
+                }
+                writer.open();
+                try {
+                    while (true) {
+                        Object[] record = new Object[desc.getFields().length];
+                        if (!reader.read(record)) {
+                            break;
+                        }
+                        writer.writeData(record);
+                    }
+                } finally {
+                    reader.close();
+                    writer.close();
+                }
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // do nothing
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new FileScanOperator(partition), null);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..76fef14
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    protected class FileWriteOperator implements IOpenableDataWriterOperator {
+        private int index;
+        private IRecordWriter writer;
+
+        FileWriteOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            throw new IllegalArgumentException();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FileSplit split = splits[index];
+            try {
+                writer = createRecordWriter(split.getLocalFile(), index);
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            writer.close();
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            try {
+                writer.write(data);
+
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    protected FileSplit[] splits;
+
+    public FileSplit[] getSplits() {
+        return splits;
+    }
+
+    public void setSplits(FileSplit[] splits) {
+        this.splits = splits;
+    }
+
+    public AbstractFileWriteOperatorDescriptor(JobSpecification spec, FileSplit[] splits) {
+        super(spec, 1, 0);
+        this.splits = splits;
+    }
+
+    protected abstract IRecordWriter createRecordWriter(File file, int index) throws Exception;
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
+                recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/CSVFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/CSVFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..54724a4
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/CSVFileScanOperatorDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class CSVFileScanOperatorDescriptor extends
+        AbstractFileScanOperatorDescriptor {
+    private static class CSVRecordReaderImpl implements IRecordReader {
+        private final BufferedReader in;
+        private final char separator;
+        private final String quotes;
+
+        CSVRecordReaderImpl(File file, RecordDescriptor desc, char separator,
+                String quotes) throws Exception {
+            in = new BufferedReader(new InputStreamReader(new FileInputStream(
+                    file)));
+            this.separator = separator;
+            this.quotes = quotes;
+        }
+
+        @Override
+        public void close() {
+            try {
+                in.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            String line = in.readLine();
+            if (line == null) {
+                return false;
+            }
+            int fid = 0;
+            char[] chars = line.toCharArray();
+            int i = 0;
+            boolean insideQuote = false;
+            char quoteChar = 0;
+            int partStart = 0;
+            boolean skipNext = false;
+            while (fid < record.length && i < chars.length) {
+                char c = chars[i];
+                if (!skipNext) {
+                    if (quotes.indexOf(c) >= 0) {
+                        if (insideQuote) {
+                            if (quoteChar == c) {
+                                insideQuote = false;
+                            }
+                        } else {
+                            insideQuote = true;
+                            quoteChar = c;
+                        }
+                    } else if (c == separator) {
+                        if (!insideQuote) {
+                            record[fid++] = String.valueOf(chars, partStart, i
+                                    - partStart);
+                            partStart = i + 1;
+                        }
+                    } else if (c == '\\') {
+                        skipNext = true;
+                    }
+                } else {
+                    skipNext = false;
+                }
+                ++i;
+            }
+            if (fid < record.length) {
+                record[fid] = String.valueOf(chars, partStart, i - partStart);
+            }
+            return true;
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final char separator;
+    private final String quotes;
+
+    public CSVFileScanOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        this(spec, splits, recordDescriptor, ',', "'\"");
+    }
+
+    public CSVFileScanOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, RecordDescriptor recordDescriptor,
+            char separator, String quotes) {
+        super(spec, splits, recordDescriptor);
+        this.separator = separator;
+        this.quotes = quotes;
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc)
+            throws Exception {
+        return new CSVRecordReaderImpl(file, desc, separator, quotes);
+    }
+
+	@Override
+	protected void configure() throws Exception {
+		// currently a no-op, but is meant to initialize , if required before it is asked 
+		// to create a record reader
+		// this is executed at the node and is useful for operators that could not be 
+		// initialized from the client completely, because of lack of information specific 
+		// to the node where the operator gets executed. 
+		
+	}
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
new file mode 100644
index 0000000..004499a
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+import java.io.Serializable;
+
+public class FileSplit implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final String nodeName;
+
+    private final File file;
+
+    public FileSplit(String nodeName, File file) {
+        this.nodeName = nodeName;
+        this.file = file;
+    }
+
+    public String getNodeName() {
+        return nodeName;
+    }
+
+    public File getLocalFile() {
+        return file;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordReader.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordReader.java
new file mode 100644
index 0000000..4516c8d
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordReader.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+public interface IRecordReader {
+
+	  public boolean read(Object[] record) throws Exception;
+
+      public void close();
+      
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordWriter.java
new file mode 100644
index 0000000..a319727
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordWriter.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+
+public interface IRecordWriter {
+	  
+      public void close();
+      
+      public void write(Object[] record) throws Exception;
+
+	
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..19c4ba9
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileScanOperatorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class LineFileScanOperatorDescriptor extends
+        AbstractFileScanOperatorDescriptor {
+    private static class LineReaderImpl extends  RecordReader {
+        private File file;
+
+        LineReaderImpl(File file) throws Exception {
+        	super(new Object[]{file});
+        	this.file = file;
+        }
+
+		@Override
+		public InputStream createInputStream(Object[] args) throws Exception{
+			this.file = (File)args[0];
+			return new FileInputStream(file) ;
+		}
+       }
+
+    private static final long serialVersionUID = 1L;
+
+    public LineFileScanOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc)
+            throws Exception {
+        return new LineReaderImpl(file);
+    }
+    
+    @Override
+	protected void configure() throws Exception {
+		// currently a no-op, but is meant to initialize , if required before it is asked 
+		// to create a record reader
+		// this is executed at the node and is useful for operators that could not be 
+		// initialized from the client completely, because of lack of information specific 
+		// to the node where the operator gets executed. 
+		
+	}
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..54fce9d
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class LineFileWriteOperatorDescriptor extends AbstractFileWriteOperatorDescriptor {
+   
+	private static class LineWriterImpl extends RecordWriter {
+		
+		File file;
+		
+        LineWriterImpl(File file,int []columns, char separator) throws Exception {
+        	super(columns,separator, new Object[]{file});
+        	this.file=file;    
+        }
+
+	    private static final long serialVersionUID = 1L;
+	
+		@Override
+		public OutputStream createOutputStream(Object[] args) throws Exception {
+			return new FileOutputStream((File)args[0]);
+		}
+	}
+    
+    private int[]columns ;
+    private char separator;
+    
+    
+	public LineFileWriteOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits) {
+        this(spec, splits, null, RecordWriter.COMMA);
+    }
+
+    public LineFileWriteOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, int[] columns) {
+        this(spec, splits, columns, RecordWriter.COMMA);
+    }
+
+    public LineFileWriteOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, int[] columns, char separator) {
+        super(spec, splits);
+        this.columns = columns;
+        this.separator = separator;
+    }
+	
+	
+    @Override
+    protected IRecordWriter createRecordWriter(File file,int index) throws Exception {
+        return new LineWriterImpl(file,columns,separator);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..9dd223e
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class RecordFileScanOperatorDescriptor extends AbstractFileScanOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public RecordFileScanOperatorDescriptor(JobSpecification spec, FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+    }
+
+    private static class RecordReaderImpl implements IRecordReader {
+        private RecordDescriptor recordDesc;
+        private DataInputStream in;
+
+        RecordReaderImpl(File file, RecordDescriptor recordDesc) throws Exception {
+            this.in = new DataInputStream(new BufferedInputStream(new FileInputStream(file)));
+            this.recordDesc = recordDesc;
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            in.mark(1);
+            if (in.read() < 0) {
+                return false;
+            }
+            in.reset();
+            for (int i = 0; i < record.length; ++i) {
+                record[i] = recordDesc.getFields()[i].deserialize(in);
+            }
+            return true;
+        }
+
+        @Override
+        public void close() {
+            try {
+                in.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc) throws Exception {
+        return new RecordReaderImpl(file, desc);
+    }
+    
+    @Override
+	protected void configure() throws Exception {
+		// currently a no-op, but is meant to initialize , if required before it is asked 
+		// to create a record reader
+		// this is executed at the node and is useful for operators that could not be 
+		// initialized from the client completely, because of lack of information specific 
+		// to the node where the operator gets executed. 
+		
+	}
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordReader.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordReader.java
new file mode 100644
index 0000000..2064cc2
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordReader.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+public  abstract class RecordReader implements IRecordReader {
+
+    private final BufferedReader bufferedReader;
+	private InputStream inputStream;
+	
+	@Override
+	public void close() {
+        try {
+        	bufferedReader.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public boolean read(Object[] record) throws Exception {
+        String line = bufferedReader.readLine();
+        if (line == null) {
+            return false;
+        }
+        record[0] = line;
+        return true;
+    }
+	
+	public abstract InputStream createInputStream(Object[] args) throws Exception;
+	
+	public RecordReader(Object[] args) throws Exception{
+		try{
+			bufferedReader = new BufferedReader(new InputStreamReader(createInputStream(args)));
+		}catch(Exception e){
+			e.printStackTrace();
+			throw e;
+		}
+	}
+	
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordWriter.java
new file mode 100644
index 0000000..1c52b25
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordWriter.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+
+public abstract class RecordWriter implements IRecordWriter{
+
+	 
+	protected final BufferedWriter bufferedWriter;
+    protected final int[] columns;
+    protected final char separator;
+    
+    public static final char COMMA = ',';
+    
+    public RecordWriter(Object [] args) throws Exception{
+    	OutputStream outputStream = createOutputStream(args);
+    	if(outputStream != null){
+    		bufferedWriter = new BufferedWriter(new OutputStreamWriter(createOutputStream(args)));
+    	}else{
+    		bufferedWriter = null;
+    	}
+    	this.columns = null;
+    	this.separator = COMMA;
+    }
+    
+    public RecordWriter(int []columns, char separator, Object[] args) throws Exception{
+    	OutputStream outputStream = createOutputStream(args);
+    	if(outputStream != null){
+    		bufferedWriter = new BufferedWriter(new OutputStreamWriter(outputStream));
+    	}else{
+    		bufferedWriter = null;
+    	}
+    	this.columns = columns;
+    	this.separator = separator;
+    }
+    
+	@Override
+     public void close() {
+         try {
+             bufferedWriter.close();
+         } catch (IOException e) {
+             e.printStackTrace();
+         }
+     }
+
+     @Override
+     public void write(Object[] record) throws Exception {
+         if (columns == null) {
+             for (int i = 0; i < record.length; ++i) {
+                 if (i != 0) {
+                     bufferedWriter.write(separator);
+                 }
+                 bufferedWriter.write(String.valueOf(record[i]));
+             }
+         } else {
+             for (int i = 0; i < columns.length; ++i) {
+                 if (i != 0) {
+                     bufferedWriter.write(separator);
+                 }
+                 bufferedWriter.write(String.valueOf(record[columns[i]]));
+             }
+         }
+         bufferedWriter.write("\n");
+     }
+     
+     public abstract OutputStream createOutputStream(Object[] args) throws Exception;
+   
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
new file mode 100644
index 0000000..784975f
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
@@ -0,0 +1,193 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+
+class GroupingHashTable {
+    /**
+     * The pointers in the link store 3 int values for each entry in the hashtable: (bufferIdx, tIndex, accumulatorIdx).
+     * 
+     * @author vinayakb
+     */
+    private static class Link {
+        private static final int INIT_POINTERS_SIZE = 9;
+
+        int[] pointers;
+        int size;
+
+        Link() {
+            pointers = new int[INIT_POINTERS_SIZE];
+            size = 0;
+        }
+
+        void add(int bufferIdx, int tIndex, int accumulatorIdx) {
+            while (size + 3 > pointers.length) {
+                pointers = Arrays.copyOf(pointers, pointers.length * 2);
+            }
+            pointers[size++] = bufferIdx;
+            pointers[size++] = tIndex;
+            pointers[size++] = accumulatorIdx;
+        }
+    }
+
+    private static final int INIT_ACCUMULATORS_SIZE = 8;
+    private final IHyracksContext ctx;
+    private final FrameTupleAppender appender;
+    private final List<ByteBuffer> buffers;
+    private final Link[] table;
+    private IAccumulatingAggregator[] accumulators;
+    private int accumulatorSize;
+
+    private int lastBIndex;
+    private final int[] fields;
+    private final int[] storedKeys;
+    private final IBinaryComparator[] comparators;
+    private final FrameTuplePairComparator ftpc;
+    private final ITuplePartitionComputer tpc;
+    private final IAccumulatingAggregatorFactory aggregatorFactory;
+    private final RecordDescriptor inRecordDescriptor;
+    private final RecordDescriptor outRecordDescriptor;
+
+    private final FrameTupleAccessor storedKeysAccessor;
+
+    GroupingHashTable(IHyracksContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
+        ITuplePartitionComputerFactory tpcf, IAccumulatingAggregatorFactory aggregatorFactory,
+        RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize) {
+        this.ctx = ctx;
+        appender = new FrameTupleAppender(ctx);
+        buffers = new ArrayList<ByteBuffer>();
+        table = new Link[tableSize];
+        accumulators = new IAccumulatingAggregator[INIT_ACCUMULATORS_SIZE];
+        accumulatorSize = 0;
+        this.fields = fields;
+        storedKeys = new int[fields.length];
+        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
+        for (int i = 0; i < fields.length; ++i) {
+            storedKeys[i] = i;
+            storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
+        }
+        comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
+        tpc = tpcf.createPartitioner();
+        this.aggregatorFactory = aggregatorFactory;
+        this.inRecordDescriptor = inRecordDescriptor;
+        this.outRecordDescriptor = outRecordDescriptor;
+        RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
+        storedKeysAccessor = new FrameTupleAccessor(ctx, storedKeysRecordDescriptor);
+        lastBIndex = -1;
+        addNewBuffer();
+    }
+
+    private void addNewBuffer() {
+        ByteBuffer buffer = ctx.getResourceManager().allocateFrame();
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        buffers.add(buffer);
+        appender.reset(buffer, true);
+        ++lastBIndex;
+    }
+
+    private void flushFrame(FrameTupleAppender appender, IFrameWriter writer) throws HyracksDataException {
+        ByteBuffer frame = appender.getBuffer();
+        frame.position(0);
+        frame.limit(frame.capacity());
+        writer.nextFrame(appender.getBuffer());
+        appender.reset(appender.getBuffer(), true);
+    }
+
+    void insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+        int entry = tpc.partition(accessor, tIndex, table.length);
+        Link link = table[entry];
+        if (link == null) {
+            link = table[entry] = new Link();
+        }
+        IAccumulatingAggregator aggregator = null;
+        for (int i = 0; i < link.size; i += 3) {
+            int sbIndex = link.pointers[i];
+            int stIndex = link.pointers[i + 1];
+            int saIndex = link.pointers[i + 2];
+            storedKeysAccessor.reset(buffers.get(sbIndex));
+            int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
+            if (c == 0) {
+                aggregator = accumulators[saIndex];
+                break;
+            }
+        }
+        if (aggregator == null) {
+            // Did not find the key. Insert a new entry.
+            if (!appender.appendProjection(accessor, tIndex, fields)) {
+                addNewBuffer();
+                if (!appender.appendProjection(accessor, tIndex, fields)) {
+                    throw new IllegalStateException();
+                }
+            }
+            int sbIndex = lastBIndex;
+            int stIndex = appender.getTupleCount() - 1;
+            if (accumulatorSize >= accumulators.length) {
+                accumulators = Arrays.copyOf(accumulators, accumulators.length * 2);
+            }
+            int saIndex = accumulatorSize++;
+            aggregator = accumulators[saIndex] = aggregatorFactory.createAggregator(inRecordDescriptor,
+                outRecordDescriptor);
+            aggregator.init(accessor, tIndex);
+            link.add(sbIndex, stIndex, saIndex);
+        }
+        aggregator.accumulate(accessor, tIndex);
+    }
+
+    void write(IFrameWriter writer) throws HyracksDataException {
+        ByteBuffer buffer = ctx.getResourceManager().allocateFrame();
+        appender.reset(buffer, true);
+        for (int i = 0; i < table.length; ++i) {
+            Link link = table[i];
+            if (link != null) {
+                for (int j = 0; j < link.size; j += 3) {
+                    int bIndex = link.pointers[j];
+                    int tIndex = link.pointers[j + 1];
+                    int aIndex = link.pointers[j + 2];
+                    ByteBuffer keyBuffer = buffers.get(bIndex);
+                    storedKeysAccessor.reset(keyBuffer);
+                    IAccumulatingAggregator aggregator = accumulators[aIndex];
+                    while (!aggregator.output(appender, storedKeysAccessor, tIndex)) {
+                        flushFrame(appender, writer);
+                    }
+                }
+            }
+        }
+        if (appender.getTupleCount() != 0) {
+            flushFrame(appender, writer);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
new file mode 100644
index 0000000..2d5d852
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
@@ -0,0 +1,159 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class HashGroupOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String HASHTABLE = "hashtable";
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] keys;
+    private final ITuplePartitionComputerFactory tpcf;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAccumulatingAggregatorFactory aggregatorFactory;
+    private final int tableSize;
+
+    public HashGroupOperatorDescriptor(JobSpecification spec, int[] keys, ITuplePartitionComputerFactory tpcf,
+            IBinaryComparatorFactory[] comparatorFactories, IAccumulatingAggregatorFactory aggregatorFactory,
+            RecordDescriptor recordDescriptor, int tableSize) {
+        super(spec, 1, 1);
+        this.keys = keys;
+        this.tpcf = tpcf;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        recordDescriptors[0] = recordDescriptor;
+        this.tableSize = tableSize;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        HashBuildActivity ha = new HashBuildActivity();
+        builder.addTask(ha);
+
+        OutputActivity oa = new OutputActivity();
+        builder.addTask(oa);
+
+        builder.addSourceEdge(0, ha, 0);
+        builder.addTargetEdge(0, oa, 0);
+        builder.addBlockingEdge(ha, oa);
+    }
+
+    private class HashBuildActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx,
+                    recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+            return new IOperatorNodePushable() {
+                private GroupingHashTable table;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    table = new GroupingHashTable(ctx, keys, comparatorFactories, tpcf, aggregatorFactory,
+                            recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0), recordDescriptors[0],
+                            tableSize);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    accessor.reset(buffer);
+                    int tupleCount = accessor.getTupleCount();
+                    for (int i = 0; i < tupleCount; ++i) {
+                        table.insert(accessor, i);
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(HASHTABLE, table);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return HashGroupOperatorDescriptor.this;
+        }
+    }
+
+    private class OutputActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            return new IOperatorNodePushable() {
+                private IFrameWriter writer;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    GroupingHashTable table = (GroupingHashTable) env.get(HASHTABLE);
+                    writer.open();
+                    table.write(writer);
+                    writer.close();
+                    env.set(HASHTABLE, null);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    // do nothing
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return HashGroupOperatorDescriptor.this;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregator.java
new file mode 100644
index 0000000..83ae684
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregator.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+
+public interface IAccumulatingAggregator {
+    /**
+     * Called once per aggregator before calling accumulate for the first time.
+     * 
+     * @param accessor
+     *            - Accessor to the data tuple.
+     * @param tIndex
+     *            - Index of the tuple in the accessor.
+     * @throws HyracksDataException
+     */
+    public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
+
+    /**
+     * Called once per tuple that belongs to this group.
+     * 
+     * @param accessor
+     *            - Accessor to data tuple.
+     * @param tIndex
+     *            - Index of tuple in the accessor.
+     * @throws HyracksDataException
+     */
+    public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
+
+    /**
+     * Called finally to emit output. This method is called until it returns true. The method is free to
+     * write out output to the provided appender until there is no more space and return false. It is the
+     * caller's responsibility to flush and make room in the appender before this method is called again.
+     * 
+     * @param appender
+     *            - Appender to write output to.
+     * @param accessor
+     *            - Accessor to access the key.
+     * @param tIndex
+     *            - Tuple index of the key in the accessor.
+     * @return true if all output is written, false if the appender is full.
+     * @throws HyracksDataException
+     */
+    public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex)
+            throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
new file mode 100644
index 0000000..d745eff
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public interface IAccumulatingAggregatorFactory extends Serializable {
+    IAccumulatingAggregator createAggregator(RecordDescriptor inRecordDesc, RecordDescriptor outRecordDescriptor);
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IGroupAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IGroupAggregator.java
new file mode 100644
index 0000000..7e2ba4b
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IGroupAggregator.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IGroupAggregator extends Serializable {
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException;
+
+    public void close() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperator.java
new file mode 100644
index 0000000..637d8f0
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperator.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+
+public class PreclusteredGroupOperator implements IOpenableDataWriterOperator {
+    private final int[] groupFields;
+
+    private final IComparator[] comparators;
+
+    private final IGroupAggregator aggregator;
+
+    private Object[] lastData;
+
+    private IOpenableDataWriter<Object[]> writer;
+
+    private List<Object[]> buffer;
+
+    private IOpenableDataReader<Object[]> reader;
+
+    public PreclusteredGroupOperator(int[] groupFields, IComparator[] comparators, IGroupAggregator aggregator) {
+        this.groupFields = groupFields;
+        this.comparators = comparators;
+        this.aggregator = aggregator;
+        buffer = new ArrayList<Object[]>();
+        reader = new IOpenableDataReader<Object[]>() {
+            private int idx;
+
+            @Override
+            public void open() {
+                idx = 0;
+            }
+
+            @Override
+            public void close() {
+            }
+
+            @Override
+            public Object[] readData() {
+                return idx >= buffer.size() ? null : buffer.get(idx++);
+            }
+        };
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (!buffer.isEmpty()) {
+            aggregate();
+        }
+        writer.close();
+    }
+
+    private void aggregate() throws HyracksDataException {
+        reader.open();
+        aggregator.aggregate(reader, writer);
+        reader.close();
+        buffer.clear();
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        lastData = null;
+        writer.open();
+    }
+
+    @Override
+    public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+        if (index != 0) {
+            throw new IllegalArgumentException();
+        }
+        this.writer = writer;
+    }
+
+    @Override
+    public void writeData(Object[] data) throws HyracksDataException {
+        if (lastData != null && compare(data, lastData) != 0) {
+            aggregate();
+        }
+        lastData = data;
+        buffer.add(data);
+    }
+
+    private int compare(Object[] d1, Object[] d2) {
+        for (int i = 0; i < groupFields.length; ++i) {
+            int fIdx = groupFields[i];
+            int c = comparators[i].compare(d1[fIdx], d2[fIdx]);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
new file mode 100644
index 0000000..d4cd57c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.group;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class PreclusteredGroupOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private final int[] groupFields;
+    private final IGroupAggregator aggregator;
+    private final IComparatorFactory[] comparatorFactories;
+
+    private static final long serialVersionUID = 1L;
+
+    public PreclusteredGroupOperatorDescriptor(JobSpecification spec, int[] groupFields,
+            IComparatorFactory[] comparatorFactories, IGroupAggregator aggregator, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.groupFields = groupFields;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregator = aggregator;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        IComparator[] comparators = new IComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createComparator();
+        }
+        return new DeserializedOperatorNodePushable(ctx, new PreclusteredGroupOperator(groupFields, comparators,
+                aggregator), recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/AbstractHadoopFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/AbstractHadoopFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..ab20332
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/AbstractHadoopFileScanOperatorDescriptor.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop;
+
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.file.IRecordReader;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.HadoopFileSplit;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractHadoopFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected HadoopFileSplit[] splits;
+
+    public AbstractHadoopFileScanOperatorDescriptor(JobSpecification spec, HadoopFileSplit[] splits,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 0, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.splits = splits;
+    }
+
+    protected abstract IRecordReader createRecordReader(HadoopFileSplit fileSplit, RecordDescriptor desc)
+            throws Exception;
+
+    protected class FileScanOperator implements IOpenableDataWriterOperator {
+        private IOpenableDataWriter<Object[]> writer;
+        private int index;
+
+        FileScanOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IndexOutOfBoundsException("Invalid index: " + index);
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            HadoopFileSplit split = splits[index];
+            RecordDescriptor desc = recordDescriptors[0];
+            try {
+                IRecordReader reader = createRecordReader(split, desc);
+                if (desc == null) {
+                    desc = recordDescriptors[0];
+                }
+                writer.open();
+                try {
+                    while (true) {
+                        Object[] record = new Object[desc.getFields().length];
+                        if (!reader.read(record)) {
+                            break;
+                        }
+                        writer.writeData(record);
+                    }
+                } finally {
+                    reader.close();
+                    writer.close();
+                }
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // do nothing
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    protected Reporter createReporter() {
+        return new Reporter() {
+            @Override
+            public Counter getCounter(Enum<?> name) {
+                return null;
+            }
+
+            @Override
+            public Counter getCounter(String group, String name) {
+                return null;
+            }
+
+            @Override
+            public InputSplit getInputSplit() throws UnsupportedOperationException {
+                return null;
+            }
+
+            @Override
+            public void incrCounter(Enum<?> key, long amount) {
+
+            }
+
+            @Override
+            public void incrCounter(String group, String counter, long amount) {
+
+            }
+
+            @Override
+            public void progress() {
+
+            }
+
+            @Override
+            public void setStatus(String status) {
+
+            }
+        };
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new FileScanOperator(partition),
+                recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/AbstractHadoopOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/AbstractHadoopOperatorDescriptor.java
new file mode 100644
index 0000000..0553090
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/AbstractHadoopOperatorDescriptor.java
@@ -0,0 +1,172 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+import edu.uci.ics.dcache.client.DCacheClient;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.IHadoopClassFactory;
+
+public abstract class AbstractHadoopOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+	
+	protected static class DataWritingOutputCollector<K, V> implements OutputCollector<K, V> {
+        private IDataWriter<Object[]> writer;
+
+        public DataWritingOutputCollector() {
+        }
+
+        public DataWritingOutputCollector(IDataWriter<Object[]> writer) {
+            this.writer = writer;
+        }
+
+        @Override
+        public void collect(Object key, Object value) throws IOException {
+           	writer.writeData(new Object[] { key, value });
+        }
+
+        public void setWriter(IDataWriter<Object[]> writer) {
+            this.writer = writer;
+        }
+    }
+
+    public static String MAPRED_CACHE_FILES = "mapred.cache.files";
+    public static String MAPRED_CACHE_LOCALFILES = "mapred.cache.localFiles";
+
+    private static final long serialVersionUID = 1L;
+    private final HashMap<String, String> jobConfMap;
+    private IHadoopClassFactory hadoopClassFactory;
+    
+    public abstract RecordDescriptor getRecordDescriptor(JobConf jobConf);
+    	
+    public AbstractHadoopOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor, JobConf jobConf,
+            IHadoopClassFactory hadoopOperatorFactory) {
+        super(spec, 1, 1);
+        jobConfMap = DatatypeHelper.jobConf2HashMap(jobConf);
+        this.hadoopClassFactory = hadoopOperatorFactory;
+        if(recordDescriptor != null){
+        	recordDescriptors[0]= recordDescriptor;
+        }else{
+        	recordDescriptors[0] = getRecordDescriptor(jobConf);
+        }
+    }
+
+     
+    public HashMap<String, String> getJobConfMap() {
+		return jobConfMap;
+	}
+
+
+	public IHadoopClassFactory getHadoopClassFactory() {
+		return hadoopClassFactory;
+	}
+
+
+	public void setHadoopClassFactory(IHadoopClassFactory hadoopClassFactory) {
+		this.hadoopClassFactory = hadoopClassFactory;
+	}
+
+
+	protected Reporter createReporter() {
+        return new Reporter() {
+            @Override
+            public Counter getCounter(Enum<?> name) {
+                return null;
+            }
+
+            @Override
+            public Counter getCounter(String group, String name) {
+                return null;
+            }    
+              
+
+            @Override
+            public InputSplit getInputSplit() throws UnsupportedOperationException {
+                return null;
+            }
+
+            @Override
+            public void incrCounter(Enum<?> key, long amount) {
+
+            }
+
+            @Override
+            public void incrCounter(String group, String counter, long amount) {
+
+            }
+
+            @Override
+            public void progress() {
+
+            }
+
+            @Override
+            public void setStatus(String status) {
+
+            }
+        };
+    }
+
+    public JobConf getJobConf() {
+        return DatatypeHelper.hashMap2JobConf(jobConfMap);
+    }
+    
+    public void populateCache(JobConf jobConf) {
+        String cache = jobConf.get(MAPRED_CACHE_FILES);
+        System.out.println("cache:" + cache);
+        if (cache == null) {
+            return;
+        }
+        String localCache = jobConf.get(MAPRED_CACHE_LOCALFILES);
+        System.out.println("localCache:" + localCache);
+        if (localCache != null) {
+            return;
+        }
+        localCache = "";
+        StringTokenizer cacheTokenizer = new StringTokenizer(cache, ",");
+        while (cacheTokenizer.hasMoreTokens()) {
+            if (!"".equals(localCache)) {
+                localCache += ",";
+            }
+            try {
+                localCache += DCacheClient.get().get(cacheTokenizer.nextToken());
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        jobConf.set(MAPRED_CACHE_LOCALFILES, localCache);
+        System.out.println("localCache:" + localCache);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HDFSWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..8357534
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,176 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.AbstractFileWriteOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IRecordWriter;
+import edu.uci.ics.hyracks.dataflow.std.file.RecordWriter;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.DatatypeHelper;
+
+
+
+public class HDFSWriteOperatorDescriptor extends
+		AbstractFileWriteOperatorDescriptor {
+	
+	private static String nullWritableClassName = NullWritable.class.getName();
+	
+	private static class HDFSLineWriterImpl extends RecordWriter {
+		
+        HDFSLineWriterImpl(FileSystem fileSystem, String hdfsPath, int[] columns, char separator)
+                throws Exception {
+    		super(columns,separator,new Object[]{fileSystem,hdfsPath});
+        }
+
+		@Override
+		public OutputStream createOutputStream(Object[] args) throws Exception {
+			FSDataOutputStream fs = ((FileSystem)args[0]).create(new Path((String)args[1]));
+			return fs;
+		}
+
+		 @Override
+	     public void write(Object[] record) throws Exception {
+	         if(!nullWritableClassName.equals((record[0].getClass().getName()))){
+	             bufferedWriter.write(String.valueOf(record[0]));
+	         }
+	         if(!nullWritableClassName.equals((record[1].getClass().getName()))){
+	        	  bufferedWriter.write(separator);	 
+	        	  bufferedWriter.write(String.valueOf(record[1]));
+	         }	 
+	         bufferedWriter.write("\n");
+	     }
+    }
+
+	private static class HDFSSequenceWriterImpl extends RecordWriter {
+		
+		private Writer writer;
+		
+		HDFSSequenceWriterImpl(FileSystem fileSystem, String hdfsPath, Writer writer)
+                throws Exception {
+    		super(null,COMMA,new Object[]{fileSystem,hdfsPath});
+    		this.writer = writer;
+        }
+
+		@Override
+		public OutputStream createOutputStream(Object[] args) throws Exception {
+			return null;
+		}
+		
+		@Override
+	     public void close() {
+	         try {
+	             writer.close();
+	         } catch (IOException e) {
+	             e.printStackTrace();
+	         }
+	     }
+
+	     @Override
+	     public void write(Object[] record) throws Exception {
+	         Object key = record[0];
+	         Object value = record[1];
+	         writer.append(key, value);
+	     }
+
+    }
+	
+    private static final long serialVersionUID = 1L;
+    private static final char COMMA = ',';
+	private char separator;
+	private boolean sequenceFileOutput = false;
+	private String keyClassName;
+	private String valueClassName;
+	Map<String,String> jobConfMap;
+    
+
+    @Override
+    protected IRecordWriter createRecordWriter(File file,int index) throws Exception {
+    	JobConf conf = DatatypeHelper.hashMap2JobConf((HashMap)jobConfMap);
+		System.out.println("replication:" + conf.get("dfs.replication"));
+    	FileSystem fileSystem = null;
+		try{
+			fileSystem = FileSystem.get(conf);
+		}catch(IOException ioe){
+			ioe.printStackTrace();
+		}
+		Path path = new Path(file.getAbsolutePath());
+		checkIfCanWriteToHDFS(new FileSplit[]{new FileSplit("localhost",file)});
+		FSDataOutputStream outputStream = fileSystem.create(path);
+		outputStream.close();
+		if(sequenceFileOutput){
+			Class  keyClass = Class.forName(keyClassName);  
+			Class valueClass= Class.forName(valueClassName);
+			conf.setClass("mapred.output.compression.codec", GzipCodec.class, CompressionCodec.class);
+			Writer writer = SequenceFile.createWriter(fileSystem, conf,path, keyClass, valueClass);
+			return new HDFSSequenceWriterImpl(fileSystem, file.getAbsolutePath(), writer);
+		}else{
+			return new HDFSLineWriterImpl(fileSystem, file.getAbsolutePath(), null, COMMA);
+		}	
+    }
+    
+    private boolean checkIfCanWriteToHDFS(FileSplit[] fileSplits) throws Exception{
+    	boolean canWrite = true;
+    	JobConf conf = DatatypeHelper.hashMap2JobConf((HashMap)jobConfMap);
+		FileSystem fileSystem = null;
+		try{
+			fileSystem = FileSystem.get(conf);
+		    for(FileSplit fileSplit : fileSplits){
+				Path path = new Path(fileSplit.getLocalFile().getAbsolutePath());
+				canWrite = !fileSystem.exists(path);
+				if(!canWrite){
+					throw new Exception(" Output path :  already exists");
+				}	
+			}
+		}catch(IOException ioe){
+			ioe.printStackTrace();
+			throw ioe;
+		}
+	    return canWrite;
+    }
+		
+	
+	public HDFSWriteOperatorDescriptor(JobSpecification jobSpec,Map<String,String> jobConfMap, FileSplit[] fileSplits,char seperator,boolean sequenceFileOutput,String keyClassName, String valueClassName) throws Exception{
+		super(jobSpec,fileSplits);
+		this.jobConfMap = jobConfMap;
+		checkIfCanWriteToHDFS(fileSplits);
+		this.separator = seperator;
+		this.sequenceFileOutput = sequenceFileOutput;
+		this.keyClassName = keyClassName;
+		this.valueClassName = valueClassName;
+	}
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopMapperOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopMapperOperatorDescriptor.java
new file mode 100644
index 0000000..6f982ed
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopMapperOperatorDescriptor.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.IHadoopClassFactory;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class HadoopMapperOperatorDescriptor<K1, V1, K2, V2> extends AbstractHadoopOperatorDescriptor {
+    private class MapperOperator implements IOpenableDataWriterOperator {
+        private OutputCollector<K2, V2> output;
+        private Reporter reporter;
+        private Mapper<K1, V1, K2, V2> mapper;
+        private IOpenableDataWriter<Object[]> writer;
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                mapper.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+            writer.close();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            JobConf jobConf = getJobConf();
+            populateCache(jobConf);
+            try {
+                mapper = createMapper();
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+            // -- - configure - --
+            mapper.configure(jobConf);
+            writer.open();
+            output = new DataWritingOutputCollector<K2, V2>(writer);
+            reporter = createReporter();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IllegalArgumentException();
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            try {
+                mapper.map((K1) data[0], (V1) data[1], output, reporter);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+    private static final String mapClassNameKey = "mapred.mapper.class";
+    private Class<? extends Mapper> mapperClass;
+
+    public HadoopMapperOperatorDescriptor(JobSpecification spec, Class<? extends Mapper> mapperClass,
+            RecordDescriptor recordDescriptor, JobConf jobConf) {
+        super(spec, recordDescriptor, jobConf, null);
+        this.mapperClass = mapperClass;
+    }
+
+    public HadoopMapperOperatorDescriptor(JobSpecification spec, JobConf jobConf, IHadoopClassFactory hadoopClassFactory) {
+        super(spec, null, jobConf, hadoopClassFactory);
+    }
+
+    public RecordDescriptor getRecordDescriptor(JobConf conf) {
+        RecordDescriptor recordDescriptor = null;
+        String mapOutputKeyClassName = conf.getMapOutputKeyClass().getName();
+        String mapOutputValueClassName = conf.getMapOutputValueClass().getName();
+        try {
+            if (getHadoopClassFactory() == null) {
+                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
+                        (Class<? extends Writable>) Class.forName(mapOutputKeyClassName),
+                        (Class<? extends Writable>) Class.forName(mapOutputValueClassName));
+            } else {
+                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
+                        (Class<? extends Writable>) getHadoopClassFactory().loadClass(mapOutputKeyClassName),
+                        (Class<? extends Writable>) getHadoopClassFactory().loadClass(mapOutputValueClassName));
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        return recordDescriptor;
+    }
+
+    private Mapper<K1, V1, K2, V2> createMapper() throws Exception {
+        if (mapperClass != null) {
+            return mapperClass.newInstance();
+        } else {
+            String mapperClassName = super.getJobConfMap().get(mapClassNameKey);
+            Object mapper = getHadoopClassFactory().createMapper(mapperClassName);
+            mapperClass = (Class<? extends Mapper>) mapper.getClass();
+            return (Mapper) mapper;
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
+                recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+    }
+
+    public Class<? extends Mapper> getMapperClass() {
+        return mapperClass;
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopReadOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopReadOperatorDescriptor.java
new file mode 100644
index 0000000..3aa5c1a
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopReadOperatorDescriptor.java
@@ -0,0 +1,236 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.NetworkInterface;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileRecordReader;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IRecordReader;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.HadoopAdapter;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.HadoopFileSplit;
+
+public class HadoopReadOperatorDescriptor extends AbstractHadoopFileScanOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private String inputFormatClassName;
+    private Map<String, String> jobConfMap;
+
+    private static class HDFSCustomReader implements IRecordReader {
+        private RecordReader hadoopRecordReader;
+        private Class inputKeyClass;
+        private Class inputValueClass;
+        private Object key;
+        private Object value;
+
+        public HDFSCustomReader(Map<String, String> jobConfMap, HadoopFileSplit inputSplit,
+                String inputFormatClassName, Reporter reporter) {
+            try {
+                JobConf conf = DatatypeHelper.hashMap2JobConf((HashMap) jobConfMap);
+                FileSystem fileSystem = null;
+                try {
+                    fileSystem = FileSystem.get(conf);
+                } catch (IOException ioe) {
+                    ioe.printStackTrace();
+                }
+
+                Class inputFormatClass = Class.forName(inputFormatClassName);
+                InputFormat inputFormat = (InputFormat) ReflectionUtils.newInstance(inputFormatClass, conf);
+                hadoopRecordReader = (RecordReader) inputFormat.getRecordReader(getFileSplit(inputSplit), conf,
+                        reporter);
+                if (hadoopRecordReader instanceof SequenceFileRecordReader) {
+                    inputKeyClass = ((SequenceFileRecordReader) hadoopRecordReader).getKeyClass();
+                    inputValueClass = ((SequenceFileRecordReader) hadoopRecordReader).getValueClass();
+                } else {
+                    inputKeyClass = hadoopRecordReader.createKey().getClass();
+                    inputValueClass = hadoopRecordReader.createValue().getClass();
+                }
+                key = inputKeyClass.newInstance();
+                value = inputValueClass.newInstance();
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+
+        public Class getInputKeyClass() {
+            return inputKeyClass;
+        }
+
+        public void setInputKeyClass(Class inputKeyClass) {
+            this.inputKeyClass = inputKeyClass;
+        }
+
+        public Class getInputValueClass() {
+            return inputValueClass;
+        }
+
+        public void setInputValueClass(Class inputValueClass) {
+            this.inputValueClass = inputValueClass;
+        }
+
+        @Override
+        public void close() {
+            try {
+                hadoopRecordReader.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            if (!hadoopRecordReader.next(key, value)) {
+                return false;
+            }
+            if (record.length == 1) {
+                record[0] = value;
+            } else {
+                record[0] = key;
+                record[1] = value;
+            }
+            return true;
+        }
+
+        private FileSplit getFileSplit(HadoopFileSplit hadoopFileSplit) {
+            FileSplit fileSplit = new FileSplit(new Path(hadoopFileSplit.getFile()), hadoopFileSplit.getStart(),
+                    hadoopFileSplit.getLength(), hadoopFileSplit.getHosts());
+            return fileSplit;
+        }
+    }
+
+    public HadoopReadOperatorDescriptor(Map<String, String> jobConfMap, JobSpecification spec,
+            HadoopFileSplit[] splits, String inputFormatClassName, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+        this.inputFormatClassName = inputFormatClassName;
+        this.jobConfMap = jobConfMap;
+    }
+
+    public HadoopReadOperatorDescriptor(Map<String, String> jobConfMap, InetSocketAddress nameNode,
+            JobSpecification spec, String inputFormatClassName, RecordDescriptor recordDescriptor) {
+        super(spec, null, recordDescriptor);
+        this.inputFormatClassName = inputFormatClassName;
+        this.jobConfMap = jobConfMap;
+    }
+
+    // public HadoopReadOperatorDescriptor(IClusterController clusterController, Map<String, String> jobConfMap,
+    // JobSpecification spec, String fileSystemURL, String inputFormatClassName, RecordDescriptor recordDescriptor) {
+    // super(spec, null, recordDescriptor);
+    // HadoopAdapter hadoopAdapter = HadoopAdapter.getInstance(fileSystemURL);
+    // String inputPathString = jobConfMap.get("mapred.input.dir");
+    // String[] inputPaths = inputPathString.split(",");
+    // Map<String, List<HadoopFileSplit>> blocksToRead = hadoopAdapter.getInputSplits(inputPaths);
+    // List<HadoopFileSplit> hadoopFileSplits = new ArrayList<HadoopFileSplit>();
+    // for (String filePath : blocksToRead.keySet()) {
+    // hadoopFileSplits.addAll(blocksToRead.get(filePath));
+    // }
+    // for (HadoopFileSplit hadoopFileSplit : hadoopFileSplits) {
+    // System.out.println(" Hadoop File Split : " + hadoopFileSplit);
+    // }
+    // super.splits = hadoopFileSplits.toArray(new HadoopFileSplit[] {});
+    // configurePartitionConstraints(clusterController, blocksToRead);
+    // this.inputFormatClassName = inputFormatClassName;
+    // this.jobConfMap = jobConfMap;
+    // }
+
+    // private void configurePartitionConstraints(IClusterController clusterController,
+    // Map<String, List<HadoopFileSplit>> blocksToRead) {
+    // List<LocationConstraint> locationConstraints = new ArrayList<LocationConstraint>();
+    // Map<String, INodeController> registry = null;
+    // try {
+    // // registry = clusterController.getRegistry();
+    // // TODO
+    // } catch (Exception e) {
+    // e.printStackTrace();
+    // }
+    // Map<String, String> hostnameToNodeIdMap = new HashMap<String, String>();
+    // NCConfig ncConfig = null;
+    // for (String nodeId : registry.keySet()) {
+    // try {
+    // ncConfig = ((INodeController) registry.get(nodeId)).getConfiguration();
+    // String ipAddress = ncConfig.dataIPAddress;
+    // String hostname = InetAddress.getByName(ipAddress).getHostName();
+    // System.out.println(" hostname :" + hostname + " nodeid:" + nodeId);
+    // hostnameToNodeIdMap.put(hostname, nodeId);
+    // } catch (Exception e) {
+    // e.printStackTrace();
+    // }
+    // }
+    //
+    // for (String filePath : blocksToRead.keySet()) {
+    // List<HadoopFileSplit> hadoopFileSplits = blocksToRead.get(filePath);
+    // for (HadoopFileSplit hadoopFileSplit : hadoopFileSplits) {
+    // String hostname = hadoopFileSplit.getHosts()[0];
+    // System.out.println("host name is :" + hostname);
+    // InetAddress address = null;
+    // try {
+    // address = InetAddress.getByName(hostname);
+    // if (address.isLoopbackAddress()) {
+    // Enumeration<NetworkInterface> netInterfaces = NetworkInterface.getNetworkInterfaces();
+    // while (netInterfaces.hasMoreElements()) {
+    // NetworkInterface ni = netInterfaces.nextElement();
+    // InetAddress inetAddress = (InetAddress) ni.getInetAddresses().nextElement();
+    // if (!inetAddress.isLoopbackAddress()) {
+    // address = inetAddress;
+    // break;
+    // }
+    // }
+    // }
+    // hostname = address.getHostName();
+    // System.out.println("cannonical host name hyracks :" + hostname);
+    // } catch (Exception e) {
+    // e.printStackTrace();
+    // }
+    // String nodeId = hostnameToNodeIdMap.get(hostname);
+    // System.out.println(" corresponding node id is :" + nodeId);
+    // LocationConstraint locationConstraint = new AbsoluteLocationConstraint(nodeId);
+    // locationConstraints.add(locationConstraint);
+    // }
+    // }
+    //
+    // PartitionConstraint partitionConstraint = new ExplicitPartitionConstraint(locationConstraints
+    // .toArray(new LocationConstraint[] {}));
+    // this.setPartitionConstraint(partitionConstraint);
+    // }
+
+    @Override
+    protected IRecordReader createRecordReader(HadoopFileSplit fileSplit, RecordDescriptor desc) throws Exception {
+        Reporter reporter = createReporter();
+        IRecordReader recordReader = new HDFSCustomReader(jobConfMap, fileSplit, inputFormatClassName, reporter);
+        return recordReader;
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopReducerOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopReducerOperatorDescriptor.java
new file mode 100644
index 0000000..d17af31
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/HadoopReducerOperatorDescriptor.java
@@ -0,0 +1,267 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.group.IGroupAggregator;
+import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperator;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.data.KeyComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.data.WritableComparingComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.ClasspathBasedHadoopClassFactory;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.dataflow.std.hadoop.util.IHadoopClassFactory;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class HadoopReducerOperatorDescriptor<K2, V2, K3, V3> extends AbstractHadoopOperatorDescriptor {
+    private class ReducerAggregator implements IGroupAggregator {
+        private Reducer<K2, V2, K3, V3> reducer;
+        private DataWritingOutputCollector<K3, V3> output;
+        private Reporter reporter;
+
+        public ReducerAggregator(Reducer<K2, V2, K3, V3> reducer) {
+            this.reducer = reducer;
+            reducer.configure(getJobConf());
+            output = new DataWritingOutputCollector<K3, V3>();
+            reporter = new Reporter() {
+
+                @Override
+                public void progress() {
+
+                }
+
+                @Override
+                public void setStatus(String arg0) {
+
+                }
+
+                @Override
+                public void incrCounter(String arg0, String arg1, long arg2) {
+
+                }
+
+                @Override
+                public void incrCounter(Enum<?> arg0, long arg1) {
+
+                }
+
+                @Override
+                public InputSplit getInputSplit() throws UnsupportedOperationException {
+                    return null;
+                }
+
+                @Override
+                public Counter getCounter(String arg0, String arg1) {
+                    return null;
+                }
+
+                @Override
+                public Counter getCounter(Enum<?> arg0) {
+                    return null;
+                }
+            };
+        }
+
+        @Override
+        public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException {
+
+            ValueIterator i = new ValueIterator();
+            i.reset(reader);
+            output.setWriter(writer);
+            try {
+
+                // -- - reduce - --
+                reducer.reduce(i.getKey(), i, output, reporter);
+
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // -- - close - --
+            try {
+                reducer.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private class ValueIterator implements Iterator<V2> {
+        private IDataReader<Object[]> reader;
+        private K2 key;
+        private V2 value;
+
+        public K2 getKey() {
+            return key;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (value == null) {
+                Object[] tuple;
+                try {
+                    tuple = reader.readData();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                if (tuple != null) {
+                    value = (V2) tuple[1];
+                }
+            }
+            return value != null;
+        }
+
+        @Override
+        public V2 next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            V2 v = value;
+            value = null;
+            return v;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+        void reset(IDataReader<Object[]> reader) {
+            this.reader = reader;
+            try {
+                Object[] tuple = reader.readData();
+                key = (K2) tuple[0];
+                value = (V2) tuple[1];
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+    private Class<? extends Reducer> reducerClass;
+    private static final String reducerClassKey = "mapred.reducer.class";
+    private static final String comparatorClassKey = "mapred.output.value.groupfn.class";
+    private IComparatorFactory comparatorFactory;
+
+    public HadoopReducerOperatorDescriptor(JobSpecification spec, IComparatorFactory comparatorFactory,
+            Class<? extends Reducer> reducerClass, RecordDescriptor recordDescriptor, JobConf jobConf) {
+        super(spec, recordDescriptor, jobConf, new ClasspathBasedHadoopClassFactory());
+        this.comparatorFactory = comparatorFactory;
+        this.reducerClass = reducerClass;
+    }
+
+    public HadoopReducerOperatorDescriptor(JobSpecification spec, JobConf conf, IHadoopClassFactory classFactory) {
+        super(spec, null, conf, classFactory);
+    }
+
+    private Reducer<K2, V2, K3, V3> createReducer() throws Exception {
+        if (reducerClass != null) {
+            return reducerClass.newInstance();
+        } else {
+            Object reducer = getHadoopClassFactory().createReducer(getJobConfMap().get(reducerClassKey));
+            reducerClass = (Class<? extends Reducer>) reducer.getClass();
+            return (Reducer) reducer;
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        try {
+            if (this.comparatorFactory == null) {
+                String comparatorClassName = getJobConfMap().get(comparatorClassKey);
+                RawComparator rawComparator = null;
+                if (comparatorClassName != null) {
+                    Class comparatorClazz = getHadoopClassFactory().loadClass(comparatorClassName);
+                    this.comparatorFactory = new KeyComparatorFactory(comparatorClazz);
+
+                } else {
+                    String mapOutputKeyClass = getJobConfMap().get("mapred.mapoutput.key.class");
+                    if (getHadoopClassFactory() != null) {
+                        rawComparator = WritableComparator.get(getHadoopClassFactory().loadClass(mapOutputKeyClass));
+                    } else {
+                        rawComparator = WritableComparator.get((Class<? extends WritableComparable>) Class
+                                .forName(mapOutputKeyClass));
+                    }
+                    this.comparatorFactory = new WritableComparingComparatorFactory(rawComparator.getClass());
+                }
+            }
+            IOpenableDataWriterOperator op = new PreclusteredGroupOperator(new int[] { 0 },
+                    new IComparator[] { comparatorFactory.createComparator() }, new ReducerAggregator(createReducer()));
+            return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
+                    getOperatorId(), 0));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Class<? extends Reducer> getReducerClass() {
+        return reducerClass;
+    }
+
+    public void setReducerClass(Class<? extends Reducer> reducerClass) {
+        this.reducerClass = reducerClass;
+    }
+
+    @Override
+    public RecordDescriptor getRecordDescriptor(JobConf conf) {
+        String outputKeyClassName = conf.get("mapred.output.key.class");
+        String outputValueClassName = conf.get("mapred.output.value.class");
+        RecordDescriptor recordDescriptor = null;
+        try {
+            if (getHadoopClassFactory() == null) {
+                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
+                        (Class<? extends Writable>) Class.forName(outputKeyClassName),
+                        (Class<? extends Writable>) Class.forName(outputValueClassName));
+            } else {
+                recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
+                        (Class<? extends Writable>) getHadoopClassFactory().loadClass(outputKeyClassName),
+                        (Class<? extends Writable>) getHadoopClassFactory().loadClass(outputValueClassName));
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            return null;
+        }
+        return recordDescriptor;
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/AbstractClassBasedDelegate.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/AbstractClassBasedDelegate.java
new file mode 100644
index 0000000..813bd0a
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/AbstractClassBasedDelegate.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+
+public class AbstractClassBasedDelegate<T> implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private Class<? extends T> klass;
+    protected transient T instance;
+
+    public AbstractClassBasedDelegate(Class<? extends T> klass) {
+        this.klass = klass;
+        init();
+    }
+
+    protected Object readResolve() throws ObjectStreamException {
+        init();
+        return this;
+    }
+
+    private void init() {
+        try {
+            instance = klass.newInstance();
+        } catch (InstantiationException e) {
+            throw new RuntimeException(e);
+        } catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/HadoopHashTuplePartitionComputerFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
new file mode 100644
index 0000000..717c1e3
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.util.ByteBufferInputStream;
+
+public class HadoopHashTuplePartitionComputerFactory<K extends Writable> implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final ISerializerDeserializer<K> keyIO;
+
+    public HadoopHashTuplePartitionComputerFactory(ISerializerDeserializer<K> keyIO) {
+        this.keyIO = keyIO;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        return new ITuplePartitionComputer() {
+            private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+            private final DataInputStream dis = new DataInputStream(bbis);
+
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                        + accessor.getFieldStartOffset(tIndex, 0);
+                bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+                K key = keyIO.deserialize(dis);
+                int h = key.hashCode();
+                if (h < 0) {
+                    h = -h;
+                }
+                return h % nParts;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
new file mode 100644
index 0000000..9d0e17b
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Partitioner;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.util.ByteBufferInputStream;
+
+public class HadoopPartitionerTuplePartitionComputerFactory<K extends Writable, V extends Writable> extends
+        AbstractClassBasedDelegate<Partitioner<K, V>> implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final ISerializerDeserializer<K> keyIO;
+    private final ISerializerDeserializer<V> valueIO;
+
+    public HadoopPartitionerTuplePartitionComputerFactory(Class<? extends Partitioner<K, V>> klass,
+            ISerializerDeserializer<K> keyIO, ISerializerDeserializer<V> valueIO) {
+        super(klass);
+        this.keyIO = keyIO;
+        this.valueIO = valueIO;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        return new ITuplePartitionComputer() {
+            private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+            private final DataInputStream dis = new DataInputStream(bbis);
+
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                        + accessor.getFieldStartOffset(tIndex, 0);
+                bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+                K key = keyIO.deserialize(dis);
+                int valueStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                        + accessor.getFieldStartOffset(tIndex, 1);
+                bbis.setByteBuffer(accessor.getBuffer(), valueStart);
+                V value = valueIO.deserialize(dis);
+                return instance.getPartition(key, value, nParts);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/KeyBinaryComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/KeyBinaryComparatorFactory.java
new file mode 100644
index 0000000..cbe7b18
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/KeyBinaryComparatorFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import org.apache.hadoop.io.RawComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class KeyBinaryComparatorFactory<T> implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private Class<? extends RawComparator<T>> cmpClass;
+
+    public KeyBinaryComparatorFactory(Class<? extends RawComparator<T>> cmpClass) {
+        this.cmpClass = cmpClass;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        final RawComparator<T> instance = ReflectionUtils.createInstance(cmpClass);
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return instance.compare(b1, s1, l1, b2, s2, l2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/KeyComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/KeyComparatorFactory.java
new file mode 100644
index 0000000..4d840e9
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/KeyComparatorFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import org.apache.hadoop.io.RawComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class KeyComparatorFactory<T> implements IComparatorFactory<T> {
+    private static final long serialVersionUID = 1L;
+    private Class<? extends RawComparator<T>> cmpClass;
+
+    public KeyComparatorFactory(Class<? extends RawComparator<T>> cmpClass) {
+        this.cmpClass = cmpClass;
+    }
+
+    @Override
+    public IComparator<T> createComparator() {
+        final RawComparator<T> instance = ReflectionUtils.createInstance(cmpClass);
+        return new IComparator<T>() {
+            @Override
+            public int compare(T o1, T o2) {
+                return instance.compare(o1, o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/WritableComparingBinaryComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/WritableComparingBinaryComparatorFactory.java
new file mode 100644
index 0000000..ade8389
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/WritableComparingBinaryComparatorFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import org.apache.hadoop.io.WritableComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class WritableComparingBinaryComparatorFactory<T> implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private Class<? extends WritableComparator> cmpClass;
+
+    public WritableComparingBinaryComparatorFactory(Class<? extends WritableComparator> cmpClass) {
+        this.cmpClass = cmpClass;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        final WritableComparator instance = ReflectionUtils.createInstance(cmpClass);
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return instance.compare(b1, s1, l1, b2, s2, l2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/WritableComparingComparatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/WritableComparingComparatorFactory.java
new file mode 100644
index 0000000..1fc8eb2
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/data/WritableComparingComparatorFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.data;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class WritableComparingComparatorFactory<T> implements IComparatorFactory<WritableComparable<T>> {
+    private Class<? extends WritableComparator> klass;
+
+    public WritableComparingComparatorFactory(Class<? extends WritableComparator> klass) {
+        this.klass = klass;
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IComparator<WritableComparable<T>> createComparator() {
+        final WritableComparator instance = ReflectionUtils.createInstance(klass);
+        return new IComparator<WritableComparable<T>>() {
+            @Override
+            public int compare(WritableComparable<T> o1, WritableComparable<T> o2) {
+                return instance.compare(o1, o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/ClasspathBasedHadoopClassFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/ClasspathBasedHadoopClassFactory.java
new file mode 100644
index 0000000..3a9833c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/ClasspathBasedHadoopClassFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
+
+public class ClasspathBasedHadoopClassFactory implements IHadoopClassFactory {
+
+	@Override
+	public Mapper createMapper(String mapClassName) throws Exception {
+		Class clazz = loadClass(mapClassName);
+		return (Mapper)clazz.newInstance();
+	}
+
+	@Override
+	public Reducer createReducer(String reduceClassName) throws Exception {
+		Class clazz = loadClass(reduceClassName);
+		return (Reducer)clazz.newInstance();
+	}
+
+	@Override
+	public Class loadClass(String className) throws Exception {
+		Class clazz = Class.forName(className);
+		return clazz;
+	}
+
+	
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/DatatypeHelper.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/DatatypeHelper.java
new file mode 100644
index 0000000..3097aaf
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/DatatypeHelper.java
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("deprecation")
+public class DatatypeHelper {
+    private static final class WritableSerializerDeserializer<T extends Writable> implements ISerializerDeserializer<T> {
+        private static final long serialVersionUID = 1L;
+
+        private Class<T> clazz;
+
+        private WritableSerializerDeserializer(Class<T> clazz) {
+            this.clazz = clazz;
+        }
+
+        private T createInstance() throws HyracksDataException {
+            // TODO remove "if", create a new WritableInstanceOperations class
+            // that deals with Writables that don't have public constructors
+            if (NullWritable.class.equals(clazz)) {
+                return (T) NullWritable.get();
+            }
+            try {
+                return clazz.newInstance();
+            } catch (InstantiationException e) {
+                throw new HyracksDataException(e);
+            } catch (IllegalAccessException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public T deserialize(DataInput in) throws HyracksDataException {
+            T o = createInstance();
+            try {
+                o.readFields(in);
+            } catch (IOException e) {
+                e.printStackTrace();
+                // throw new HyracksDataException(e);
+            }
+            return o;
+        }
+
+        @Override
+        public void serialize(T instance, DataOutput out) throws HyracksDataException {
+            try {
+                instance.write(out);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    public static ISerializerDeserializer<? extends Writable> createSerializerDeserializer(
+            Class<? extends Writable> fClass) {
+        return new WritableSerializerDeserializer(fClass);
+    }
+
+    public static RecordDescriptor createKeyValueRecordDescriptor(Class<? extends Writable> keyClass,
+            Class<? extends Writable> valueClass) {
+        ISerializerDeserializer[] fields = new ISerializerDeserializer[2];
+        fields[0] = createSerializerDeserializer(keyClass);
+        fields[1] = createSerializerDeserializer(valueClass);
+        return new RecordDescriptor(fields);
+    }
+
+    public static RecordDescriptor createOneFieldRecordDescriptor(Class<? extends Writable> fieldClass) {
+        ISerializerDeserializer[] fields = new ISerializerDeserializer[1];
+        fields[0] = createSerializerDeserializer(fieldClass);
+        return new RecordDescriptor(fields);
+    }
+
+    public static JobConf hashMap2JobConf(HashMap<String, String> jobConfMap) {
+        JobConf jobConf;
+        synchronized (Configuration.class) {
+            jobConf = new JobConf();
+            for (Entry<String, String> entry : jobConfMap.entrySet()) {
+                jobConf.set(entry.getKey(), entry.getValue());
+            }
+        }
+        return jobConf;
+    }
+
+    public static HashMap<String, String> jobConf2HashMap(JobConf jobConf) {
+        HashMap<String, String> jobConfMap = new HashMap<String, String>();
+        for (Entry<String, String> entry : jobConf) {
+            jobConfMap.put(entry.getKey(), entry.getValue());
+        }
+        return jobConfMap;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/DuplicateKeyMapper.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/DuplicateKeyMapper.java
new file mode 100644
index 0000000..08020e0
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/DuplicateKeyMapper.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import java.util.Properties;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.IMapper;
+
+public class DuplicateKeyMapper implements IMapper {
+
+    @Override
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+        writer.writeData(new Object[] { data[0], data[1], data[0] });
+
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/HadoopAdapter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/HadoopAdapter.java
new file mode 100644
index 0000000..b8c4911
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/HadoopAdapter.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.StringUtils;
+
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class HadoopAdapter {
+
+	private static ClientProtocol namenode;
+	private static FileSystem fileSystem;
+	private static JobConf jobConf;
+	private static HadoopAdapter instance;
+	
+	public static final String DFS_DATA_DIR = "dfs.data.dir";
+	public static final String FS_DEFAULT_NAME = "fs.default.name";
+	public static final String DFS_REPLICATION = "dfs.replication";
+	
+	public static HadoopAdapter getInstance(String fileSystemURL){
+		if(instance == null){
+			jobConf = new JobConf(true);
+			String [] urlParts = parseURL(fileSystemURL);
+			jobConf.set(FS_DEFAULT_NAME, fileSystemURL);
+			instance = new HadoopAdapter(new InetSocketAddress(urlParts[1], Integer.parseInt(urlParts[2])));
+		}
+		return instance;
+	}
+	
+	public static JobConf getConf() {
+		return jobConf;
+	}
+
+	private HadoopAdapter (InetSocketAddress address){
+		try{
+			this.namenode = getNameNode(address);
+			fileSystem = FileSystem.get(jobConf);
+		}catch(IOException ioe){
+			ioe.printStackTrace();
+		}
+	}
+	
+	private static String[] parseURL(String urlString){
+		String[] urlComponents = urlString.split(":");
+		urlComponents[1] = urlComponents[1].substring(2);
+		return urlComponents;
+	}
+	
+	
+	public Map<String,List<HadoopFileSplit>> getInputSplits(String[] inputPaths){
+	    List<HadoopFileSplit> hadoopFileSplits = new ArrayList<HadoopFileSplit>();
+    	Path[] paths = new Path[inputPaths.length];
+    	int index =0;
+    	for(String inputPath : inputPaths){
+    		paths[index++] = new Path(StringUtils.unEscapeString(inputPaths[0]));
+    	}
+    	Map<String,List<HadoopFileSplit>> fileSplitInfo = getBlocksToRead(paths);
+    	return fileSplitInfo;
+	}
+	
+	private static Map<String,List<HadoopFileSplit>> getBlocksToRead(Path[] inputPaths){
+		Map<String,List<HadoopFileSplit>> hadoopFileSplitMap = new HashMap<String,List<HadoopFileSplit>>();
+		for (int i=0;i<inputPaths.length;i++){
+			try{
+				String absolutePathPrint = getAbsolutePath(inputPaths[i]);
+				FileStatus[] fileStatuses = namenode.getListing(absolutePathPrint);
+				for(int j=0;j<fileStatuses.length;j++){
+			    	Path filePath = fileStatuses[j].getPath();
+			    	String absolutePath = getAbsolutePath(filePath);
+			    	List<HadoopFileSplit> fileSplits = getFileBlocks(absolutePath,fileStatuses[j]);
+			    	if(fileSplits!=null && fileSplits.size() > 0){
+			    		hadoopFileSplitMap.put(absolutePath, fileSplits);
+			    	}	
+			    }		
+			   }catch(IOException ioe){
+				ioe.printStackTrace();
+		    }
+			
+		}
+		return hadoopFileSplitMap;
+	}
+	
+	private static ClientProtocol getNameNode(InetSocketAddress address) throws IOException{
+		return (ClientProtocol)getProtocol(ClientProtocol.class, address, new JobConf());
+	}
+	
+	private static String getAbsolutePath(Path path){
+		StringBuffer absolutePath = new StringBuffer();
+		List<String> ancestorPath = new ArrayList<String>();
+		Path pathElement=path;
+		while(pathElement != null){
+			ancestorPath.add(0, pathElement.getName());
+			pathElement = pathElement.getParent();
+		}
+		ancestorPath.remove(0);
+		for(String s : ancestorPath){
+			absolutePath.append("/");
+			absolutePath.append(s);
+		}
+		return new String(absolutePath);
+	}
+	
+	private static VersionedProtocol getProtocol(Class protocolClass, InetSocketAddress inetAddress, JobConf jobConf) throws IOException{
+		VersionedProtocol versionedProtocol = RPC.getProxy(protocolClass, ClientProtocol.versionID, inetAddress, jobConf);	
+		return versionedProtocol;
+	}
+	
+	private static List<HadoopFileSplit> getFileBlocks(String absolutePath,FileStatus fileStatus){
+		List<HadoopFileSplit> hadoopFileSplits = new ArrayList<HadoopFileSplit>();
+		try{
+			LocatedBlocks locatedBlocks = namenode.getBlockLocations(absolutePath, 0, fileStatus.getLen());
+			long blockSize = fileSystem.getBlockSize(new Path(absolutePath));
+			if(locatedBlocks !=null){
+	    		int index = 0;
+				for(LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()){
+					DatanodeInfo[] datanodeInfos = locatedBlock.getLocations(); // all datanodes having this block
+					String [] hostnames = new String[datanodeInfos.length];
+					int datanodeCount =0;
+					for(DatanodeInfo datanodeInfo : datanodeInfos){
+						hostnames[datanodeCount++] = datanodeInfo.getHostName();
+					}	
+					HadoopFileSplit hadoopFileSplit = new HadoopFileSplit(absolutePath,new Long(index * blockSize).longValue(),new Long(blockSize).longValue(),hostnames);
+			    	hadoopFileSplits.add(hadoopFileSplit);
+			    	index++;
+			    	}
+	    	}	
+		}catch(Exception e){
+			e.printStackTrace();
+		}
+		return hadoopFileSplits;
+	}
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/HadoopFileSplit.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/HadoopFileSplit.java
new file mode 100644
index 0000000..2e0df99
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/HadoopFileSplit.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.fs.Path;
+
+public class HadoopFileSplit implements Serializable{
+	/**
+	 * 
+	 */
+	private static final long serialVersionUID = 1L;
+	private String filePath;
+    private long start;
+	private long length;
+    private String[] hosts;
+    
+    public HadoopFileSplit(String filePath, long start, long length, String[] hosts){
+    	this.filePath = filePath;
+    	this.start = start;
+    	this.length = length;
+    	this.hosts = hosts;
+    }
+
+	public String getFile() {
+		return filePath;
+	}
+
+	public void setFile(String file) {
+		this.filePath = file;
+	}
+
+	public long getStart() {
+		return start;
+	}
+
+	public void setStart(long start) {
+		this.start = start;
+	}
+
+	public long getLength() {
+		return length;
+	}
+
+	public void setLength(long length) {
+		this.length = length;
+	}
+
+	public String[] getHosts() {
+		return hosts;
+	}
+
+	public void setHosts(String[] hosts) {
+		this.hosts = hosts;
+	}
+	
+	public String toString(){
+		StringBuilder stringBuilder = new StringBuilder();
+		stringBuilder.append(filePath + " " + start + " " + length +  "\n");
+		for(String host : hosts){
+			stringBuilder.append(host);
+			stringBuilder.append(",");
+		}
+		return new String(stringBuilder);
+	}
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/IHadoopClassFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/IHadoopClassFactory.java
new file mode 100644
index 0000000..8502a55
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/IHadoopClassFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
+
+public interface IHadoopClassFactory extends Serializable{
+
+	public Mapper createMapper(String mapClassName) throws Exception;
+	
+	public Reducer createReducer(String reduceClassName) throws Exception;
+	
+	public Class loadClass(String className) throws Exception;
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/PreappendLongWritableMapper.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/PreappendLongWritableMapper.java
new file mode 100644
index 0000000..69573c8
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/hadoop/util/PreappendLongWritableMapper.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.hadoop.util;
+
+import java.util.Properties;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.IMapper;
+
+public class PreappendLongWritableMapper implements IMapper {
+
+    @Override
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+        writer.writeData(new Object[] { new LongWritable(0), new Text(String.valueOf(data[0])) });
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
new file mode 100644
index 0000000..42916f7
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -0,0 +1,348 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.RepartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class GraceHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String SMALLRELATION = "RelR";
+    private static final String LARGERELATION = "RelS";
+    private static final String NUM_PARTITION = "NUMBER_PARTITIONS";
+    private static final long serialVersionUID = 1L;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final int inputsize0;
+    private final int recordsPerFrame;
+    private final int memsize;
+    private final double factor;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    private int numReadI1 = 0;
+    private int numWrite = 0;
+    private int numReadI2 = 0;
+
+    public GraceHashJoinOperatorDescriptor(JobSpecification spec, int memsize, int inputsize0, int recordsPerFrame,
+            double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.recordsPerFrame = recordsPerFrame;
+        this.factor = factor;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        HashPartitionActivityNode rpart = new HashPartitionActivityNode(SMALLRELATION, keys0, 0);
+        HashPartitionActivityNode spart = new HashPartitionActivityNode(LARGERELATION, keys1, 1);
+        JoinActivityNode join = new JoinActivityNode();
+
+        builder.addTask(rpart);
+        builder.addSourceEdge(0, rpart, 0);
+
+        builder.addTask(spart);
+        builder.addSourceEdge(1, spart, 0);
+
+        builder.addTask(join);
+        builder.addBlockingEdge(rpart, spart);
+        builder.addBlockingEdge(spart, join);
+
+        builder.addTargetEdge(0, join, 0);
+    }
+
+    public int getMemorySize() {
+        return memsize;
+    }
+
+    private class HashPartitionActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+        private String relationName;
+        private int operatorInputIndex;
+        private int keys[];
+
+        public HashPartitionActivityNode(String relationName, int keys[], int operatorInputIndex) {
+            this.relationName = relationName;
+            this.keys = keys;
+            this.operatorInputIndex = operatorInputIndex;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                final IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private final FrameTupleAccessor accessor0 = new FrameTupleAccessor(ctx,
+                        recordDescProvider.getInputRecordDescriptor(getOperatorId(), operatorInputIndex));
+
+                ITuplePartitionComputer hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories)
+                        .createPartitioner();
+
+                private final FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                private ByteBuffer[] outbufs;
+                private File[] files;
+                private FileChannel[] channels;
+                private final int numPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    for (int i = 0; i < numPartitions; i++) {
+                        try {
+                            ByteBuffer head = outbufs[i];
+                            accessor0.reset(head);
+                            if (accessor0.getTupleCount() > 0) {
+                                FileChannel wChannel = channels[i];
+                                if (wChannel == null) {
+                                    wChannel = new RandomAccessFile(files[i], "rw").getChannel();
+                                    channels[i] = wChannel;
+                                }
+                                wChannel.write(head);
+                                numWrite++;
+                            }
+                        } catch (IOException e) {
+                            throw new HyracksDataException("error generating partition " + files[i].getName());
+                        }
+                    }
+
+                    env.set(relationName, channels);
+                    env.set(NUM_PARTITION, numPartitions);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    accessor0.reset(buffer);
+                    int tCount = accessor0.getTupleCount();
+                    for (int i = 0; i < tCount; ++i) {
+
+                        int entry = hpc.partition(accessor0, i, numPartitions);
+                        boolean newBuffer = false;
+                        ByteBuffer outbuf = outbufs[entry];
+                        while (true) {
+                            appender.reset(outbuf, newBuffer);
+                            if (appender.append(accessor0, i)) {
+                                break;
+                            } else {
+                                // buffer is full, ie. we cannot fit the tuple
+                                // into the buffer -- write it to disk
+                                try {
+
+                                    FileChannel wChannel = channels[entry];
+                                    if (wChannel == null) {
+                                        wChannel = new RandomAccessFile(files[entry], "rw").getChannel();
+                                        channels[entry] = wChannel;
+                                    }
+
+                                    wChannel.write(outbuf);
+                                    numWrite++;
+                                    outbuf.clear();
+                                    newBuffer = true;
+                                } catch (IOException e) {
+                                    throw new HyracksDataException("error generating partition "
+                                            + files[entry].getName());
+                                }
+                            }
+                        }
+                    }
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    outbufs = new ByteBuffer[numPartitions];
+                    files = new File[numPartitions];
+                    channels = new FileChannel[numPartitions];
+                    for (int i = 0; i < numPartitions; i++) {
+                        try {
+                            files[i] = ctx.getResourceManager().createFile(relationName, null);
+                            files[i].deleteOnExit();
+                            outbufs[i] = ctx.getResourceManager().allocateFrame();
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+                }
+
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return GraceHashJoinOperatorDescriptor.this;
+        }
+    }
+
+    private class JoinActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private InMemoryHashJoin joiner;
+
+                private IFrameWriter writer;
+                private FileChannel[] channelsR;
+                private FileChannel[] channelsS;
+                private int numPartitions;
+                private int[] maxBufferRi;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    channelsR = (FileChannel[]) env.get(SMALLRELATION);
+                    channelsS = (FileChannel[]) env.get(LARGERELATION);
+                    numPartitions = (Integer) env.get(NUM_PARTITION);
+
+                    ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(numPartitions,
+                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)).createPartitioner();
+                    ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(numPartitions,
+                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)).createPartitioner();
+
+                    writer.open();// open for probe
+
+                    maxBufferRi = new int[numPartitions];
+
+                    ByteBuffer buffer = ctx.getResourceManager().allocateFrame();// input
+                    // buffer
+                    int tableSize = (int) (numPartitions * recordsPerFrame * factor);
+                    for (int partitionid = 0; partitionid < numPartitions; partitionid++) {
+                        int counter = 0;
+                        int state = 0;
+                        try {
+                            FileChannel inChannelR = channelsR[partitionid];
+                            if (inChannelR != null) {
+                                inChannelR.position(0);
+                                while (state != -1) {
+
+                                    joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0),
+                                            hpcRep0, new FrameTupleAccessor(ctx, rd1), hpcRep1,
+                                            new FrameTuplePairComparator(keys0, keys1, comparators));
+                                    // build
+
+                                    state = inChannelR.read(buffer);
+                                    while (state != -1) {
+
+                                        ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                                        FrameUtils.copy(buffer, copyBuffer);
+                                        joiner.build(copyBuffer);
+                                        numReadI1++;
+                                        counter++;
+                                        if (counter > maxBufferRi[partitionid])
+                                            maxBufferRi[partitionid] = counter;
+
+                                        buffer.clear();
+                                        state = inChannelR.read(buffer);
+                                    }
+
+                                    // probe
+
+                                    buffer.clear();
+
+                                    FileChannel inChannelS = channelsS[partitionid];
+                                    if (inChannelS != null) {
+                                        inChannelS.position(0);
+                                        while (inChannelS.read(buffer) != -1) {
+                                            joiner.join(buffer, writer);
+                                            numReadI2++;
+                                            buffer.clear();
+                                        }
+                                        inChannelS.close();
+                                        joiner.closeJoin(writer);
+                                    }
+                                }
+                                inChannelR.close();
+                            }
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+                    writer.close();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(LARGERELATION, null);
+                    env.set(SMALLRELATION, null);
+                    env.set(NUM_PARTITION, null);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalStateException();
+                    }
+                    this.writer = writer;
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return GraceHashJoinOperatorDescriptor.this;
+        }
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
new file mode 100644
index 0000000..cc58ff0
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -0,0 +1,565 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.RepartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String JOINER0 = "joiner0";
+    private static final String SMALLRELATION = "RelR";
+    private static final String LARGERELATION = "RelS";
+    private static final String MEM_HASHTABLE = "MEMORY_HASHTABLE";
+    private static final String NUM_PARTITION = "NUMBER_B_PARTITIONS"; // B
+    private final int memsize;
+    private static final long serialVersionUID = 1L;
+    private final int inputsize0;
+    private final double factor;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final int recordsPerFrame;
+
+    private int numReadI1 = 0;
+    private int numWriteI1 = 0;
+    private int numReadI2 = 0;
+    private int numWriteI2 = 0;
+
+    /**
+     * @param spec
+     * @param memsize
+     *            in frames
+     * @param inputsize0
+     *            in frames
+     * @param recordsPerFrame
+     * @param factor
+     * @param keys0
+     * @param keys1
+     * @param hashFunctionFactories
+     * @param comparatorFactories
+     * @param recordDescriptor
+     * @throws HyracksDataException
+     */
+    public HybridHashJoinOperatorDescriptor(JobSpecification spec, int memsize, int inputsize0, int recordsPerFrame,
+            double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor)
+            throws HyracksDataException {
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.factor = factor;
+        this.recordsPerFrame = recordsPerFrame;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(SMALLRELATION);
+        PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(LARGERELATION);
+
+        builder.addTask(phase1);
+        builder.addSourceEdge(0, phase1, 0);
+
+        builder.addTask(phase2);
+        builder.addSourceEdge(1, phase2, 0);
+
+        builder.addBlockingEdge(phase1, phase2);
+
+        builder.addTargetEdge(0, phase2, 0);
+    }
+
+    private class BuildAndPartitionActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+        private String relationName;
+
+        public BuildAndPartitionActivityNode(String relationName) {
+            super();
+            this.relationName = relationName;
+
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private InMemoryHashJoin joiner0;
+                private final FrameTupleAccessor accessor0 = new FrameTupleAccessor(ctx, rd0);
+                ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
+                        .createPartitioner();
+                private final FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                private final FrameTupleAppender ftappender = new FrameTupleAppender(ctx);
+                private ByteBuffer[] bufferForPartitions;
+                private final ByteBuffer inBuffer = ctx.getResourceManager().allocateFrame();
+                private File[] files;
+                private FileChannel[] channels;
+                private int memoryForHashtable;
+                private int B;
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    if (memoryForHashtable != 0)
+                        build(inBuffer);
+
+                    for (int i = 0; i < B; i++) {
+                        try {
+                            ByteBuffer buf = bufferForPartitions[i];
+                            accessor0.reset(buf);
+                            if (accessor0.getTupleCount() > 0) {
+                                FileChannel wChannel = channels[i];
+                                if (wChannel == null) {
+                                    wChannel = new RandomAccessFile(files[i], "rw").getChannel();
+                                    channels[i] = wChannel;
+                                }
+                                wChannel.write(buf);
+                                numWriteI1++;
+                            }
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+
+                    env.set(relationName, channels);
+                    env.set(JOINER0, joiner0);
+                    env.set(NUM_PARTITION, B);
+                    env.set(MEM_HASHTABLE, memoryForHashtable);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+
+                    if (memoryForHashtable != memsize - 2) {
+                        accessor0.reset(buffer);
+                        int tCount = accessor0.getTupleCount();
+                        for (int i = 0; i < tCount; ++i) {
+                            int entry = -1;
+                            if (memoryForHashtable == 0) {
+                                entry = hpc0.partition(accessor0, i, B);
+                                boolean newBuffer = false;
+                                ByteBuffer bufBi = bufferForPartitions[entry];
+                                while (true) {
+                                    appender.reset(bufBi, newBuffer);
+                                    if (appender.append(accessor0, i)) {
+                                        break;
+                                    } else {
+                                        try {
+                                            FileChannel wChannel = channels[entry];
+                                            if (wChannel == null) {
+                                                wChannel = new RandomAccessFile(files[entry], "rw").getChannel();
+                                                channels[entry] = wChannel;
+                                            }
+                                            wChannel.write(bufBi);
+                                            numWriteI1++;
+                                            bufBi.clear();
+                                            newBuffer = true;
+                                        } catch (IOException e) {
+                                            throw new HyracksDataException(e);
+                                        }
+                                    }
+                                }
+                            } else {
+                                entry = hpc0.partition(accessor0, i, (int) (inputsize0 * factor / nPartitions));
+                                if (entry < memoryForHashtable) {
+                                    while (true) {
+                                        if (!ftappender.append(accessor0, i)) {
+                                            build(inBuffer);
+
+                                            ftappender.reset(inBuffer, true);
+                                        } else
+                                            break;
+                                    }
+                                } else {
+                                    entry %= B;
+                                    boolean newBuffer = false;
+                                    ByteBuffer bufBi = bufferForPartitions[entry];
+                                    while (true) {
+                                        appender.reset(bufBi, newBuffer);
+                                        if (appender.append(accessor0, i)) {
+                                            break;
+                                        } else {
+                                            try {
+                                                FileChannel wChannel;
+                                                if (channels[entry] == null) {
+                                                    wChannel = new RandomAccessFile(files[entry], "rw").getChannel();
+                                                    channels[entry] = wChannel;
+                                                } else {
+                                                    wChannel = channels[entry];
+                                                }
+                                                wChannel.write(bufBi);
+                                                numWriteI1++;
+                                                bufBi.clear();
+                                                newBuffer = true;
+                                            } catch (IOException e) {
+                                                throw new HyracksDataException(e);
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+
+                        }
+                    } else {
+                        build(buffer);
+                    }
+
+                }
+
+                private void build(ByteBuffer inBuffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                    FrameUtils.copy(inBuffer, copyBuffer);
+                    joiner0.build(copyBuffer);
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    if (memsize > 1) {
+                        if (memsize > inputsize0) {
+                            B = 0;
+                        } else {
+                            B = (int) (Math.ceil((double) (inputsize0 * factor / nPartitions - memsize)
+                                    / (double) (memsize - 1)));
+                        }
+                        if (B <= 0) {
+                            // becomes in-memory HJ
+                            memoryForHashtable = memsize - 2;
+                            B = 0;
+                        } else {
+                            memoryForHashtable = memsize - B - 2;
+                            if (memoryForHashtable < 0) {
+                                memoryForHashtable = 0;
+                                B = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
+                            }
+                        }
+                    } else {
+                        throw new HyracksDataException("not enough memory");
+                    }
+
+                    ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
+                            .createPartitioner();
+                    ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
+                            .createPartitioner();
+                    int tableSize = (int) (memoryForHashtable * recordsPerFrame * factor);
+                    joiner0 = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0), hpc0,
+                            new FrameTupleAccessor(ctx, rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
+                                    comparators));
+                    files = new File[B];
+                    channels = new FileChannel[B];
+                    bufferForPartitions = new ByteBuffer[B];
+                    for (int i = 0; i < B; i++) {
+                        try {
+                            files[i] = ctx.getResourceManager().createFile(relationName, null);
+                            files[i].deleteOnExit();
+                            bufferForPartitions[i] = ctx.getResourceManager().allocateFrame();
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+
+                    ftappender.reset(inBuffer, true);
+                }
+
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return HybridHashJoinOperatorDescriptor.this;
+        }
+    }
+
+    private class PartitionAndJoinActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+        private String largeRelation;
+
+        public PartitionAndJoinActivityNode(String relationName) {
+            super();
+            this.largeRelation = relationName;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private InMemoryHashJoin joiner0;
+                private final FrameTupleAccessor accessor1 = new FrameTupleAccessor(ctx, rd1);
+                private ITuplePartitionComputerFactory hpcf0 = new FieldHashPartitionComputerFactory(keys0,
+                        hashFunctionFactories);
+                private ITuplePartitionComputerFactory hpcf1 = new FieldHashPartitionComputerFactory(keys1,
+                        hashFunctionFactories);
+                ITuplePartitionComputer hpc1 = hpcf1.createPartitioner();
+
+                private final FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                private final FrameTupleAppender ftap = new FrameTupleAppender(ctx);
+                private final ByteBuffer inBuffer = ctx.getResourceManager().allocateFrame();
+                private final ByteBuffer outBuffer = ctx.getResourceManager().allocateFrame();
+                private IFrameWriter writer;
+                private FileChannel[] channelsR;
+                private FileChannel[] channelsS;
+                private File filesS[];
+                private ByteBuffer[] bufferForPartitions;
+                private int B;
+                private int memoryForHashtable;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    joiner0 = (InMemoryHashJoin) env.get(JOINER0);
+                    writer.open();
+                    channelsR = (FileChannel[]) env.get(SMALLRELATION);
+                    B = (Integer) env.get(NUM_PARTITION);
+                    memoryForHashtable = (Integer) env.get(MEM_HASHTABLE);
+                    filesS = new File[B];
+                    channelsS = new FileChannel[B];
+                    bufferForPartitions = new ByteBuffer[B];
+                    for (int i = 0; i < B; i++) {
+                        try {
+                            filesS[i] = ctx.getResourceManager().createFile(largeRelation, null);
+                            filesS[i].deleteOnExit();
+                            bufferForPartitions[i] = ctx.getResourceManager().allocateFrame();
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+                    appender.reset(outBuffer, true);
+                    ftap.reset(inBuffer, true);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    if (memoryForHashtable != memsize - 2) {
+                        accessor1.reset(buffer);
+                        int tupleCount1 = accessor1.getTupleCount();
+                        for (int i = 0; i < tupleCount1; ++i) {
+
+                            int entry = -1;
+                            if (memoryForHashtable == 0) {
+                                entry = hpc1.partition(accessor1, i, B);
+                                boolean newBuffer = false;
+                                ByteBuffer outbuf = bufferForPartitions[entry];
+                                while (true) {
+                                    appender.reset(outbuf, newBuffer);
+                                    if (appender.append(accessor1, i)) {
+                                        break;
+                                    } else {
+                                        try {
+                                            FileChannel wChannel = channelsS[entry];
+                                            if (wChannel == null) {
+                                                wChannel = new RandomAccessFile(filesS[entry], "rw").getChannel();
+                                                channelsS[entry] = wChannel;
+                                            }
+
+                                            wChannel.write(outbuf);
+                                            numWriteI2++;
+                                            outbuf.clear();
+                                            newBuffer = true;
+                                        } catch (IOException e) {
+                                            throw new HyracksDataException(e);
+                                        }
+                                    }
+                                }
+                            } else {
+                                entry = hpc1.partition(accessor1, i, (int) (inputsize0 * factor / nPartitions));
+                                if (entry < memoryForHashtable) {
+                                    while (true) {
+                                        if (!ftap.append(accessor1, i)) {
+                                            joiner0.join(inBuffer, writer);
+                                            ftap.reset(inBuffer, true);
+                                        } else
+                                            break;
+                                    }
+
+                                } else {
+                                    entry %= B;
+                                    boolean newBuffer = false;
+                                    ByteBuffer outbuf = bufferForPartitions[entry];
+                                    while (true) {
+                                        appender.reset(outbuf, newBuffer);
+                                        if (appender.append(accessor1, i)) {
+                                            break;
+                                        } else {
+                                            try {
+                                                FileChannel wChannel = channelsS[entry];
+                                                if (wChannel == null) {
+                                                    wChannel = new RandomAccessFile(filesS[entry], "rw").getChannel();
+                                                    channelsS[entry] = wChannel;
+                                                    wChannel = channelsS[entry];
+                                                }
+                                                wChannel.write(outbuf);
+                                                numWriteI2++;
+                                                outbuf.clear();
+                                                newBuffer = true;
+                                            } catch (IOException e) {
+                                                throw new HyracksDataException(e);
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    } else {
+                        joiner0.join(buffer, writer);
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    joiner0.join(inBuffer, writer);
+                    joiner0.closeJoin(writer);
+                    ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(B, hpcf0).createPartitioner();
+                    ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(B, hpcf1).createPartitioner();
+                    if (memoryForHashtable != memsize - 2) {
+                        int[] memRi = new int[B];
+                        for (int i = 0; i < B; i++) {
+                            try {
+                                FileChannel wChannel = channelsS[i];
+                                if (wChannel != null) {
+                                    ByteBuffer outbuf = bufferForPartitions[i];
+                                    accessor1.reset(outbuf);
+                                    if (accessor1.getTupleCount() > 0) {
+                                        wChannel.write(outbuf);
+                                        numWriteI2++;
+                                    }
+                                }
+                            } catch (IOException e) {
+                                throw new HyracksDataException(e);
+                            }
+                        }
+
+                        inBuffer.clear();
+                        int tableSize = -1;
+                        if (memoryForHashtable == 0) {
+                            tableSize = (int) (B * recordsPerFrame * factor);
+                        } else {
+                            tableSize = (int) (memsize * recordsPerFrame * factor);
+                        }
+                        for (int partitionid = 0; partitionid < B; partitionid++) {
+
+                            int state = 0;
+                            try {
+                                FileChannel inChannel = channelsR[partitionid];
+                                if (inChannel != null) {
+                                    inChannel.position(0);
+                                    InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize,
+                                            new FrameTupleAccessor(ctx, rd0), hpcRep0,
+                                            new FrameTupleAccessor(ctx, rd1), hpcRep1, new FrameTuplePairComparator(
+                                                    keys0, keys1, comparators));
+                                    state = inChannel.read(inBuffer);
+                                    while (state != -1) {
+                                        numReadI1++;
+                                        ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                                        FrameUtils.copy(inBuffer, copyBuffer);
+                                        joiner.build(copyBuffer);
+                                        inBuffer.clear();
+                                        memRi[partitionid]++;
+                                        state = inChannel.read(inBuffer);
+                                    }
+                                    appender.reset(outBuffer, false);
+
+                                    inBuffer.clear();
+
+                                    FileChannel inChannelS = channelsS[partitionid];
+                                    if (inChannelS != null) {
+                                        inChannelS.position(0);
+                                        while (inChannelS.read(inBuffer) != -1) {
+                                            numReadI2++;
+                                            joiner.join(inBuffer, writer);
+                                            inBuffer.clear();
+                                        }
+                                        inChannelS.close();
+                                        joiner.closeJoin(writer);
+                                    }
+                                    inChannel.close();
+                                }
+                            } catch (IOException e) {
+                                throw new HyracksDataException(e);
+                            }
+                        }
+                    }
+                    writer.close();
+                    env.set(LARGERELATION, null);
+                    env.set(SMALLRELATION, null);
+                    env.set(JOINER0, null);
+                    env.set(MEM_HASHTABLE, null);
+                    env.set(NUM_PARTITION, null);
+
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalStateException();
+                    }
+                    this.writer = writer;
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return HybridHashJoinOperatorDescriptor.this;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
new file mode 100644
index 0000000..4a12563
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+
+public class InMemoryHashJoin {
+    private final Link[] table;
+    private final List<ByteBuffer> buffers;
+    private final FrameTupleAccessor accessor0;
+    private final ITuplePartitionComputer tpc0;
+    private final FrameTupleAccessor accessor1;
+    private final ITuplePartitionComputer tpc1;
+    private final FrameTupleAppender appender;
+    private final FrameTuplePairComparator tpComparator;
+    private final ByteBuffer outBuffer;
+
+    public InMemoryHashJoin(IHyracksContext ctx, int tableSize, FrameTupleAccessor accessor0,
+            ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
+            FrameTuplePairComparator comparator) {
+        table = new Link[tableSize];
+        buffers = new ArrayList<ByteBuffer>();
+        this.accessor0 = accessor0;
+        this.tpc0 = tpc0;
+        this.accessor1 = accessor1;
+        this.tpc1 = tpc1;
+        appender = new FrameTupleAppender(ctx);
+        tpComparator = comparator;
+        outBuffer = ctx.getResourceManager().allocateFrame();
+        appender.reset(outBuffer, true);
+    }
+
+    public void build(ByteBuffer buffer) throws HyracksDataException {
+        buffers.add(buffer);
+        int bIndex = buffers.size() - 1;
+        accessor0.reset(buffer);
+        int tCount = accessor0.getTupleCount();
+        for (int i = 0; i < tCount; ++i) {
+            int entry = tpc0.partition(accessor0, i, table.length);
+            long tPointer = (((long) bIndex) << 32) + i;
+            Link link = table[entry];
+            if (link == null) {
+                link = table[entry] = new Link();
+            }
+            link.add(tPointer);
+        }
+    }
+
+    public void join(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+        accessor1.reset(buffer);
+        int tupleCount1 = accessor1.getTupleCount();
+        for (int i = 0; i < tupleCount1; ++i) {
+            int entry = tpc1.partition(accessor1, i, table.length);
+            Link link = table[entry];
+            if (link != null) {
+                for (int j = 0; j < link.size; ++j) {
+                    long pointer = link.pointers[j];
+                    int bIndex = (int) ((pointer >> 32) & 0xffffffff);
+                    int tIndex = (int) (pointer & 0xffffffff);
+                    accessor0.reset(buffers.get(bIndex));
+                    int c = tpComparator.compare(accessor0, tIndex, accessor1, i);
+                    if (c == 0) {
+                        if (!appender.appendConcat(accessor0, tIndex, accessor1, i)) {
+                            flushFrame(outBuffer, writer);
+                            appender.reset(outBuffer, true);
+                            if (!appender.appendConcat(accessor0, tIndex, accessor1, i)) {
+                                throw new IllegalStateException();
+                            }
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    public void closeJoin(IFrameWriter writer) throws HyracksDataException {
+        if (appender.getTupleCount() > 0) {
+            flushFrame(outBuffer, writer);
+        }
+    }
+
+    private void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        writer.nextFrame(outBuffer);
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+    }
+
+    private static class Link {
+        private static final int INIT_POINTERS_SIZE = 8;
+
+        long[] pointers;
+        int size;
+
+        Link() {
+            pointers = new long[INIT_POINTERS_SIZE];
+            size = 0;
+        }
+
+        void add(long pointer) {
+            if (size >= pointers.length) {
+                pointers = Arrays.copyOf(pointers, pointers.length * 2);
+            }
+            pointers[size++] = pointer;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
new file mode 100644
index 0000000..fec37b1
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String JOINER = "joiner";
+
+    private static final long serialVersionUID = 1L;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final int tableSize;
+
+    public InMemoryHashJoinOperatorDescriptor(JobSpecification spec, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int tableSize) {
+        super(spec, 2, 1);
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.tableSize = tableSize;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        HashBuildActivityNode hba = new HashBuildActivityNode();
+        HashProbeActivityNode hpa = new HashProbeActivityNode();
+
+        builder.addTask(hba);
+        builder.addSourceEdge(0, hba, 0);
+
+        builder.addTask(hpa);
+        builder.addSourceEdge(1, hpa, 0);
+        builder.addTargetEdge(0, hpa, 0);
+
+        builder.addBlockingEdge(hba, hpa);
+    }
+
+    private class HashBuildActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private InMemoryHashJoin joiner;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
+                            .createPartitioner();
+                    ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
+                            .createPartitioner();
+                    joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0), hpc0,
+                            new FrameTupleAccessor(ctx, rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
+                                    comparators));
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                    FrameUtils.copy(buffer, copyBuffer);
+                    joiner.build(copyBuffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(JOINER, joiner);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    throw new IllegalArgumentException();
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemoryHashJoinOperatorDescriptor.this;
+        }
+    }
+
+    private class HashProbeActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private IFrameWriter writer;
+                private InMemoryHashJoin joiner;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    joiner = (InMemoryHashJoin) env.get(JOINER);
+                    writer.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    joiner.join(buffer, writer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    joiner.closeJoin(writer);
+                    writer.close();
+                    env.set(JOINER, null);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    if (index != 0) {
+                        throw new IllegalStateException();
+                    }
+                    this.writer = writer;
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemoryHashJoinOperatorDescriptor.this;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/ReflectionBasedMapperFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/ReflectionBasedMapperFactory.java
new file mode 100644
index 0000000..3001aea
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/ReflectionBasedMapperFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.IMapper;
+import edu.uci.ics.hyracks.dataflow.std.IMapperFactory;
+
+public class ReflectionBasedMapperFactory implements IMapperFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final Class<? extends IMapper> mapperClass;
+
+    public ReflectionBasedMapperFactory(Class<? extends IMapper> mapperClass) {
+        this.mapperClass = mapperClass;
+    }
+
+    @Override
+    public IMapper createMapper() throws HyracksDataException {
+        try {
+            return mapperClass.newInstance();
+        } catch (InstantiationException e) {
+            throw new HyracksDataException(e);
+        } catch (IllegalAccessException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/SamplerMapFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/SamplerMapFactory.java
new file mode 100644
index 0000000..9c8a5d7
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/SamplerMapFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.map;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.IMapper;
+import edu.uci.ics.hyracks.dataflow.std.IMapperFactory;
+
+public class SamplerMapFactory implements IMapperFactory {
+    private static final long serialVersionUID = 1L;
+    private final float sampleRatio;
+
+    public SamplerMapFactory(float sampleRatio) {
+        this.sampleRatio = sampleRatio;
+    }
+
+    @Override
+    public IMapper createMapper() throws HyracksDataException {
+        return new IMapper() {
+            private Random random = new Random();
+
+            @Override
+            public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+                if (random.nextFloat() <= sampleRatio) {
+                    writer.writeData(data);
+                }
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
new file mode 100644
index 0000000..fe48e3c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameDeserializer;
+import edu.uci.ics.hyracks.comm.io.SerializingDataWriter;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+
+public final class DeserializedOperatorNodePushable implements IOperatorNodePushable {
+    private final IHyracksContext ctx;
+
+    private final IOpenableDataWriterOperator delegate;
+
+    private final FrameDeserializer deserializer;
+
+    public DeserializedOperatorNodePushable(IHyracksContext ctx, IOpenableDataWriterOperator delegate,
+            RecordDescriptor inRecordDesc) {
+        this.ctx = ctx;
+        this.delegate = delegate;
+        deserializer = inRecordDesc == null ? null : new FrameDeserializer(ctx, inRecordDesc);
+    }
+
+    @Override
+    public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+        delegate.setDataWriter(index, new SerializingDataWriter(ctx, recordDesc, writer));
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        delegate.close();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        deserializer.reset(buffer);
+        while (!deserializer.done()) {
+            delegate.writeData(deserializer.deserializeRecord());
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        delegate.open();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
new file mode 100644
index 0000000..e237238
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.util;
+
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+
+public class ReferenceEntry {
+    private final int runid;
+    private FrameTupleAccessor acccessor;
+    private int tupleIndex;
+
+    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex) {
+        super();
+        this.runid = runid;
+        this.acccessor = fta;
+        this.tupleIndex = tupleIndex;
+    }
+
+    public int getRunid() {
+        return runid;
+    }
+
+    public FrameTupleAccessor getAccessor() {
+        return acccessor;
+    }
+
+    public void setAccessor(FrameTupleAccessor fta) {
+        this.acccessor = fta;
+    }
+
+    public int getTupleIndex() {
+        return tupleIndex;
+    }
+
+    public void setTupleIndex(int tupleIndex) {
+        this.tupleIndex = tupleIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
new file mode 100644
index 0000000..a99a53a
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.Comparator;
+
+import edu.uci.ics.hyracks.api.context.IHyracksContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+
+public class ReferencedPriorityQueue {
+    private final IHyracksContext ctx;
+    private final RecordDescriptor recordDescriptor;
+    private final ReferenceEntry entries[];
+    private final int size;
+    private final BitSet runAvail;
+    private int nItems;
+
+    private final Comparator<ReferenceEntry> comparator;
+
+    public ReferencedPriorityQueue(IHyracksContext ctx, RecordDescriptor recordDescriptor, int initSize,
+            Comparator<ReferenceEntry> comparator) throws IOException {
+        this.ctx = ctx;
+        this.recordDescriptor = recordDescriptor;
+        if (initSize < 1)
+            throw new IllegalArgumentException();
+        this.comparator = comparator;
+        nItems = initSize;
+        size = (initSize + 1) & 0xfffffffe;
+        entries = new ReferenceEntry[size];
+        runAvail = new BitSet(size);
+        runAvail.set(0, initSize, true);
+        for (int i = 0; i < size; i++) {
+            entries[i] = new ReferenceEntry(i, null, -1);
+        }
+    }
+
+    /**
+     * Retrieve the top entry without removing it
+     * 
+     * @return the top entry
+     */
+    public ReferenceEntry peek() {
+        return entries[0];
+    }
+
+    /**
+     * compare the new entry with entries within the queue, to find a spot for
+     * this new entry
+     * 
+     * @param entry
+     * @return runid of this entry
+     * @throws IOException
+     */
+    public int popAndReplace(FrameTupleAccessor fta, int tIndex) throws IOException {
+        ReferenceEntry entry = entries[0];
+        if (entry.getAccessor() == null) {
+            entry.setAccessor(new FrameTupleAccessor(ctx, recordDescriptor));
+        }
+        entry.getAccessor().reset(fta.getBuffer());
+        entry.setTupleIndex(tIndex);
+
+        add(entry);
+        return entry.getRunid();
+    }
+
+    /**
+     * Push entry into priority queue
+     * 
+     * @param e
+     *            the new Entry
+     */
+    private void add(ReferenceEntry e) {
+        ReferenceEntry min = entries[0];
+        int slot = (size >> 1) + (min.getRunid() >> 1);
+
+        ReferenceEntry curr = e;
+        while (!runAvail.isEmpty() && slot > 0) {
+            int c = 0;
+            if (!runAvail.get(entries[slot].getRunid())) {
+                // run of entries[slot] is exhausted, i.e. not available, curr
+                // wins
+                c = 1;
+            } else if (entries[slot].getAccessor() != null /*
+                                                            * entries[slot] is
+                                                            * not MIN value
+                                                            */
+                    && runAvail.get(curr.getRunid() /* curr run is available */)) {
+
+                if (curr.getAccessor() != null) {
+                    c = comparator.compare(entries[slot], curr);
+                } else {
+                    // curr is MIN value, wins
+                    c = 1;
+                }
+            }
+
+            if (c <= 0) { // curr lost
+                // entries[slot] swaps up
+                ReferenceEntry tmp = entries[slot];
+                entries[slot] = curr;
+                curr = tmp;// winner to pass up
+            }// else curr wins
+            slot >>= 1;
+        }
+        // set new entries[0]
+        entries[0] = curr;
+    }
+
+    /**
+     * Pop is called only when a run is exhausted
+     * 
+     * @return
+     */
+    public ReferenceEntry pop() {
+        ReferenceEntry min = entries[0];
+        runAvail.clear(min.getRunid());
+        add(min);
+        nItems--;
+        return min;
+    }
+
+    public boolean areRunsExhausted() {
+        return runAvail.isEmpty();
+    }
+
+    public int size() {
+        return nItems;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBuffer.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBuffer.java
new file mode 100644
index 0000000..a8ac4c2
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBuffer.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.util.Collection;
+
+public class SynchronizedBoundedBuffer<T> {
+    private static final int QUEUE_SIZE = 8192;
+    private Object[] buffer;
+    private int head;
+    private int tail;
+
+    public SynchronizedBoundedBuffer() {
+        buffer = new Object[QUEUE_SIZE];
+        head = 0;
+        tail = 0;
+    }
+
+    public synchronized void put(T o) throws InterruptedException {
+        while (full()) {
+            wait();
+        }
+        buffer[tail] = o;
+        tail = (tail + 1) % QUEUE_SIZE;
+        notifyAll();
+    }
+
+    public synchronized void putAll(Collection<? extends T> c) throws InterruptedException {
+        for (T o : c) {
+            while (full()) {
+                wait();
+            }
+            buffer[tail] = o;
+            tail = (tail + 1) % QUEUE_SIZE;
+        }
+        notifyAll();
+    }
+
+    public synchronized T get() throws InterruptedException {
+        while (empty()) {
+            wait();
+        }
+        T o = (T) buffer[head];
+        head = (head + 1) % QUEUE_SIZE;
+        notifyAll();
+        return o;
+    }
+
+    private boolean empty() {
+        return head == tail;
+    }
+
+    private boolean full() {
+        return (tail + 1) % QUEUE_SIZE == head;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBufferDataReader.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBufferDataReader.java
new file mode 100644
index 0000000..b218024
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBufferDataReader.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.util;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class SynchronizedBoundedBufferDataReader implements IOpenableDataReader<Object[]> {
+    private SynchronizedBoundedBuffer<Object[]> queue;
+
+    public SynchronizedBoundedBufferDataReader(SynchronizedBoundedBuffer<Object[]> queue) {
+        this.queue = queue;
+    }
+
+    @Override
+    public Object[] readData() throws HyracksDataException {
+        try {
+            return queue.get();
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() {
+        queue = null;
+    }
+
+    @Override
+    public void open() {
+        // do nothing
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBufferDataWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBufferDataWriter.java
new file mode 100644
index 0000000..05d3360
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SynchronizedBoundedBufferDataWriter.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.dataflow.std.util;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class SynchronizedBoundedBufferDataWriter implements IOpenableDataWriter<Object[]> {
+    private SynchronizedBoundedBuffer<Object[]> queue;
+
+    public SynchronizedBoundedBufferDataWriter(SynchronizedBoundedBuffer<Object[]> queue) {
+        this.queue = queue;
+    }
+
+    @Override
+    public void writeData(Object[] data) throws HyracksDataException {
+        try {
+            queue.put(data);
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            queue.put(null);
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
+        }
+        queue = null;
+    }
+
+    @Override
+    public void open() {
+        // do nothing
+    }
+}
\ No newline at end of file