merge hyracks_dev_next r847:977

git-svn-id: https://hyracks.googlecode.com/svn/branches/aggregators_dev_next@978 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-api/pom.xml b/hyracks-api/pom.xml
index f73c27a..c18995d 100644
--- a/hyracks-api/pom.xml
+++ b/hyracks-api/pom.xml
@@ -1,9 +1,6 @@
 <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-api</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
@@ -45,5 +42,15 @@
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-ipc</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
+  	<dependency>
+  		<groupId>org.apache.commons</groupId>
+  		<artifactId>commons-lang3</artifactId>
+  		<version>3.1</version>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index 6e1eedc..866d307 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
-import java.rmi.Remote;
 import java.util.EnumSet;
 import java.util.Map;
 
@@ -22,7 +21,7 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 
-public interface IHyracksClientInterface extends Remote {
+public interface IHyracksClientInterface {
     public ClusterControllerInfo getClusterControllerInfo() throws Exception;
 
     public void createApplication(String appName) throws Exception;
@@ -35,7 +34,7 @@
 
     public JobStatus getJobStatus(JobId jobId) throws Exception;
 
-    public void start(JobId jobId) throws Exception;
+    public void startJob(JobId jobId) throws Exception;
 
     public void waitForCompletion(JobId jobId) throws Exception;
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index c6ca51c..75cc245 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -52,6 +52,16 @@
     }
 
     public String toString() {
-        return "ANID:[" + odId + "]:" + id;
+        return "ANID:" + odId + ":" + id;
+    }
+
+    public static ActivityId parse(String str) {
+        if (str.startsWith("ANID:")) {
+            str = str.substring(5);
+            int idIdx = str.lastIndexOf(':');
+            return new ActivityId(OperatorDescriptorId.parse(str.substring(0, idIdx)), Integer.parseInt(str
+                    .substring(idIdx + 1)));
+        }
+        throw new IllegalArgumentException("Unable to parse: " + str);
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
index b858736..8794e09 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
@@ -49,4 +49,12 @@
     public String toString() {
         return "ODID:" + id;
     }
+
+    public static OperatorDescriptorId parse(String str) {
+        if (str.startsWith("ODID:")) {
+            str = str.substring(5);
+            return new OperatorDescriptorId(Integer.parseInt(str));
+        }
+        throw new IllegalArgumentException("Unable to parse: " + str);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
index d4e6972..0fb44c1 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
@@ -52,6 +52,15 @@
 
     @Override
     public String toString() {
-        return "TAID:[" + taskId + "]:" + attempt;
+        return "TAID:" + taskId + ":" + attempt;
+    }
+
+    public static TaskAttemptId parse(String str) {
+        if (str.startsWith("TAID:")) {
+            str = str.substring(5);
+            int idIdx = str.lastIndexOf(':');
+            return new TaskAttemptId(TaskId.parse(str.substring(0, idIdx)), Integer.parseInt(str.substring(idIdx + 1)));
+        }
+        throw new IllegalArgumentException("Unable to parse: " + str);
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
index ee63355..7e0b22d 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
@@ -52,6 +52,15 @@
 
     @Override
     public String toString() {
-        return "TID:[" + activityId + "]:" + partition;
+        return "TID:" + activityId + ":" + partition;
+    }
+
+    public static TaskId parse(String str) {
+        if (str.startsWith("TID:")) {
+            str = str.substring(4);
+            int idIdx = str.lastIndexOf(':');
+            return new TaskId(ActivityId.parse(str.substring(0, idIdx)), Integer.parseInt(str.substring(idIdx + 1)));
+        }
+        throw new IllegalArgumentException("Unable to parse: " + str);
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
index 4922723..b04fe86 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
@@ -21,25 +21,29 @@
     private static final long serialVersionUID = 1L;
 
     private final ISerializerDeserializer[] fields;
-    private final ITypeTrait[] typeTraits;
-    
+    private final ITypeTraits[] typeTraits;
+
     // leaving this constructor for backwards-compatibility
     public RecordDescriptor(ISerializerDeserializer[] fields) {
         this.fields = fields;
         this.typeTraits = null;
     }
-    
+
     // temporarily adding constructor to include type traits
-    public RecordDescriptor(ISerializerDeserializer[] fields, ITypeTrait[] typeTraits) {
-    	this.fields = fields;
+    public RecordDescriptor(ISerializerDeserializer[] fields, ITypeTraits[] typeTraits) {
+        this.fields = fields;
         this.typeTraits = typeTraits;
     }
 
+    public int getFieldCount() {
+        return fields.length;
+    }
+
     public ISerializerDeserializer[] getFields() {
         return fields;
     }
-    
-    public ITypeTrait[] getTypeTraits() {
-    	return typeTraits;
+
+    public ITypeTraits[] getTypeTraits() {
+        return typeTraits;
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
index 930d299..2b6d361 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
@@ -22,6 +22,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
@@ -33,7 +34,6 @@
 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.util.Pair;
 
 public class JobActivityGraph implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -148,8 +148,8 @@
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
                 .getConnectorOperatorMap().get(cdId);
 
-        OperatorDescriptorId consumerOpId = connEdge.second.first.getOperatorId();
-        int consumerInputIdx = connEdge.second.second;
+        OperatorDescriptorId consumerOpId = connEdge.getRight().getLeft().getOperatorId();
+        int consumerInputIdx = connEdge.getRight().getRight();
 
         for (ActivityId anId : operatorActivityMap.get(consumerOpId)) {
             List<Integer> anInputs = activityInputMap.get(anId);
@@ -168,8 +168,8 @@
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
                 .getConnectorOperatorMap().get(cdId);
 
-        OperatorDescriptorId producerOpId = connEdge.first.first.getOperatorId();
-        int producerInputIdx = connEdge.first.second;
+        OperatorDescriptorId producerOpId = connEdge.getLeft().getLeft().getOperatorId();
+        int producerInputIdx = connEdge.getLeft().getRight();
 
         for (ActivityId anId : operatorActivityMap.get(producerOpId)) {
             List<Integer> anOutputs = activityOutputMap.get(anId);
@@ -209,13 +209,11 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject jplan = new JSONObject();
 
-        jplan.put("type", "plan");
         jplan.put("flags", jobFlags.toString());
 
         JSONArray jans = new JSONArray();
         for (IActivity an : activityNodes.values()) {
             JSONObject jan = new JSONObject();
-            jan.put("type", "activity");
             jan.put("id", an.getActivityId().toString());
             jan.put("java-class", an.getClass().getName());
             jan.put("operator-id", an.getActivityId().getOperatorDescriptorId().toString());
@@ -225,7 +223,6 @@
                 JSONArray jInputs = new JSONArray();
                 for (int i = 0; i < inputs.size(); ++i) {
                     JSONObject jInput = new JSONObject();
-                    jInput.put("type", "activity-input");
                     jInput.put("input-port", i);
                     jInput.put("connector-id", inputs.get(i).getConnectorId().toString());
                     jInputs.put(jInput);
@@ -238,7 +235,6 @@
                 JSONArray jOutputs = new JSONArray();
                 for (int i = 0; i < outputs.size(); ++i) {
                     JSONObject jOutput = new JSONObject();
-                    jOutput.put("type", "activity-output");
                     jOutput.put("output-port", i);
                     jOutput.put("connector-id", outputs.get(i).getConnectorId().toString());
                     jOutputs.put(jOutput);
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 23b33f0..9c6b88c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -22,6 +22,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
@@ -33,7 +34,6 @@
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.util.Pair;
 
 public class JobSpecification implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -94,10 +94,11 @@
             IOperatorDescriptor consumerOp, int consumerPort) {
         insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
         insertIntoIndexedMap(opOutputMap, producerOp.getOperatorId(), producerPort, conn);
-        connectorOpMap.put(conn.getConnectorId(),
-                new Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>(
-                        new Pair<IOperatorDescriptor, Integer>(producerOp, producerPort),
-                        new Pair<IOperatorDescriptor, Integer>(consumerOp, consumerPort)));
+        connectorOpMap.put(
+                conn.getConnectorId(),
+                Pair.<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> of(
+                        Pair.<IOperatorDescriptor, Integer> of(producerOp, producerPort),
+                        Pair.<IOperatorDescriptor, Integer> of(consumerOp, consumerPort)));
     }
 
     public void setProperty(String name, Serializable value) {
@@ -126,19 +127,19 @@
     public RecordDescriptor getConnectorRecordDescriptor(IConnectorDescriptor conn) {
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
                 .getConnectorId());
-        return connInfo.first.first.getOutputRecordDescriptors()[connInfo.first.second];
+        return connInfo.getLeft().getLeft().getOutputRecordDescriptors()[connInfo.getLeft().getRight()];
     }
 
     public IOperatorDescriptor getConsumer(IConnectorDescriptor conn) {
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
                 .getConnectorId());
-        return connInfo.second.first;
+        return connInfo.getRight().getLeft();
     }
 
     public int getConsumerInputIndex(IConnectorDescriptor conn) {
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
                 .getConnectorId());
-        return connInfo.second.second;
+        return connInfo.getRight().getRight();
     }
 
     public IConnectorDescriptor getInputConnectorDescriptor(IOperatorDescriptor op, int inputIndex) {
@@ -180,13 +181,13 @@
     public IOperatorDescriptor getProducer(IConnectorDescriptor conn) {
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
                 .getConnectorId());
-        return connInfo.first.first;
+        return connInfo.getLeft().getLeft();
     }
 
     public int getProducerOutputIndex(IConnectorDescriptor conn) {
         Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
                 .getConnectorId());
-        return connInfo.first.second;
+        return connInfo.getLeft().getRight();
     }
 
     public List<OperatorDescriptorId> getRoots() {
@@ -264,8 +265,6 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject jjob = new JSONObject();
 
-        jjob.put("type", "job");
-
         JSONArray jopArray = new JSONArray();
         for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> e : opMap.entrySet()) {
             jopArray.put(e.getValue().toJSON());
@@ -277,12 +276,11 @@
             JSONObject conn = new JSONObject();
             Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connection = connectorOpMap
                     .get(e.getKey());
-            conn.put("type", "connector-info");
             if (connection != null) {
-                conn.put("in-operator-id", connection.first.first.getOperatorId().toString());
-                conn.put("in-operator-port", connection.first.second.intValue());
-                conn.put("out-operator-id", connection.second.first.getOperatorId().toString());
-                conn.put("out-operator-port", connection.second.second.intValue());
+                conn.put("in-operator-id", connection.getLeft().getLeft().getOperatorId().toString());
+                conn.put("in-operator-port", connection.getLeft().getRight().intValue());
+                conn.put("out-operator-id", connection.getRight().getLeft().getOperatorId().toString());
+                conn.put("out-operator-port", connection.getRight().getRight().intValue());
             }
             conn.put("connector", e.getValue().toJSON());
             jcArray.put(conn);
diff --git a/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java b/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
index e1a1fd2..168280b 100644
--- a/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
+++ b/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.hyracks.cli.commands;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.cli.Session;
 
@@ -12,7 +12,7 @@
     public ConnectCommand(String hostPortStr) {
         int idx = hostPortStr.indexOf(':');
         host = hostPortStr;
-        port = 1099;
+        port = 1098;
         if (idx != -1) {
             host = hostPortStr.substring(0, idx);
             port = Integer.valueOf(hostPortStr.substring(idx + 1));
@@ -22,7 +22,7 @@
     @Override
     public void run(Session session) throws Exception {
         System.err.println("Connecting to host: " + host + ", port: " + port);
-        IHyracksClientConnection conn = new HyracksRMIConnection(host, port);
+        IHyracksClientConnection conn = new HyracksConnection(host, port);
         session.setConnection(conn);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index bcc3c30..4575adc 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -15,8 +15,7 @@
 package edu.uci.ics.hyracks.control.cc;
 
 import java.io.File;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
+import java.net.InetSocketAddress;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Hashtable;
@@ -36,14 +35,15 @@
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.context.ICCContext;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.ipc.HyracksClientInterfaceDelegateIPCI;
 import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.web.WebServer;
+import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
 import edu.uci.ics.hyracks.control.cc.work.GetJobStatusConditionVariableWork;
@@ -69,6 +69,8 @@
 import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
+import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerDelegateIPCI;
+import edu.uci.ics.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
@@ -76,14 +78,19 @@
 import edu.uci.ics.hyracks.control.common.logs.LogFile;
 import edu.uci.ics.hyracks.control.common.work.FutureValue;
 import edu.uci.ics.hyracks.control.common.work.WorkQueue;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
 
 public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
         IHyracksClientInterface {
-    private static final long serialVersionUID = 1L;
+    private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
 
     private final CCConfig ccConfig;
 
-    private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
+    private IPCSystem clusterIPC;
+
+    private IPCSystem clientIPC;
 
     private final LogFile jobLog;
 
@@ -105,12 +112,10 @@
 
     private final WorkQueue workQueue;
 
-    private final Executor taskExecutor;
+    private final Executor executor;
 
     private final Timer timer;
 
-    private final CCClientInterface ccci;
-
     private final ICCContext ccContext;
 
     private final DeadNodeSweeper sweeper;
@@ -125,7 +130,12 @@
         ipAddressNodeNameMap = new HashMap<String, Set<String>>();
         applications = new Hashtable<String, CCApplicationContext>();
         serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
-        taskExecutor = Executors.newCachedThreadPool();
+        executor = Executors.newCachedThreadPool();
+        IIPCI ccIPCI = new ClusterControllerDelegateIPCI(this);
+        clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI, executor);
+        IIPCI ciIPCI = new HyracksClientInterfaceDelegateIPCI(this);
+        clientIPC = new IPCSystem(new InetSocketAddress(ccConfig.clientNetIpAddress, ccConfig.clientNetPort), ciIPCI,
+                executor);
         webServer = new WebServer(this);
         activeRunMap = new HashMap<JobId, JobRun>();
         runMapArchive = new LinkedHashMap<JobId, JobRun>() {
@@ -137,7 +147,6 @@
         };
         workQueue = new WorkQueue();
         this.timer = new Timer(true);
-        ccci = new CCClientInterface(this);
         ccContext = new ICCContext() {
             @Override
             public Map<String, Set<String>> getIPAddressNodeMap() {
@@ -151,9 +160,8 @@
     @Override
     public void start() throws Exception {
         LOGGER.log(Level.INFO, "Starting ClusterControllerService: " + this);
-        Registry registry = LocateRegistry.createRegistry(ccConfig.port);
-        registry.rebind(IHyracksClientInterface.class.getName(), ccci);
-        registry.rebind(IClusterController.class.getName(), this);
+        clusterIPC.start();
+        clientIPC.start();
         webServer.setPort(ccConfig.httpPort);
         webServer.start();
         workQueue.start();
@@ -174,6 +182,14 @@
         LOGGER.log(Level.INFO, "Stopped ClusterControllerService");
     }
 
+    public ServerContext getServerContext() {
+        return serverCtx;
+    }
+
+    public ICCContext getCCContext() {
+        return ccContext;
+    }
+
     public Map<String, CCApplicationContext> getApplicationMap() {
         return applications;
     }
@@ -195,7 +211,7 @@
     }
 
     public Executor getExecutor() {
-        return taskExecutor;
+        return executor;
     }
 
     public Map<String, NodeControllerState> getNodeMap() {
@@ -221,11 +237,14 @@
 
     @Override
     public NodeParameters registerNode(NodeRegistration reg) throws Exception {
-        INodeController nodeController = reg.getNodeController();
+        InetSocketAddress ncAddress = reg.getNodeControllerAddress();
         String id = reg.getNodeId();
+
+        IIPCHandle ncIPCHandle = clusterIPC.getHandle(reg.getNodeControllerAddress());
+        INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
+
         NodeControllerState state = new NodeControllerState(nodeController, reg);
         workQueue.scheduleAndSync(new RegisterNodeWork(this, id, state));
-        nodeController.notifyRegistration(this);
         LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
         NodeParameters params = new NodeParameters();
         params.setClusterControllerInfo(info);
@@ -235,10 +254,8 @@
     }
 
     @Override
-    public void unregisterNode(INodeController nodeController) throws Exception {
-        String id = nodeController.getId();
-        workQueue.scheduleAndSync(new UnregisterNodeWork(this, id));
-        LOGGER.log(Level.INFO, "Unregistered INodeController");
+    public void unregisterNode(String nodeId) throws Exception {
+        workQueue.schedule(new UnregisterNodeWork(this, nodeId));
     }
 
     @Override
@@ -268,7 +285,7 @@
     }
 
     @Override
-    public void start(JobId jobId) throws Exception {
+    public void startJob(JobId jobId) throws Exception {
         JobStartWork jse = new JobStartWork(this, jobId);
         workQueue.schedule(jse);
     }
@@ -295,13 +312,9 @@
 
     @Override
     public void createApplication(String appName) throws Exception {
-        synchronized (applications) {
-            if (applications.containsKey(appName)) {
-                throw new HyracksException("Duplicate application with name: " + appName + " being created.");
-            }
-            CCApplicationContext appCtx = new CCApplicationContext(serverCtx, ccContext, appName);
-            applications.put(appName, appCtx);
-        }
+        FutureValue<Object> fv = new FutureValue<Object>();
+        workQueue.schedule(new ApplicationCreateWork(this, appName, fv));
+        fv.get();
     }
 
     @Override
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
index 6b7e143..585c7a1 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
@@ -15,17 +15,22 @@
 package edu.uci.ics.hyracks.control.cc.adminconsole.pages;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.wicket.markup.html.basic.Label;
-import org.apache.wicket.markup.html.panel.EmptyPanel;
 import org.apache.wicket.request.mapper.parameter.PageParameters;
 import org.apache.wicket.util.string.StringValue;
 import org.json.JSONArray;
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.work.GetJobActivityGraphJSONWork;
@@ -35,8 +40,7 @@
 public class JobDetailsPage extends AbstractPage {
     private static final long serialVersionUID = 1L;
 
-    private static final int HEIGHT = 9;
-    private static final int WIDTH = 9;
+    private static final int HEIGHT = 29;
 
     public JobDetailsPage(PageParameters params) throws Exception {
         ClusterControllerService ccs = getAdminConsoleApplication().getClusterControllerService();
@@ -47,19 +51,105 @@
 
         GetJobSpecificationJSONWork gjsw = new GetJobSpecificationJSONWork(ccs, jobId);
         ccs.getWorkQueue().scheduleAndSync(gjsw);
-        add(new Label("job-specification", gjsw.getJSON().toString()));
+        Label jobspec = new Label("job-specification", gjsw.getJSON().toString());
+        jobspec.setEscapeModelStrings(false);
+        add(jobspec);
 
         GetJobActivityGraphJSONWork gjagw = new GetJobActivityGraphJSONWork(ccs, jobId);
         ccs.getWorkQueue().scheduleAndSync(gjagw);
-        add(new Label("job-activity-graph", gjagw.getJSON().toString()));
+        Label jag = new Label("job-activity-graph", gjagw.getJSON().toString());
+        jag.setEscapeModelStrings(false);
+        add(jag);
+
+        JSONObject jagO = gjagw.getJSON();
+
+        Map<ActivityId, String> activityMap = new HashMap<ActivityId, String>();
+        if (jagO.has("activities")) {
+            JSONArray aArray = jagO.getJSONArray("activities");
+            for (int i = 0; i < aArray.length(); ++i) {
+                JSONObject aO = aArray.getJSONObject(i);
+                ActivityId aid = ActivityId.parse(aO.getString("id"));
+                String className = aO.getString("java-class");
+
+                activityMap.put(aid, className);
+            }
+        }
 
         GetJobRunJSONWork gjrw = new GetJobRunJSONWork(ccs, jobId);
         ccs.getWorkQueue().scheduleAndSync(gjrw);
-        add(new Label("job-run", gjrw.getJSON().toString()));
+        Label jobrun = new Label("job-run", gjrw.getJSON().toString());
+        jobrun.setEscapeModelStrings(false);
+        add(jobrun);
 
         JSONObject jrO = gjrw.getJSON();
 
-        List<TaskProfile> taskProfiles = new ArrayList<TaskProfile>();
+        List<TaskClusterAttempt[]> tcList = new ArrayList<TaskClusterAttempt[]>();
+        long minTime = Long.MAX_VALUE;
+        long maxTime = Long.MIN_VALUE;
+        if (jrO.has("activity-clusters")) {
+            JSONArray acA = jrO.getJSONArray("activity-clusters");
+            for (int i = 0; i < acA.length(); ++i) {
+                JSONObject acO = acA.getJSONObject(i);
+                if (acO.has("plan")) {
+                    JSONObject planO = acO.getJSONObject("plan");
+                    if (planO.has("task-clusters")) {
+                        JSONArray tcA = planO.getJSONArray("task-clusters");
+                        for (int j = 0; j < tcA.length(); ++j) {
+                            JSONObject tcO = tcA.getJSONObject(j);
+                            String tcId = tcO.getString("task-cluster-id");
+                            if (tcO.has("attempts")) {
+                                JSONArray tcaA = tcO.getJSONArray("attempts");
+                                TaskClusterAttempt[] tcAttempts = new TaskClusterAttempt[tcaA.length()];
+                                for (int k = 0; k < tcaA.length(); ++k) {
+                                    JSONObject tcaO = tcaA.getJSONObject(k);
+                                    int attempt = tcaO.getInt("attempt");
+                                    long startTime = tcaO.getLong("start-time");
+                                    long endTime = tcaO.getLong("end-time");
+
+                                    tcAttempts[k] = new TaskClusterAttempt(tcId, attempt, startTime, endTime);
+                                    if (startTime < minTime) {
+                                        minTime = startTime;
+                                    }
+                                    if (endTime > maxTime) {
+                                        maxTime = endTime;
+                                    }
+                                    if (tcaO.has("task-attempts")) {
+                                        JSONArray taArray = tcaO.getJSONArray("task-attempts");
+                                        tcAttempts[k].tasks = new TaskAttempt[taArray.length()];
+                                        for (int l = 0; l < taArray.length(); ++l) {
+                                            JSONObject taO = taArray.getJSONObject(l);
+                                            TaskAttemptId taId = TaskAttemptId.parse(taO.getString("task-attempt-id"));
+                                            TaskAttempt ta = new TaskAttempt(taId, taO.getLong("start-time"),
+                                                    taO.getLong("end-time"));
+                                            tcAttempts[k].tasks[l] = ta;
+                                            TaskId tid = taId.getTaskId();
+                                            ta.name = activityMap.get(tid.getActivityId());
+                                            ta.partition = tid.getPartition();
+                                        }
+                                        Arrays.sort(tcAttempts[k].tasks, new Comparator<TaskAttempt>() {
+                                            @Override
+                                            public int compare(TaskAttempt o1, TaskAttempt o2) {
+                                                return o1.startTime < o2.startTime ? -1
+                                                        : (o1.startTime > o2.startTime ? 1 : 0);
+                                            }
+                                        });
+                                    }
+                                }
+                                Arrays.sort(tcAttempts, new Comparator<TaskClusterAttempt>() {
+                                    @Override
+                                    public int compare(TaskClusterAttempt o1, TaskClusterAttempt o2) {
+                                        return o1.startTime < o2.startTime ? -1 : (o1.startTime > o2.startTime ? 1 : 0);
+                                    }
+                                });
+                                tcList.add(tcAttempts);
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+        Map<TaskAttemptId, TaskProfile> tpMap = new HashMap<TaskAttemptId, TaskProfile>();
         if (jrO.has("profile")) {
             JSONObject pO = jrO.getJSONObject("profile");
             if (pO.has("joblets")) {
@@ -70,22 +160,27 @@
                         JSONArray tasksA = jobletO.getJSONArray("tasks");
                         for (int j = 0; j < tasksA.length(); ++j) {
                             JSONObject taskO = tasksA.getJSONObject(j);
-                            String activityId = taskO.getString("activity-id");
+                            ActivityId activityId = ActivityId.parse(taskO.getString("activity-id"));
                             int partition = taskO.getInt("partition");
                             int attempt = taskO.getInt("attempt");
+                            TaskAttemptId taId = new TaskAttemptId(new TaskId(activityId, partition), attempt);
                             if (taskO.has("partition-send-profile")) {
                                 JSONArray taskProfilesA = taskO.getJSONArray("partition-send-profile");
                                 for (int k = 0; k < taskProfilesA.length(); ++k) {
                                     JSONObject ppO = taskProfilesA.getJSONObject(k);
                                     long openTime = ppO.getLong("open-time");
                                     long closeTime = ppO.getLong("close-time");
+                                    int resolution = ppO.getInt("resolution");
+                                    long offset = ppO.getLong("offset");
                                     JSONArray frameTimesA = ppO.getJSONArray("frame-times");
                                     long[] frameTimes = new long[frameTimesA.length()];
                                     for (int l = 0; l < frameTimes.length; ++l) {
-                                        frameTimes[l] = frameTimesA.getLong(l);
+                                        frameTimes[l] = frameTimesA.getInt(l) + offset;
                                     }
-                                    taskProfiles.add(new TaskProfile(activityId, partition, attempt, openTime,
-                                            closeTime, frameTimes));
+                                    TaskProfile tp = new TaskProfile(taId, openTime, closeTime, frameTimes, resolution);
+                                    if (!tpMap.containsKey(tp.taId)) {
+                                        tpMap.put(tp.taId, tp);
+                                    }
                                 }
                             }
                         }
@@ -93,68 +188,147 @@
                 }
             }
         }
-        if (!taskProfiles.isEmpty()) {
-            Collections.sort(taskProfiles, new Comparator<TaskProfile>() {
+
+        if (!tcList.isEmpty()) {
+            Collections.sort(tcList, new Comparator<TaskClusterAttempt[]>() {
                 @Override
-                public int compare(TaskProfile o1, TaskProfile o2) {
-                    return o1.openTime < o2.openTime ? -1 : (o1.openTime > o2.openTime ? 1 : 0);
+                public int compare(TaskClusterAttempt[] o1, TaskClusterAttempt[] o2) {
+                    if (o1.length == 0) {
+                        return o2.length == 0 ? 0 : -1;
+                    } else if (o2.length == 0) {
+                        return 1;
+                    }
+                    return o1[0].startTime < o2[0].startTime ? -1 : (o1[0].startTime > o2[0].startTime ? 1 : 0);
                 }
             });
-            long startTime = taskProfiles.get(0).openTime;
-            long timeRange = taskProfiles.get(taskProfiles.size() - 1).closeTime - startTime;
-            int n = taskProfiles.size();
+            long range = maxTime - minTime;
+
+            double leftOffset = 20;
+
+            int xWidth = 1024;
+            double width = ((double) xWidth) / range;
             StringBuilder buffer = new StringBuilder();
-            buffer.append("<svg viewBox=\"0 0 ").append((timeRange + 1) * (WIDTH + 1))
-                    .append(' ').append((n + 1) * (HEIGHT + 1)).append("\" version=\"1.1\"\n");
-            buffer.append("xmlns=\"http://www.w3.org/2000/svg\">\n");
-            for (int i = 0; i < n; ++i) {
-                TaskProfile tp = taskProfiles.get(i);
-                open(buffer, i, tp.openTime - startTime);
-                for (long ft : tp.frameTimes) {
-                    nextFrame(buffer, i, ft - startTime);
+            int y = 0;
+            for (TaskClusterAttempt[] tcAttempts : tcList) {
+                for (int i = 0; i < tcAttempts.length; ++i) {
+                    TaskClusterAttempt tca = tcAttempts[i];
+                    long startTime = tca.startTime - minTime;
+                    long endTime = tca.endTime - minTime;
+                    buffer.append("<rect x=\"").append(startTime * width + leftOffset).append("\" y=\"")
+                            .append(y * (HEIGHT + 1)).append("\" width=\"").append(width * (endTime - startTime))
+                            .append("\" height=\"").append(HEIGHT).append("\"/>\n");
+                    buffer.append("<text x=\"").append(endTime * width + leftOffset + 20).append("\" y=\"")
+                            .append(y * (HEIGHT + 1) + HEIGHT * 3 / 4).append("\">")
+                            .append((endTime - startTime) + " ms").append("</text>\n");
+                    ++y;
+                    for (int j = 0; j < tca.tasks.length; ++j) {
+                        TaskAttempt ta = tca.tasks[j];
+                        long tStartTime = ta.startTime - minTime;
+                        long tEndTime = ta.endTime - minTime;
+                        buffer.append("<rect x=\"").append(tStartTime * width + leftOffset).append("\" y=\"")
+                                .append(y * (HEIGHT + 1) + HEIGHT / 4).append("\" width=\"")
+                                .append(width * (tEndTime - tStartTime)).append("\" height=\"").append(HEIGHT / 2)
+                                .append("\" style=\"fill:rgb(255,255,255);stroke-width:1;stroke:rgb(0,0,0)\"/>\n");
+                        buffer.append("<text x=\"").append(tEndTime * width + leftOffset + 20).append("\" y=\"")
+                                .append(y * (HEIGHT + 1) + HEIGHT * 3 / 4).append("\">")
+                                .append((tEndTime - tStartTime) + " ms (" + ta.name + ":" + ta.partition + ")")
+                                .append("</text>\n");
+                        TaskProfile tp = tpMap.get(ta.taId);
+                        if (tp != null) {
+                            for (int k = 0; k < tp.frameTimes.length; ++k) {
+                                long taOpenTime = tp.openTime - minTime;
+                                buffer.append("<rect x=\"")
+                                        .append(taOpenTime * width + leftOffset)
+                                        .append("\" y=\"")
+                                        .append(y * (HEIGHT + 1) + HEIGHT / 4)
+                                        .append("\" width=\"1\" height=\"")
+                                        .append(HEIGHT / 2)
+                                        .append("\" style=\"fill:rgb(255,0,0);stroke-width:1;stroke:rgb(255,0,0)\"/>\n");
+                                for (int l = 0; l < tp.frameTimes.length; ++l) {
+                                    long ft = tp.frameTimes[l];
+                                    long ftn = l < tp.frameTimes.length - 1 ? tp.frameTimes[l + 1] : ft;
+                                    long taNextTime = ft - minTime;
+                                    long barWidth = ftn - ft;
+                                    buffer.append("<rect x=\"")
+                                            .append(taNextTime * width + leftOffset)
+                                            .append("\" y=\"")
+                                            .append(y * (HEIGHT + 1) + HEIGHT / 4)
+                                            .append("\" width=\"")
+                                            .append(barWidth == 0 ? 1 : (barWidth * width))
+                                            .append("\" height=\"")
+                                            .append(HEIGHT / 2)
+                                            .append("\" style=\"fill:rgb(0,255,0);stroke-width:1;stroke:rgb(0,255,0)\"/>\n");
+                                }
+                                long taCloseTime = tp.closeTime - minTime;
+                                buffer.append("<rect x=\"")
+                                        .append(taCloseTime * width + leftOffset)
+                                        .append("\" y=\"")
+                                        .append(y * (HEIGHT + 1) + HEIGHT / 4)
+                                        .append("\" width=\"1\" height=\"")
+                                        .append(HEIGHT / 2)
+                                        .append("\" style=\"fill:rgb(0,0,255);stroke-width:1;stroke:rgb(0,0,255)\"/>\n");
+                            }
+                        }
+                        ++y;
+                    }
                 }
-                close(buffer, i, tp.closeTime - startTime);
             }
+            buffer.append("<rect x=\"").append(leftOffset).append("\" y=\"").append(0).append("\" width=\"").append(1)
+                    .append("\" height=\"").append((y + 2) * (HEIGHT + 1)).append("\"/>\n");
+            buffer.append("<rect x=\"").append(0).append("\" y=\"").append((y + 1) * (HEIGHT + 1))
+                    .append("\" width=\"").append(xWidth + 2 * leftOffset).append("\" height=\"").append(1)
+                    .append("\"/>\n");
             buffer.append("</svg>");
-            Label markup = new Label("job-timeline", buffer.toString());
+            Label markup = new Label("job-timeline",
+                    "<svg version=\"1.1\"\nxmlns=\"http://www.w3.org/2000/svg\" width=\"" + (xWidth * 1.5)
+                            + "\" height=\"" + ((y + 2) * (HEIGHT + 1)) + "\">\n" + buffer.toString());
             markup.setEscapeModelStrings(false);
             add(markup);
-        } else {
-            add(new EmptyPanel("job-timeline"));
         }
     }
 
-    private void open(StringBuilder buffer, int i, long openTime) {
-        buffer.append("<rect x=\"").append(openTime * (WIDTH + 1)).append("\" y=\"").append(i * (HEIGHT + 1))
-                .append("\" width=\"").append(WIDTH).append("\" height=\"").append(HEIGHT).append("\"/>\n");
+    private static class TaskAttempt {
+        private TaskAttemptId taId;
+        private long startTime;
+        private long endTime;
+        private String name;
+        private int partition;
+
+        public TaskAttempt(TaskAttemptId taId, long startTime, long endTime) {
+            this.taId = taId;
+            this.startTime = startTime;
+            this.endTime = endTime;
+        }
     }
 
-    private void close(StringBuilder buffer, int i, long closeTime) {
-        buffer.append("<rect x=\"").append(closeTime * (WIDTH + 1)).append("\" y=\"").append(i * (HEIGHT + 1))
-                .append("\" width=\"").append(WIDTH).append("\" height=\"").append(HEIGHT).append("\"/>\n");
-    }
+    private static class TaskClusterAttempt {
+        private String tcId;
+        private int attempt;
+        private long startTime;
+        private long endTime;
+        private TaskAttempt[] tasks;
 
-    private void nextFrame(StringBuilder buffer, int i, long frameTime) {
-        buffer.append("<rect x=\"").append(frameTime * (WIDTH + 1)).append("\" y=\"").append(i * (HEIGHT + 1))
-                .append("\" width=\"").append(WIDTH).append("\" height=\"").append(HEIGHT).append("\"/>\n");
+        public TaskClusterAttempt(String tcId, int attempt, long startTime, long endTime) {
+            this.tcId = tcId;
+            this.attempt = attempt;
+            this.startTime = startTime;
+            this.endTime = endTime;
+        }
     }
 
     private static class TaskProfile {
-        private String activityId;
-        private int partition;
-        private int attempt;
+        private TaskAttemptId taId;
         private long openTime;
         private long closeTime;
         private long[] frameTimes;
+        private int resolution;
 
-        public TaskProfile(String activityId, int partition, int attempt, long openTime, long closeTime,
-                long[] frameTimes) {
-            this.activityId = activityId;
-            this.partition = partition;
-            this.activityId = activityId;
+        public TaskProfile(TaskAttemptId taId, long openTime, long closeTime, long[] frameTimes, int resolution) {
+            this.taId = taId;
             this.openTime = openTime;
             this.closeTime = closeTime;
             this.frameTimes = frameTimes;
+            this.resolution = resolution;
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index df003b2..a72a866 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -316,6 +316,8 @@
                             JSONObject attempt = new JSONObject();
                             attempt.put("attempt", tca.getAttempt());
                             attempt.put("status", tca.getStatus());
+                            attempt.put("start-time", tca.getStartTime());
+                            attempt.put("end-time", tca.getEndTime());
 
                             JSONArray taskAttempts = new JSONArray();
                             for (TaskAttempt ta : tca.getTaskAttempts()) {
@@ -324,6 +326,8 @@
                                 taskAttempt.put("task-attempt-id", ta.getTaskAttemptId());
                                 taskAttempt.put("status", ta.getStatus());
                                 taskAttempt.put("node-id", ta.getNodeId());
+                                taskAttempt.put("start-time", ta.getStartTime());
+                                taskAttempt.put("end-time", ta.getEndTime());
                                 String failureDetails = ta.getFailureDetails();
                                 if (failureDetails != null) {
                                     taskAttempt.put("failure-details", failureDetails);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
index 43495db..7c0dd57 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -29,7 +29,7 @@
 
     private final TaskAttemptId taskId;
 
-    private final Task taskState;
+    private final Task task;
 
     private String nodeId;
 
@@ -37,10 +37,16 @@
 
     private String failureDetails;
 
-    public TaskAttempt(TaskClusterAttempt tcAttempt, TaskAttemptId taskId, Task taskState) {
+    private long startTime;
+
+    private long endTime;
+
+    public TaskAttempt(TaskClusterAttempt tcAttempt, TaskAttemptId taskId, Task task) {
         this.tcAttempt = tcAttempt;
         this.taskId = taskId;
-        this.taskState = taskState;
+        this.task = task;
+        startTime = -1;
+        endTime = -1;
     }
 
     public TaskClusterAttempt getTaskClusterAttempt() {
@@ -51,8 +57,8 @@
         return taskId;
     }
 
-    public Task getTaskState() {
-        return taskState;
+    public Task getTask() {
+        return task;
     }
 
     public String getNodeId() {
@@ -75,4 +81,20 @@
         this.status = status;
         this.failureDetails = details;
     }
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
index 9d33d12..84848bb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
@@ -32,9 +32,15 @@
 
     private int pendingTaskCounter;
 
+    private long startTime;
+
+    private long endTime;
+
     public TaskClusterAttempt(TaskCluster taskCluster, int attempt) {
         this.taskCluster = taskCluster;
         this.attempt = attempt;
+        startTime = -1;
+        endTime = -1;
     }
 
     public TaskCluster getTaskCluster() {
@@ -61,6 +67,22 @@
         return status;
     }
 
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public long getEndTime() {
+        return endTime;
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+    }
+
     public void initializePendingTaskCounter() {
         pendingTaskCounter = taskAttempts.length;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
index afa1946..2dcbb3e 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -22,9 +22,10 @@
 import java.util.Set;
 import java.util.logging.Logger;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
 import edu.uci.ics.hyracks.control.common.job.PartitionState;
@@ -52,7 +53,7 @@
             while (i.hasNext()) {
                 PartitionRequest req = i.next();
                 if (partitionDescriptor.getState().isAtLeast(req.getMinimumState())) {
-                    matches.add(new Pair<PartitionDescriptor, PartitionRequest>(partitionDescriptor, req));
+                    matches.add(Pair.<PartitionDescriptor, PartitionRequest> of(partitionDescriptor, req));
                     i.remove();
                     matched = true;
                     if (!partitionDescriptor.isReusable()) {
@@ -88,7 +89,7 @@
             while (i.hasNext()) {
                 PartitionDescriptor descriptor = i.next();
                 if (descriptor.getState().isAtLeast(partitionRequest.getMinimumState())) {
-                    match = new Pair<PartitionDescriptor, PartitionRequest>(descriptor, partitionRequest);
+                    match = Pair.<PartitionDescriptor, PartitionRequest> of(descriptor, partitionRequest);
                     if (!descriptor.isReusable()) {
                         i.remove();
                     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
index 90c6ad1..c36cf62 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
@@ -1,8 +1,9 @@
 package edu.uci.ics.hyracks.control.cc.partitions;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.common.base.INodeController;
@@ -11,23 +12,14 @@
 
 public class PartitionUtils {
     public static void reportPartitionMatch(ClusterControllerService ccs, final PartitionId pid,
-            Pair<PartitionDescriptor, PartitionRequest> match) {
-        PartitionDescriptor desc = match.first;
-        PartitionRequest req = match.second;
+            Pair<PartitionDescriptor, PartitionRequest> match) throws Exception {
+        PartitionDescriptor desc = match.getLeft();
+        PartitionRequest req = match.getRight();
 
         NodeControllerState producerNCS = ccs.getNodeMap().get(desc.getNodeId());
         NodeControllerState requestorNCS = ccs.getNodeMap().get(req.getNodeId());
         final NetworkAddress dataport = producerNCS.getDataPort();
         final INodeController requestorNC = requestorNCS.getNodeController();
-        ccs.getExecutor().execute(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    requestorNC.reportPartitionAvailability(pid, dataport);
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }
-            }
-        });
+        requestorNC.reportPartitionAvailability(pid, dataport);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
index 0e59c8df..f407ae3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
@@ -22,13 +22,14 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivity;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
 import edu.uci.ics.hyracks.control.cc.job.ActivityClusterId;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -57,7 +58,7 @@
                         int producerOutputIndex = spec.getProducerOutputIndex(conn);
                         ActivityId inTask = jag.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
                         if (!eqSet.getActivities().contains(inTask)) {
-                            return new Pair<ActivityId, ActivityId>(t, inTask);
+                            return Pair.<ActivityId, ActivityId> of(t, inTask);
                         }
                     }
                 }
@@ -70,7 +71,7 @@
                         int consumerInputIndex = spec.getConsumerInputIndex(conn);
                         ActivityId outTask = jag.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
                         if (!eqSet.getActivities().contains(outTask)) {
-                            return new Pair<ActivityId, ActivityId>(t, outTask);
+                            return Pair.<ActivityId, ActivityId> of(t, outTask);
                         }
                     }
                 }
@@ -102,7 +103,7 @@
             changed = false;
             Pair<ActivityId, ActivityId> pair = findMergePair(jag, spec, stages);
             if (pair != null) {
-                merge(stageMap, stages, pair.first, pair.second);
+                merge(stageMap, stages, pair.getLeft(), pair.getRight());
                 changed = true;
             }
         }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
index c721eca..929462b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -25,6 +25,8 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
 import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
@@ -38,7 +40,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
 import edu.uci.ics.hyracks.control.cc.job.ActivityClusterPlan;
 import edu.uci.ics.hyracks.control.cc.job.ActivityPlan;
@@ -128,7 +129,7 @@
                         Set<TaskId> cluster = taskClusterMap.get(ac1TaskStates[i].getTaskId());
                         for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
                             TaskId targetTID = ac2TaskStates[j].getTaskId();
-                            cInfoList.add(new Pair<TaskId, ConnectorDescriptorId>(targetTID, cdId));
+                            cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
                             IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
                             if (cPolicy.requiresProducerConsumerCoscheduling()) {
                                 cluster.add(targetTID);
@@ -148,12 +149,13 @@
                 List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(tid);
                 if (cInfoList != null) {
                     for (Pair<TaskId, ConnectorDescriptorId> p : cInfoList) {
-                        Task targetTS = activityPlanMap.get(p.first.getActivityId()).getTasks()[p.first.getPartition()];
+                        Task targetTS = activityPlanMap.get(p.getLeft().getActivityId()).getTasks()[p.getLeft()
+                                .getPartition()];
                         TaskCluster targetTC = targetTS.getTaskCluster();
                         if (targetTC != tc) {
-                            ConnectorDescriptorId cdId = p.second;
-                            PartitionId pid = new PartitionId(jobRun.getJobId(), cdId, tid.getPartition(),
-                                    p.first.getPartition());
+                            ConnectorDescriptorId cdId = p.getRight();
+                            PartitionId pid = new PartitionId(jobRun.getJobId(), cdId, tid.getPartition(), p.getLeft()
+                                    .getPartition());
                             tc.getProducedPartitions().add(pid);
                             targetTC.getRequiredPartitions().add(pid);
                             partitionProducingTaskClusterMap.put(pid, tc);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index 3feb5a6..e2c9ca3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -387,6 +387,7 @@
             }
             taskAttempt.setNodeId(nodeId);
             taskAttempt.setStatus(TaskAttempt.TaskStatus.RUNNING, null);
+            taskAttempt.setStartTime(System.currentTimeMillis());
             List<TaskAttemptDescriptor> tads = taskAttemptMap.get(nodeId);
             if (tads == null) {
                 tads = new ArrayList<TaskAttemptDescriptor>();
@@ -399,6 +400,7 @@
         tcAttempt.initializePendingTaskCounter();
         tcAttempts.add(tcAttempt);
         tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.RUNNING);
+        tcAttempt.setStartTime(System.currentTimeMillis());
         inProgressTaskClusters.add(tc);
     }
 
@@ -446,7 +448,7 @@
                     public void run() {
                         try {
                             node.getNodeController().startTasks(appName, jobId, JavaSerializationUtils.serialize(jag),
-                                    taskDescriptors, connectorPolicies, null);
+                                    taskDescriptors, connectorPolicies);
                         } catch (IOException e) {
                             e.printStackTrace();
                         } catch (Exception e) {
@@ -459,10 +461,11 @@
     }
 
     private void abortJob(Exception exception) {
-        for (TaskCluster tc : inProgressTaskClusters) {
+        Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<TaskCluster>(inProgressTaskClusters);
+        for (TaskCluster tc : inProgressTaskClustersCopy) {
             abortTaskCluster(findLastTaskClusterAttempt(tc));
         }
-        inProgressTaskClusters.clear();
+        assert inProgressTaskClusters.isEmpty();
         ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exception));
     }
 
@@ -477,6 +480,7 @@
             LOGGER.info("Checking " + taId + ": " + ta.getStatus());
             if (status == TaskAttempt.TaskStatus.RUNNING || status == TaskAttempt.TaskStatus.COMPLETED) {
                 ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
+                ta.setEndTime(System.currentTimeMillis());
                 List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta.getNodeId());
                 if (abortTaskAttempts == null) {
                     abortTaskAttempts = new ArrayList<TaskAttemptId>();
@@ -524,6 +528,7 @@
             TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
             if (tca != null) {
                 abortTaskCluster(tca);
+                tca.setEndTime(System.currentTimeMillis());
                 tca.setStatus(TaskClusterAttempt.TaskClusterStatus.ABORTED);
             }
         }
@@ -571,14 +576,16 @@
 
     public void notifyTaskComplete(TaskAttempt ta) throws HyracksException {
         TaskAttemptId taId = ta.getTaskAttemptId();
-        TaskCluster tc = ta.getTaskState().getTaskCluster();
+        TaskCluster tc = ta.getTask().getTaskCluster();
         TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
         if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
             TaskAttempt.TaskStatus taStatus = ta.getStatus();
             if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
                 ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
+                ta.setEndTime(System.currentTimeMillis());
                 if (lastAttempt.decrementPendingTasksCounter() == 0) {
                     lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
+                    lastAttempt.setEndTime(System.currentTimeMillis());
                     inProgressTaskClusters.remove(tc);
                     startRunnableActivityClusters();
                 }
@@ -604,13 +611,14 @@
         try {
             LOGGER.info("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
-            TaskCluster tc = ta.getTaskState().getTaskCluster();
+            TaskCluster tc = ta.getTask().getTaskCluster();
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
                 LOGGER.info("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
                 ta.setStatus(TaskAttempt.TaskStatus.FAILED, details);
                 abortTaskCluster(lastAttempt);
                 lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
+                lastAttempt.setEndTime(System.currentTimeMillis());
                 abortDoomedTaskClusters();
                 if (lastAttempt.getAttempt() >= ac.getMaxTaskClusterAttempts()) {
                     abortJob(new HyracksException(details));
@@ -648,6 +656,7 @@
                                 assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
                                 if (deadNodes.contains(ta.getNodeId())) {
                                     ta.setStatus(TaskAttempt.TaskStatus.FAILED, "Node " + ta.getNodeId() + " failed");
+                                    ta.setEndTime(System.currentTimeMillis());
                                     abort = true;
                                 }
                             }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 321b6b8..e30a718 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.Set;
+import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -25,6 +26,8 @@
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 public class JobCleanupWork extends AbstractWork {
+    private static final Logger LOGGER = Logger.getLogger(JobCleanupWork.class.getName());
+
     private ClusterControllerService ccs;
     private JobId jobId;
     private JobStatus status;
@@ -40,6 +43,14 @@
     @Override
     public void run() {
         final JobRun run = ccs.getActiveRunMap().get(jobId);
+        if (run == null) {
+            LOGGER.warning("Unable to find JobRun with id: " + jobId);
+            return;
+        }
+        if (run.getPendingStatus() != null) {
+            LOGGER.warning("Ignoring duplicate cleanup for JobRun with id: " + jobId);
+            return;
+        }
         Set<String> targetNodes = run.getParticipatingNodeIds();
         run.getCleanupPendingNodeIds().addAll(targetNodes);
         run.setPendingStatus(status, exception);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
index 87b5878..2a844ad 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
@@ -16,8 +16,9 @@
 
 import java.util.List;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
@@ -46,7 +47,11 @@
         List<Pair<PartitionDescriptor, PartitionRequest>> matches = pmm
                 .registerPartitionDescriptor(partitionDescriptor);
         for (Pair<PartitionDescriptor, PartitionRequest> match : matches) {
-            PartitionUtils.reportPartitionMatch(ccs, pid, match);
+            try {
+                PartitionUtils.reportPartitionMatch(ccs, pid, match);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
         }
     }
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
index 19716a4..a71875f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
@@ -14,8 +14,9 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import org.apache.commons.lang3.tuple.Pair;
+
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
@@ -43,7 +44,11 @@
         PartitionMatchMaker pmm = run.getPartitionMatchMaker();
         Pair<PartitionDescriptor, PartitionRequest> match = pmm.matchPartitionRequest(partitionRequest);
         if (match != null) {
-            PartitionUtils.reportPartitionMatch(ccs, pid, match);
+            try {
+                PartitionUtils.reportPartitionMatch(ccs, pid, match);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
         }
     }
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
index 2838ed4..2694af4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
@@ -39,7 +39,7 @@
     @Override
     protected void performEvent(TaskAttempt ta) {
         try {
-            ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+            ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
             JobRun run = ac.getJobRun();
             if (statistics != null) {
                 JobProfile jobProfile = run.getJobProfile();
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
index b80d928..3201bec 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
@@ -31,7 +31,7 @@
 
     @Override
     protected void performEvent(TaskAttempt ta) {
-        ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+        ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
         ac.getJobRun().getScheduler().notifyTaskFailure(ta, ac, details);
     }
 
diff --git a/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html b/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html
index f11e480..d2068fd 100644
--- a/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html
+++ b/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.html
@@ -5,5 +5,5 @@
     </div>
     <div id="job-run" wicket:id="job-run" style="display: none;">
     </div>
-    <div wicket:id="job-timeline"></div>
+    <div wicket:id="job-timeline" style="overflow: auto;"></div>
 </wicket:extend>
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
index cf64554..42d0ecb 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
@@ -14,14 +14,9 @@
  */
 package edu.uci.ics.hyracks.control.common;
 
-import java.rmi.RemoteException;
-import java.rmi.server.UnicastRemoteObject;
-
 import edu.uci.ics.hyracks.control.common.service.IService;
 
-public abstract class AbstractRemoteService extends UnicastRemoteObject implements IService {
-    private static final long serialVersionUID = 1L;
-
-    public AbstractRemoteService() throws RemoteException {
+public abstract class AbstractRemoteService implements IService {
+    public AbstractRemoteService() {
     }
 }
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 53003d0..40e9347 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.hyracks.control.common.base;
 
-import java.rmi.Remote;
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
@@ -27,10 +26,10 @@
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
 
-public interface IClusterController extends Remote {
+public interface IClusterController {
     public NodeParameters registerNode(NodeRegistration reg) throws Exception;
 
-    public void unregisterNode(INodeController nodeController) throws Exception;
+    public void unregisterNode(String nodeId) throws Exception;
 
     public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
             throws Exception;
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
index fdf49e9..47f4ceb 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.hyracks.control.common.base;
 
-import java.rmi.Remote;
 import java.util.List;
 import java.util.Map;
 
@@ -25,23 +24,16 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 
-public interface INodeController extends Remote {
-    public String getId() throws Exception;
-
-    public NCConfig getConfiguration() throws Exception;
-
+public interface INodeController {
     public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
-            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, byte[] ctxVarBytes) throws Exception;
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) throws Exception;
 
     public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
 
     public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception;
 
-    public void notifyRegistration(IClusterController ccs) throws Exception;
-
     public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
             throws Exception;
 
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index 5091d12..4536fc4 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -19,8 +19,17 @@
 import org.kohsuke.args4j.Option;
 
 public class CCConfig {
-    @Option(name = "-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
-    public int port = 1099;
+    @Option(name = "-client-net-ip-address", usage = "Sets the IP Address to listen for connections from clients", required = true)
+    public String clientNetIpAddress;
+
+    @Option(name = "-client-net-port", usage = "Sets the port to listen for connections from clients (default 1098)")
+    public int clientNetPort = 1098;
+
+    @Option(name = "-cluster-net-ip-address", usage = "Sets the IP Address to listen for connections from ", required = true)
+    public String clusterNetIpAddress;
+
+    @Option(name = "-cluster-net-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
+    public int clusterNetPort = 1099;
 
     @Option(name = "-http-port", usage = "Sets the http port for the Cluster Controller (default: 19001)")
     public int httpPort = 19001;
@@ -44,8 +53,14 @@
     public String ccRoot = "ClusterControllerService";
 
     public void toCommandLine(List<String> cList) {
-        cList.add("-port");
-        cList.add(String.valueOf(port));
+        cList.add("-client-net-ip-address");
+        cList.add(clientNetIpAddress);
+        cList.add("-client-net-port");
+        cList.add(String.valueOf(clientNetPort));
+        cList.add("-cluster-net-ip-address");
+        cList.add(clusterNetIpAddress);
+        cList.add("-cluster-net-port");
+        cList.add(String.valueOf(clusterNetPort));
         cList.add("-http-port");
         cList.add(String.valueOf(httpPort));
         cList.add("-heartbeat-period");
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 02ce0f8..c55f34a 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -22,16 +22,19 @@
 public class NCConfig implements Serializable {
     private static final long serialVersionUID = 1L;
 
-    @Option(name = "-cc-host", usage = "Cluster Controller host name")
+    @Option(name = "-cc-host", usage = "Cluster Controller host name", required = true)
     public String ccHost;
 
     @Option(name = "-cc-port", usage = "Cluster Controller port (default: 1099)")
     public int ccPort = 1099;
 
-    @Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster")
+    @Option(name = "-cluster-net-ip-address", usage = "IP Address to bind cluster listener", required = true)
+    public String clusterNetIPAddress;
+
+    @Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster", required = true)
     public String nodeId;
 
-    @Option(name = "-data-ip-address", usage = "IP Address to bind data listener")
+    @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
     public String dataIPAddress;
 
     @Option(name = "-frame-size", usage = "Frame Size to use for data communication (default: 32768)")
@@ -54,6 +57,8 @@
         cList.add(ccHost);
         cList.add("-cc-port");
         cList.add(String.valueOf(ccPort));
+        cList.add("-cluster-net-ip-address");
+        cList.add(clusterNetIPAddress);
         cList.add("-node-id");
         cList.add(nodeId);
         cList.add("-data-ip-address");
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
index ff92c2d..f6dde10 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -15,15 +15,15 @@
 package edu.uci.ics.hyracks.control.common.controllers;
 
 import java.io.Serializable;
+import java.net.InetSocketAddress;
 
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatSchema;
 
 public final class NodeRegistration implements Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final INodeController nc;
+    private final InetSocketAddress ncAddress;
 
     private final String nodeId;
 
@@ -41,9 +41,9 @@
 
     private final HeartbeatSchema hbSchema;
 
-    public NodeRegistration(INodeController nc, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
+    public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
             String osName, String arch, String osVersion, int nProcessors, HeartbeatSchema hbSchema) {
-        this.nc = nc;
+        this.ncAddress = ncAddress;
         this.nodeId = nodeId;
         this.ncConfig = ncConfig;
         this.dataPort = dataPort;
@@ -54,8 +54,8 @@
         this.hbSchema = hbSchema;
     }
 
-    public INodeController getNodeController() {
-        return nc;
+    public InetSocketAddress getNodeControllerAddress() {
+        return ncAddress;
     }
 
     public String getNodeId() {
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
index 5457ac5..46a964a 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -3,6 +3,7 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
@@ -32,12 +33,13 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject json = new JSONObject();
 
-        json.put("type", "job-profile");
         json.put("job-id", jobId.toString());
         populateCounters(json);
+        JSONArray jobletsArray = new JSONArray();
         for (JobletProfile p : jobletProfiles.values()) {
-            json.accumulate("joblets", p.toJSON());
+            jobletsArray.put(p.toJSON());
         }
+        json.put("joblets", jobletsArray);
 
         return json;
     }
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
index 78f885d..0c60006 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
@@ -47,7 +47,6 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject json = new JSONObject();
 
-        json.put("type", "joblet-profile");
         json.put("node-id", nodeId.toString());
         populateCounters(json);
         JSONArray tasks = new JSONArray();
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
index f6568d9..ef61796 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
@@ -17,6 +17,7 @@
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
 
 public class PartitionProfile implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -27,13 +28,13 @@
 
     private final long closeTime;
 
-    private final byte[] frameTimes;
+    private final MultiResolutionEventProfiler mrep;
 
-    public PartitionProfile(PartitionId pid, long openTime, long closeTime, byte[] frameTimes) {
+    public PartitionProfile(PartitionId pid, long openTime, long closeTime, MultiResolutionEventProfiler mrep) {
         this.pid = pid;
         this.openTime = openTime;
         this.closeTime = closeTime;
-        this.frameTimes = frameTimes;
+        this.mrep = mrep;
     }
 
     public PartitionId getPartitionId() {
@@ -48,7 +49,7 @@
         return closeTime;
     }
 
-    public byte[] getFrameTimes() {
-        return frameTimes;
+    public MultiResolutionEventProfiler getSamples() {
+        return mrep;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
index 3764a01..2116f61 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
-import java.io.ByteArrayInputStream;
 import java.util.Map;
 
 import org.json.JSONArray;
@@ -23,6 +22,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
 
 public class TaskProfile extends AbstractProfile {
     private static final long serialVersionUID = 1L;
@@ -48,7 +48,6 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject json = new JSONObject();
 
-        json.put("type", "task-profile");
         json.put("activity-id", taskAttemptId.getTaskId().getActivityId().toString());
         json.put("partition", taskAttemptId.getTaskId().getPartition());
         json.put("attempt", taskAttemptId.getAttempt());
@@ -65,24 +64,17 @@
                 ppObj.put("partition-id", pidObj);
                 ppObj.put("open-time", pp.getOpenTime());
                 ppObj.put("close-time", pp.getCloseTime());
-                JSONArray ftArray = new JSONArray();
-                byte[] ftb = pp.getFrameTimes();
-                ByteArrayInputStream bais = new ByteArrayInputStream(ftb);
-                long value = 0;
-                int vLen = 0;
-                long time = pp.getOpenTime();
-                for (int i = 0; i < ftb.length; ++i) {
-                    byte b = (byte) bais.read();
-                    ++vLen;
-                    value += (((long) (b & 0xef)) << ((vLen - 1) * 7));
-                    if ((b & 0x80) == 0) {
-                        time += value;
-                        ftArray.put(time);
-                        vLen = 0;
-                        value = 0;
-                    }
+                MultiResolutionEventProfiler samples = pp.getSamples();
+                ppObj.put("offset", samples.getOffset());
+                int resolution = samples.getResolution();
+                int sampleCount = samples.getCount();
+                JSONArray ftA = new JSONArray();
+                int[] ft = samples.getSamples();
+                for (int i = 0; i < sampleCount; ++i) {
+                    ftA.put(ft[i]);
                 }
-                ppObj.put("frame-times", ftArray);
+                ppObj.put("frame-times", ftA);
+                ppObj.put("resolution", resolution);
                 pspArray.put(ppObj);
             }
             json.put("partition-send-profile", pspArray);
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
index f5ae1f4..36f0c49 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
@@ -16,6 +16,8 @@
 
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
@@ -27,15 +29,21 @@
     private final WorkerThread thread;
     private final Semaphore stopSemaphore;
     private boolean stopped;
+    private final AtomicInteger enqueueCount;
+    private final AtomicInteger dequeueCount;
 
     public WorkQueue() {
         queue = new LinkedBlockingQueue<AbstractWork>();
         thread = new WorkerThread();
         stopSemaphore = new Semaphore(1);
+        enqueueCount = new AtomicInteger();
+        dequeueCount = new AtomicInteger();
     }
 
     public void start() throws HyracksException {
         stopped = false;
+        enqueueCount.set(0);
+        dequeueCount.set(0);
         try {
             stopSemaphore.acquire();
         } catch (InterruptedException e) {
@@ -61,6 +69,10 @@
     }
 
     public void schedule(AbstractWork event) {
+        enqueueCount.incrementAndGet();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Enqueue: " + enqueueCount);
+        }
         if (LOGGER.isLoggable(event.logLevel())) {
             LOGGER.log(event.logLevel(), "Scheduling: " + event);
         }
@@ -80,7 +92,7 @@
         @Override
         public void run() {
             try {
-                Runnable r;
+                AbstractWork r;
                 while (true) {
                     synchronized (WorkQueue.this) {
                         if (stopped) {
@@ -92,7 +104,14 @@
                     } catch (InterruptedException e) {
                         continue;
                     }
+                    dequeueCount.incrementAndGet();
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("Dequeue: " + dequeueCount + "/" + enqueueCount);
+                    }
                     try {
+                        if (LOGGER.isLoggable(r.logLevel())) {
+                            LOGGER.log(r.logLevel(), "Executing: " + r);
+                        }
                         r.run();
                     } catch (Exception e) {
                         e.printStackTrace();
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 1a98ebd..07d8ad7 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -145,7 +145,7 @@
     }
 
     public synchronized void notifyTaskComplete(Task task) throws Exception {
-        taskMap.remove(task);
+        taskMap.remove(task.getTaskAttemptId());
         try {
             TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), task.getPartitionSendProfile());
             task.dumpProfile(taskProfile);
@@ -159,7 +159,7 @@
     }
 
     public synchronized void notifyTaskFailed(Task task, String details) throws Exception {
-        taskMap.remove(task);
+        taskMap.remove(task.getTaskAttemptId());
         try {
             nodeController.getClusterController().notifyTaskFailure(jobId, task.getTaskAttemptId(),
                     nodeController.getId(), details);
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 3fd10a8..d869515 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -23,8 +23,7 @@
 import java.lang.management.RuntimeMXBean;
 import java.lang.management.ThreadMXBean;
 import java.net.InetAddress;
-import java.rmi.registry.LocateRegistry;
-import java.rmi.registry.Registry;
+import java.net.InetSocketAddress;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Hashtable;
@@ -58,6 +57,8 @@
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatSchema;
+import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerRemoteProxy;
+import edu.uci.ics.hyracks.control.common.ipc.NodeControllerDelegateIPCI;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 import edu.uci.ics.hyracks.control.common.work.FutureValue;
@@ -74,18 +75,20 @@
 import edu.uci.ics.hyracks.control.nc.work.DestroyApplicationWork;
 import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
 import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
 
 public class NodeControllerService extends AbstractRemoteService implements INodeController {
     private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
 
-    private static final long serialVersionUID = 1L;
-
     private NCConfig ncConfig;
 
     private final String id;
 
     private final IHyracksRootContext ctx;
 
+    private final IPCSystem ipc;
+
     private final PartitionManager partitionManager;
 
     private final ConnectionManager connectionManager;
@@ -122,6 +125,8 @@
         this.ncConfig = ncConfig;
         id = ncConfig.nodeId;
         executor = Executors.newCachedThreadPool();
+        NodeControllerDelegateIPCI ipci = new NodeControllerDelegateIPCI(this);
+        ipc = new IPCSystem(new InetSocketAddress(ncConfig.clusterNetIPAddress, 0), ipci, executor);
         this.ctx = new RootHyracksContext(ncConfig.frameSize, new IOManager(getDevices(ncConfig.ioDevices), executor));
         if (id == null) {
             throw new Exception("id not set");
@@ -160,27 +165,28 @@
     @Override
     public void start() throws Exception {
         LOGGER.log(Level.INFO, "Starting NodeControllerService");
+        ipc.start();
         connectionManager.start();
-        Registry registry = LocateRegistry.getRegistry(ncConfig.ccHost, ncConfig.ccPort);
-        IClusterController cc = (IClusterController) registry.lookup(IClusterController.class.getName());
+        IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
+        this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
         HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
         for (int i = 0; i < gcInfos.length; ++i) {
             gcInfos[i] = new HeartbeatSchema.GarbageCollectorInfo(gcMXBeans.get(i).getName());
         }
         HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
-        this.nodeParameters = cc.registerNode(new NodeRegistration(this, id, ncConfig, connectionManager
-                .getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean
-                .getAvailableProcessors(), hbSchema));
+        this.nodeParameters = ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig,
+                connectionManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(),
+                osMXBean.getAvailableProcessors(), hbSchema));
         queue.start();
 
-        heartbeatTask = new HeartbeatTask(cc);
+        heartbeatTask = new HeartbeatTask(ccs);
 
         // Schedule heartbeat generator.
         timer.schedule(heartbeatTask, 0, nodeParameters.getHeartbeatPeriod());
 
         if (nodeParameters.getProfileDumpPeriod() > 0) {
             // Schedule profile dump generator.
-            timer.schedule(new ProfileDumpTask(cc), 0, nodeParameters.getProfileDumpPeriod());
+            timer.schedule(new ProfileDumpTask(ccs), 0, nodeParameters.getProfileDumpPeriod());
         }
 
         LOGGER.log(Level.INFO, "Started NodeControllerService");
@@ -196,7 +202,6 @@
         LOGGER.log(Level.INFO, "Stopped NodeControllerService");
     }
 
-    @Override
     public String getId() {
         return id;
     }
@@ -236,23 +241,17 @@
     @Override
     public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
             List<TaskAttemptDescriptor> taskDescriptors,
-            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, byte[] ctxVarBytes) throws Exception {
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap) throws Exception {
         StartTasksWork stw = new StartTasksWork(this, appName, jobId, jagBytes, taskDescriptors, connectorPoliciesMap);
-        queue.scheduleAndSync(stw);
+        queue.schedule(stw);
     }
 
     @Override
     public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception {
         CleanupJobletWork cjw = new CleanupJobletWork(this, jobId, status);
-        queue.scheduleAndSync(cjw);
+        queue.schedule(cjw);
     }
 
-    @Override
-    public void notifyRegistration(IClusterController ccs) throws Exception {
-        this.ccs = ccs;
-    }
-
-    @Override
     public NCConfig getConfiguration() throws Exception {
         return ncConfig;
     }
@@ -260,20 +259,24 @@
     @Override
     public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
         AbortTasksWork atw = new AbortTasksWork(this, jobId, tasks);
-        queue.scheduleAndSync(atw);
+        queue.schedule(atw);
     }
 
     @Override
     public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
             throws Exception {
-        CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState);
-        queue.scheduleAndSync(caw);
+        FutureValue<Object> fv = new FutureValue<Object>();
+        CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState, fv);
+        queue.schedule(caw);
+        fv.get();
     }
 
     @Override
     public void destroyApplication(String appName) throws Exception {
-        DestroyApplicationWork daw = new DestroyApplicationWork(this, appName);
-        queue.scheduleAndSync(daw);
+        FutureValue<Object> fv = new FutureValue<Object>();
+        DestroyApplicationWork daw = new DestroyApplicationWork(this, appName, fv);
+        queue.schedule(daw);
+        fv.get();
     }
 
     @Override
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
index b9eaf1f..6e38ef7 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
@@ -63,7 +63,8 @@
         this.ctx = ctx;
         serverChannel = ServerSocketChannel.open();
         ServerSocket serverSocket = serverChannel.socket();
-        serverSocket.bind(new InetSocketAddress(inetAddress, 0));
+        serverSocket.bind(new InetSocketAddress(inetAddress, 0), 0);
+        serverSocket.setReuseAddress(true);
         stopped = false;
         connectionListener = new ConnectionListenerThread();
         dataListener = new DataListenerThread();
@@ -97,6 +98,7 @@
         public ConnectionListenerThread() {
             super("Hyracks NC Connection Listener");
             setDaemon(true);
+            setPriority(MAX_PRIORITY);
         }
 
         @Override
@@ -159,6 +161,7 @@
                         if (!pendingIncomingConnections.isEmpty()) {
                             for (SocketChannel sc : pendingIncomingConnections) {
                                 sc.configureBlocking(false);
+                                sc.socket().setReuseAddress(true);
                                 SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
                                 ByteBuffer buffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
                                 scKey.attach(buffer);
@@ -170,6 +173,7 @@
                             for (INetworkChannel nc : pendingOutgoingConnections) {
                                 SocketChannel sc = SocketChannel.open();
                                 sc.configureBlocking(false);
+                                sc.socket().setReuseAddress(true);
                                 SelectionKey scKey = sc.register(selector, 0);
                                 scKey.attach(nc);
                                 nc.setSelectionKey(scKey);
@@ -202,6 +206,7 @@
                                             } catch (HyracksException e) {
                                                 key.cancel();
                                                 sc.close();
+                                                channel.abort();
                                             }
                                         } else {
                                             buffer.compact();
@@ -210,15 +215,19 @@
                                 } else {
                                     INetworkChannel channel = (INetworkChannel) key.attachment();
                                     boolean close = false;
+                                    boolean error = false;
                                     try {
                                         close = channel.dispatchNetworkEvent();
                                     } catch (IOException e) {
                                         e.printStackTrace();
-                                        close = true;
+                                        error = true;
                                     }
-                                    if (close) {
+                                    if (close || error) {
                                         key.cancel();
                                         sc.close();
+                                        if (error) {
+                                            channel.abort();
+                                        }
                                     }
                                 }
                             }
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
index 58f7088..23cf514 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
@@ -130,7 +130,7 @@
     public synchronized boolean dispatchNetworkEvent() throws IOException {
         if (aborted) {
             eos = true;
-            monitor.notifyEndOfStream(this);
+            monitor.notifyFailure(this);
             return true;
         }
         if (key.isConnectable()) {
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
index ea4dbd6..affa01c 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ProfilingPartitionWriterFactory.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.hyracks.control.nc.profiling;
 
-import java.io.ByteArrayOutputStream;
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -23,10 +22,13 @@
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.PartitionProfile;
 import edu.uci.ics.hyracks.control.nc.Task;
 
 public class ProfilingPartitionWriterFactory implements IPartitionWriterFactory {
+    private static final int N_SAMPLES = 64;
+
     private final IHyracksTaskContext ctx;
 
     private final IConnectorDescriptor cd;
@@ -52,31 +54,17 @@
 
             private long closeTime;
 
-            private long prevTime;
-
-            private ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            MultiResolutionEventProfiler mrep = new MultiResolutionEventProfiler(N_SAMPLES);
 
             @Override
             public void open() throws HyracksDataException {
-                baos.reset();
                 openTime = System.currentTimeMillis();
-                prevTime = openTime;
                 writer.open();
             }
 
             @Override
             public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                long time = System.currentTimeMillis();
-                long diff = time - prevTime;
-                prevTime = time;
-                do {
-                    byte b = (byte) (diff & 0x7f);
-                    diff >>= 7;
-                    if (diff != 0) {
-                        b |= 0x80;
-                    }
-                    baos.write(b);
-                } while (diff != 0);
+                mrep.reportEvent();
                 writer.nextFrame(buffer);
             }
 
@@ -89,8 +77,7 @@
             public void close() throws HyracksDataException {
                 closeTime = System.currentTimeMillis();
                 ((Task) ctx).setPartitionSendProfile(new PartitionProfile(new PartitionId(ctx.getJobletContext()
-                        .getJobId(), cd.getConnectorId(), senderIndex, receiverIndex), openTime, closeTime, baos
-                        .toByteArray()));
+                        .getJobId(), cd.getConnectorId(), senderIndex, receiverIndex), openTime, closeTime, mrep));
                 writer.close();
             }
         };
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java
index 4e51142..7bebb53 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobletWork.java
@@ -46,10 +46,19 @@
         }
         ncs.getPartitionManager().unregisterPartitions(jobId);
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
-        Joblet joblet = jobletMap.get(jobId);
+        Joblet joblet = jobletMap.remove(jobId);
         if (joblet != null) {
             joblet.cleanup(status);
         }
-        ncs.getClusterController().notifyJobletCleanup(jobId, ncs.getId());
+        ncs.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    ncs.getClusterController().notifyJobletCleanup(jobId, ncs.getId());
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        });
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
index 120e376..5b60a08 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
@@ -44,41 +45,49 @@
 
     private final byte[] serializedDistributedState;
 
+    private final FutureValue<Object> fv;
+
     public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
-            byte[] serializedDistributedState) {
+            byte[] serializedDistributedState, FutureValue<Object> fv) {
         this.ncs = ncs;
         this.appName = appName;
         this.deployHar = deployHar;
         this.serializedDistributedState = serializedDistributedState;
+        this.fv = fv;
     }
 
     @Override
     protected void doRun() throws Exception {
-        NCApplicationContext appCtx;
-        Map<String, NCApplicationContext> applications = ncs.getApplications();
-        if (applications.containsKey(appName)) {
-            throw new HyracksException("Duplicate application with name: " + appName + " being created.");
-        }
-        appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
-        applications.put(appName, appCtx);
-        if (deployHar) {
-            NCConfig ncConfig = ncs.getConfiguration();
-            NodeParameters nodeParameters = ncs.getNodeParameters();
-            HttpClient hc = new DefaultHttpClient();
-            HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
-                    + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
-            HttpResponse response = hc.execute(get);
-            InputStream is = response.getEntity().getContent();
-            OutputStream os = appCtx.getHarOutputStream();
-            try {
-                IOUtils.copyLarge(is, os);
-            } finally {
-                os.close();
-                is.close();
+        try {
+            NCApplicationContext appCtx;
+            Map<String, NCApplicationContext> applications = ncs.getApplications();
+            if (applications.containsKey(appName)) {
+                throw new HyracksException("Duplicate application with name: " + appName + " being created.");
             }
+            appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
+            applications.put(appName, appCtx);
+            if (deployHar) {
+                NCConfig ncConfig = ncs.getConfiguration();
+                NodeParameters nodeParameters = ncs.getNodeParameters();
+                HttpClient hc = new DefaultHttpClient();
+                HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
+                        + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
+                HttpResponse response = hc.execute(get);
+                InputStream is = response.getEntity().getContent();
+                OutputStream os = appCtx.getHarOutputStream();
+                try {
+                    IOUtils.copyLarge(is, os);
+                } finally {
+                    os.close();
+                    is.close();
+                }
+            }
+            appCtx.initializeClassPath();
+            appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
+            appCtx.initialize();
+            fv.setValue(null);
+        } catch (Exception e) {
+            fv.setException(e);
         }
-        appCtx.initializeClassPath();
-        appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
-        appCtx.initialize();
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
index 6a5fbfc..8ac0b5c 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
@@ -18,6 +18,7 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
@@ -29,17 +30,25 @@
 
     private final String appName;
 
-    public DestroyApplicationWork(NodeControllerService ncs, String appName) {
+    private FutureValue<Object> fv;
+
+    public DestroyApplicationWork(NodeControllerService ncs, String appName, FutureValue<Object> fv) {
         this.ncs = ncs;
         this.appName = appName;
+        this.fv = fv;
     }
 
     @Override
     protected void doRun() throws Exception {
-        Map<String, NCApplicationContext> applications = ncs.getApplications();
-        ApplicationContext appCtx = applications.remove(appName);
-        if (appCtx != null) {
-            appCtx.deinitialize();
+        try {
+            Map<String, NCApplicationContext> applications = ncs.getApplications();
+            ApplicationContext appCtx = applications.remove(appName);
+            if (appCtx != null) {
+                appCtx.deinitialize();
+            }
+            fv.setValue(null);
+        } catch (Exception e) {
+            fv.setException(e);
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-common/pom.xml b/hyracks-dataflow-common/pom.xml
index 50dc4d2..142a91e 100644
--- a/hyracks-dataflow-common/pom.xml
+++ b/hyracks-dataflow-common/pom.xml
@@ -1,9 +1,6 @@
 <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-common</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
@@ -31,5 +28,10 @@
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-data-std</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
index 61638d1..beaf862 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
@@ -62,7 +62,7 @@
     public Object[] deserializeRecord() throws HyracksDataException {
         int start = frameTupleAccessor.getTupleStartOffset(tIndex) + frameTupleAccessor.getFieldSlotsLength();
         bbis.setByteBuffer(buffer, start);
-        Object[] record = new Object[recordDescriptor.getFields().length];
+        Object[] record = new Object[recordDescriptor.getFieldCount()];
         for (int i = 0; i < record.length; ++i) {
             Object instance = recordDescriptor.getFields()[i].deserialize(di);
             if (LOGGER.isLoggable(Level.FINEST)) {
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index 8ab5d4f..6c51d14 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -24,11 +24,12 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 /**
- * FrameTupleCursor is used to navigate over tuples in a Frame.
- * A frame is formatted with tuple data concatenated starting at offset 0, one tuple after another.
- * Offset FS - 4 holds an int indicating the number of tuples (N) in the frame. FS - ((i + 1) * 4) for i from
- * 0 to N - 1 holds an int indicating the offset of the (i + 1)^th tuple.
- * Every tuple is organized as a sequence of ints indicating the end of each field in the tuple relative to the end of the
+ * FrameTupleCursor is used to navigate over tuples in a Frame. A frame is
+ * formatted with tuple data concatenated starting at offset 0, one tuple after
+ * another. Offset FS - 4 holds an int indicating the number of tuples (N) in
+ * the frame. FS - ((i + 1) * 4) for i from 0 to N - 1 holds an int indicating
+ * the offset of the (i + 1)^th tuple. Every tuple is organized as a sequence of
+ * ints indicating the end of each field in the tuple relative to the end of the
  * field slots.
  * 
  * @author vinayakb
@@ -86,7 +87,7 @@
 
     @Override
     public int getFieldSlotsLength() {
-        return recordDescriptor.getFields().length * 4;
+        return getFieldCount() * 4;
     }
 
     public void prettyPrint() {
@@ -96,7 +97,7 @@
         System.err.println("TC: " + tc);
         for (int i = 0; i < tc; ++i) {
             System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
-            for (int j = 0; j < recordDescriptor.getFields().length; ++j) {
+            for (int j = 0; j < getFieldCount(); ++j) {
                 System.err.print(j + ":(" + getFieldStartOffset(i, j) + ", " + getFieldEndOffset(i, j) + ") ");
                 System.err.print("{");
                 bbis.setByteBuffer(buffer, getTupleStartOffset(i) + getFieldSlotsLength() + getFieldStartOffset(i, j));
@@ -113,6 +114,6 @@
 
     @Override
     public int getFieldCount() {
-        return recordDescriptor.getFields().length;
+        return recordDescriptor.getFieldCount();
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index e8f43cd..cbe2ae6 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -41,7 +41,7 @@
 
     public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
         buffer = ctx.allocateFrame();
-        tb = new ArrayTupleBuilder(recordDescriptor.getFields().length);
+        tb = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
         this.recordDescriptor = recordDescriptor;
         this.frameWriter = frameWriter;
         tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java
index 53b8d23..60fd395 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/UTF8StringNormalizedKeyComputerFactory.java
@@ -16,7 +16,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class UTF8StringNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
     private static final long serialVersionUID = 1L;
@@ -26,14 +26,14 @@
         return new INormalizedKeyComputer() {
             @Override
             public int normalize(byte[] bytes, int start, int length) {
-                int len = StringUtils.getUTFLen(bytes, start);
+                int len = UTF8StringPointable.getUTFLen(bytes, start);
                 int nk = 0;
                 int offset = start + 2;
                 for (int i = 0; i < 2; ++i) {
                     nk <<= 16;
                     if (i < len) {
-                        nk += ((int) StringUtils.charAt(bytes, offset)) & 0xffff;
-                        offset += StringUtils.charSize(bytes, offset);
+                        nk += ((int) UTF8StringPointable.charAt(bytes, offset)) & 0xffff;
+                        offset += UTF8StringPointable.charSize(bytes, offset);
                     }
                 }
                 return nk;
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
index 8d69b95..7769539 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
@@ -39,6 +39,9 @@
         return new ITuplePartitionComputer() {
             @Override
             public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) {
+                if (nParts == 1) {
+                    return 0;
+                }
                 int h = 0;
                 int startOffset = accessor.getTupleStartOffset(tIndex);
                 int slotLength = accessor.getFieldSlotsLength();
@@ -49,7 +52,7 @@
                     int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
                     int fh = hashFn
                             .hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
-                    h += fh;
+                    h = h * 31 + fh;
                 }
                 if (h < 0) {
                     h = -h;
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
index bcaeb22..a29209c 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
@@ -18,79 +18,6 @@
 import java.io.IOException;
 
 public class StringUtils {
-    public static char charAt(byte[] b, int s) {
-        int c = 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) | ((b[s + 1]) & 0x3F));
-
-            case 14:
-                return (char) (((c & 0x0F) << 12) | (((b[s + 1]) & 0x3F) << 6) | (((b[s + 2]) & 0x3F) << 0));
-
-            default:
-                throw new IllegalArgumentException();
-        }
-    }
-
-    public static int charSize(byte[] b, int s) {
-        int c = 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 int getModifiedUTF8Len(char c) {
-        if (c >= 0x0000 && c <= 0x007F) {
-            return 1;
-        } else if (c <= 0x07FF) {
-            return 2;
-        } else {
-            return 3;
-        }
-    }
-
-    public static int getStrLen(byte[] b, int s) {
-        int pos = s + 2;
-        int end = pos + getUTFLen(b, s);
-        int charCount = 0;
-        while (pos < end) {
-            charCount++;
-            pos += charSize(b, pos);
-        }
-        return charCount;
-    }
-
-    public static int getUTFLen(byte[] b, int s) {
-        return ((b[s] & 0xff) << 8) + ((b[s + 1] & 0xff) << 0);
-    }
-
     public static void writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
 
         if (c >= 0x0000 && c <= 0x007F) {
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
index d690279..87d9b35 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
@@ -18,11 +18,14 @@
 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.ITypeTrait;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.FloatBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.BooleanPointable;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
@@ -32,31 +35,31 @@
 
 @SuppressWarnings("rawtypes")
 public class SerdeUtils {
-    public static ITypeTrait[] serdesToTypeTraits(ISerializerDeserializer[] serdes, int numSerdes) {
-        ITypeTrait[] typeTraits = new ITypeTrait[numSerdes];
+    public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes, int numSerdes) {
+        ITypeTraits[] typeTraits = new ITypeTraits[numSerdes];
         for (int i = 0; i < numSerdes; i++) {
             typeTraits[i] = serdeToTypeTrait(serdes[i]);
         }
         return typeTraits;
     }
 
-    public static ITypeTrait serdeToTypeTrait(ISerializerDeserializer serde) {
+    public static ITypeTraits serdeToTypeTrait(ISerializerDeserializer serde) {
         if (serde instanceof IntegerSerializerDeserializer) {
-            return ITypeTrait.INTEGER_TYPE_TRAIT;
+            return IntegerPointable.TYPE_TRAITS;
         }
         if (serde instanceof Integer64SerializerDeserializer) {
-            return ITypeTrait.INTEGER64_TYPE_TRAIT;
+            return LongPointable.TYPE_TRAITS;
         }
         if (serde instanceof FloatSerializerDeserializer) {
-            return ITypeTrait.FLOAT_TYPE_TRAIT;
+            return FloatPointable.TYPE_TRAITS;
         }
         if (serde instanceof DoubleSerializerDeserializer) {
-            return ITypeTrait.DOUBLE_TYPE_TRAIT;
+            return DoublePointable.TYPE_TRAITS;
         }
         if (serde instanceof BooleanSerializerDeserializer) {
-            return ITypeTrait.BOOLEAN_TYPE_TRAIT;
+            return BooleanPointable.TYPE_TRAITS;
         }
-        return ITypeTrait.VARLEN_TYPE_TRAIT;
+        return UTF8StringPointable.TYPE_TRAITS;
     }
 
     public static IBinaryComparator[] serdesToComparators(ISerializerDeserializer[] serdes, int numSerdes) {
@@ -68,27 +71,10 @@
     }
 
     public static IBinaryComparator serdeToComparator(ISerializerDeserializer serde) {
-        if (serde instanceof IntegerSerializerDeserializer) {
-            return IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        }
-        if (serde instanceof Integer64SerializerDeserializer) {
-            throw new UnsupportedOperationException("Binary comparator for Integer64 not implemented.");
-        }
-        if (serde instanceof FloatSerializerDeserializer) {
-            return FloatBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        }
-        if (serde instanceof DoubleSerializerDeserializer) {
-            return DoubleBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        }
-        if (serde instanceof BooleanSerializerDeserializer) {
-            throw new UnsupportedOperationException("Binary comparator for Boolean not implemented.");
-        }
-        if (serde instanceof UTF8StringSerializerDeserializer) {
-            return UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        }
-        throw new UnsupportedOperationException("Binary comparator for + " + serde.toString() + " not implemented.");
+        IBinaryComparatorFactory f = serdeToComparatorFactory(serde);
+        return f.createBinaryComparator();
     }
-    
+
     public static IBinaryComparatorFactory[] serdesToComparatorFactories(ISerializerDeserializer[] serdes, int numSerdes) {
         IBinaryComparatorFactory[] comparatorsFactories = new IBinaryComparatorFactory[numSerdes];
         for (int i = 0; i < numSerdes; i++) {
@@ -99,22 +85,22 @@
 
     public static IBinaryComparatorFactory serdeToComparatorFactory(ISerializerDeserializer serde) {
         if (serde instanceof IntegerSerializerDeserializer) {
-            return IntegerBinaryComparatorFactory.INSTANCE;
+            return PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         }
         if (serde instanceof Integer64SerializerDeserializer) {
-            throw new UnsupportedOperationException("Binary comparator factory for Integer64 not implemented.");
+            return PointableBinaryComparatorFactory.of(LongPointable.FACTORY);
         }
         if (serde instanceof FloatSerializerDeserializer) {
-            return FloatBinaryComparatorFactory.INSTANCE;
+            return PointableBinaryComparatorFactory.of(FloatPointable.FACTORY);
         }
         if (serde instanceof DoubleSerializerDeserializer) {
-            return DoubleBinaryComparatorFactory.INSTANCE;
+            return PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
         }
         if (serde instanceof BooleanSerializerDeserializer) {
             throw new UnsupportedOperationException("Binary comparator factory for Boolean not implemented.");
         }
         if (serde instanceof UTF8StringSerializerDeserializer) {
-            return UTF8StringBinaryComparatorFactory.INSTANCE;
+            return PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
         }
         throw new UnsupportedOperationException("Binary comparator for + " + serde.toString() + " not implemented.");
     }
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index ef2b4cb..2841509 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -142,8 +142,8 @@
 
     @SuppressWarnings("deprecation")
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IRecordDescriptorProvider recordDescProvider,
-            final int partition, int nPartitions)
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
             @Override
@@ -191,7 +191,7 @@
                     RecordDescriptor outputRecordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
                             (Class<? extends Writable>) hadoopRecordReader.createKey().getClass(),
                             (Class<? extends Writable>) hadoopRecordReader.createValue().getClass());
-                    int nFields = outputRecordDescriptor.getFields().length;
+                    int nFields = outputRecordDescriptor.getFieldCount();
                     ArrayTupleBuilder tb = new ArrayTupleBuilder(nFields);
                     writer.open();
                     try {
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
index f3535f8..4ffc060 100644
--- 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
@@ -41,7 +41,6 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject jconn = new JSONObject();
 
-        jconn.put("type", "connector");
         jconn.put("id", getConnectorId().getId());
         jconn.put("java-class", getClass().getName());
 
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
index 15fc75d..1bb13ca 100644
--- 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
@@ -75,7 +75,6 @@
     @Override
     public JSONObject toJSON() throws JSONException {
         JSONObject jop = new JSONObject();
-        jop.put("type", "operator");
         jop.put("id", getOperatorId().getId());
         jop.put("java-class", getClass().getName());
         jop.put("in-arity", getInputArity());
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
index 33819a8..2b37c7b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
@@ -172,6 +172,7 @@
                 PartitionId pid = (PartitionId) channel.getAttachment();
                 int senderIndex = pid.getSenderIndex();
                 failSenders.set(senderIndex);
+                eosSenders.set(senderIndex);
                 NonDeterministicPartitionCollector.this.notifyAll();
             }
         }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
index 5bc6ee5..8544891 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
@@ -75,7 +75,7 @@
             writer.open();
             try {
                 while (true) {
-                    Object[] record = new Object[desc.getFields().length];
+                    Object[] record = new Object[desc.getFieldCount()];
                     if (!reader.read(record)) {
                         break;
                     }
@@ -107,8 +107,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new DeserializedFileScanOperator(partition), null);
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index e0f5a65..47a3158 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -60,8 +60,8 @@
      * @see edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.uci.ics.hyracks.api.context.IHyracksContext, edu.uci.ics.hyracks.api.job.IOperatorEnvironment, edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int, int)
      */
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            final int partition, int nPartitions)
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
         // Output files
         final FileSplit[] splits = fileSplitProvider.getFileSplits();
@@ -96,7 +96,7 @@
                         int start = frameTupleAccessor.getTupleStartOffset(tIndex)
                                 + frameTupleAccessor.getFieldSlotsLength();
                         bbis.setByteBuffer(buffer, start);
-                        Object[] record = new Object[recordDescriptor.getFields().length];
+                        Object[] record = new Object[recordDescriptor.getFieldCount()];
                         for (int i = 0; i < record.length; ++i) {
                             Object instance = recordDescriptor.getFields()[i].deserialize(di);
                             if (i == 0) {
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
index f4cfe98..ef427d1 100644
--- 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
@@ -34,223 +34,242 @@
 import edu.uci.ics.hyracks.dataflow.common.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;
+	/**
+	 * 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;
+		int[] pointers;
+		int size;
 
-        Link() {
-            pointers = new int[INIT_POINTERS_SIZE];
-            size = 0;
-        }
+		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;
-        }
-    }
+		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_AGG_STATE_SIZE = 8;
-    private final IHyracksTaskContext ctx;
+	private static final int INIT_AGG_STATE_SIZE = 8;
+	private final IHyracksTaskContext ctx;
 
-    private final List<ByteBuffer> buffers;
-    private final Link[] table;
-    /**
-     * Aggregate states: a list of states for all groups maintained in the main
-     * memory.
-     */
-    private AggregateState[] aggregateStates;
-    private int accumulatorSize;
+	private final List<ByteBuffer> buffers;
+	private final Link[] table;
+	/**
+	 * Aggregate states: a list of states for all groups maintained in the main
+	 * memory.
+	 */
+	private AggregateState[] aggregateStates;
+	private int accumulatorSize;
 
-    private int lastBIndex;
-    private final int[] storedKeys;
-    private final IBinaryComparator[] comparators;
-    private final FrameTuplePairComparator ftpc;
-    private final ITuplePartitionComputer tpc;
-    private final IAggregatorDescriptor aggregator;
+	private int lastBIndex;
+	private final int[] storedKeys;
+	private final int[] keys;
+	private final IBinaryComparator[] comparators;
+	private final FrameTuplePairComparator ftpc;
+	private final ITuplePartitionComputer tpc;
+	private final IAggregatorDescriptor aggregator;
 
-    private final FrameTupleAppender appender;
+	private final FrameTupleAppender appender;
 
-    private final FrameTupleAccessor storedKeysAccessor;
+	private final FrameTupleAccessor storedKeysAccessor;
 
-    private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
+	private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
 
-    GroupingHashTable(IHyracksTaskContext ctx, int[] fields,
-            IBinaryComparatorFactory[] comparatorFactories,
-            ITuplePartitionComputerFactory tpcf,
-            IAggregatorDescriptorFactory aggregatorFactory,
-            RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, int tableSize)
-            throws HyracksDataException {
-        this.ctx = ctx;
+	GroupingHashTable(IHyracksTaskContext ctx, int[] fields,
+			IBinaryComparatorFactory[] comparatorFactories,
+			ITuplePartitionComputerFactory tpcf,
+			IAggregatorDescriptorFactory aggregatorFactory,
+			RecordDescriptor inRecordDescriptor,
+			RecordDescriptor outRecordDescriptor, int tableSize)
+			throws HyracksDataException {
+		this.ctx = ctx;
 
-        buffers = new ArrayList<ByteBuffer>();
-        table = new Link[tableSize];
+		buffers = new ArrayList<ByteBuffer>();
+		table = new Link[tableSize];
 
-        storedKeys = new int[fields.length];
-        @SuppressWarnings("rawtypes")
-        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
-        for (int i = 0; i < fields.length; ++i) {
-            storedKeys[i] = i;
-            storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
-        }
+		keys = fields;
+		storedKeys = new int[fields.length];
+		@SuppressWarnings("rawtypes")
+		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();
+		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();
 
-        int[] keyFieldsInPartialResults = new int[fields.length];
-        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
-            keyFieldsInPartialResults[i] = i;
-        }
+		int[] keyFieldsInPartialResults = new int[fields.length];
+		for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+			keyFieldsInPartialResults[i] = i;
+		}
 
-        this.aggregator = aggregatorFactory.createAggregator(ctx,
-                inRecordDescriptor, outRecordDescriptor, fields,
-                keyFieldsInPartialResults);
+		this.aggregator = aggregatorFactory.createAggregator(ctx,
+				inRecordDescriptor, outRecordDescriptor, fields,
+				keyFieldsInPartialResults);
 
-        this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
-        accumulatorSize = 0;
+		this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
+		accumulatorSize = 0;
 
-        RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(
-                storedKeySerDeser);
-        storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
-                storedKeysRecordDescriptor);
-        lastBIndex = -1;
-        
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        
-        addNewBuffer();
+		RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(
+				storedKeySerDeser);
+		storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+				storedKeysRecordDescriptor);
+		lastBIndex = -1;
 
-        if (fields.length < outRecordDescriptor.getFields().length) {
-            stateTupleBuilder = new ArrayTupleBuilder(
-                    outRecordDescriptor.getFields().length);
-        } else {
-            stateTupleBuilder = new ArrayTupleBuilder(
-                    outRecordDescriptor.getFields().length + 1);
-        }
-        outputTupleBuilder = new ArrayTupleBuilder(
-                outRecordDescriptor.getFields().length);
-    }
+		appender = new FrameTupleAppender(ctx.getFrameSize());
 
-    private void addNewBuffer() {
-        ByteBuffer buffer = ctx.allocateFrame();
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        buffers.add(buffer);
-        appender.reset(buffer, true);
-        ++lastBIndex;
-    }
+		addNewBuffer();
 
-    void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
-        int entry = tpc.partition(accessor, tIndex, table.length);
-        Link link = table[entry];
-        if (link == null) {
-            link = table[entry] = new Link();
-        }
-        int saIndex = -1;
-        for (int i = 0; i < link.size; i += 3) {
-            int sbIndex = link.pointers[i];
-            int stIndex = link.pointers[i + 1];
-            storedKeysAccessor.reset(buffers.get(sbIndex));
-            int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
-            if (c == 0) {
-                saIndex = link.pointers[i + 2];
-                break;
-            }
-        }
-        if (saIndex < 0) {
-            // Did not find the key. Insert a new entry.
-            saIndex = accumulatorSize++;
-            // Add keys
+		if (fields.length < outRecordDescriptor.getFields().length) {
+			stateTupleBuilder = new ArrayTupleBuilder(
+					outRecordDescriptor.getFields().length);
+		} else {
+			stateTupleBuilder = new ArrayTupleBuilder(
+					outRecordDescriptor.getFields().length + 1);
+		}
+		outputTupleBuilder = new ArrayTupleBuilder(
+				outRecordDescriptor.getFields().length);
+	}
 
-            // Add aggregation fields
-            AggregateState newState = aggregator.createAggregateStates();
+	private void addNewBuffer() {
+		ByteBuffer buffer = ctx.allocateFrame();
+		buffer.position(0);
+		buffer.limit(buffer.capacity());
+		buffers.add(buffer);
+		appender.reset(buffer, true);
+		++lastBIndex;
+	}
 
-            stateTupleBuilder.reset();
+	void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
+		int entry = tpc.partition(accessor, tIndex, table.length);
+		Link link = table[entry];
+		if (link == null) {
+			link = table[entry] = new Link();
+		}
+		int saIndex = -1;
+		for (int i = 0; i < link.size; i += 3) {
+			int sbIndex = link.pointers[i];
+			int stIndex = link.pointers[i + 1];
+			storedKeysAccessor.reset(buffers.get(sbIndex));
+			int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
+			if (c == 0) {
+				saIndex = link.pointers[i + 2];
+				break;
+			}
+		}
+		if (saIndex < 0) {
+			// Did not find the key. Insert a new entry.
+			saIndex = accumulatorSize++;
+			// Add keys
 
-            aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
+			// Add aggregation fields
+			AggregateState newState = aggregator.createAggregateStates();
 
-            if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
-                    stateTupleBuilder.getByteArray(), 0,
-                    stateTupleBuilder.getSize())) {
-                addNewBuffer();
-                if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
-                        stateTupleBuilder.getByteArray(), 0,
-                        stateTupleBuilder.getSize())) {
-                    throw new HyracksDataException(
-                            "Cannot init the aggregate state in a single frame.");
-                }
-            }
-            
-            if (accumulatorSize >= aggregateStates.length) {
-                aggregateStates = Arrays.copyOf(aggregateStates,
-                        aggregateStates.length * 2);
-            }
+			stateTupleBuilder.reset();
+			for (int k = 0; k < keys.length; k++) {
+				stateTupleBuilder.addField(accessor, tIndex, keys[k]);
+			}
 
-            aggregateStates[saIndex] = newState;
+			aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
 
-            link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
+			if (!appender.appendSkipEmptyField(
+					stateTupleBuilder.getFieldEndOffsets(),
+					stateTupleBuilder.getByteArray(), 0,
+					stateTupleBuilder.getSize())) {
+				addNewBuffer();
+				if (!appender.appendSkipEmptyField(
+						stateTupleBuilder.getFieldEndOffsets(),
+						stateTupleBuilder.getByteArray(), 0,
+						stateTupleBuilder.getSize())) {
+					throw new HyracksDataException(
+							"Cannot init the aggregate state in a single frame.");
+				}
+			}
 
-        } else {
-            aggregator.aggregate(accessor, tIndex, null, 0,
-                    aggregateStates[saIndex]);
-        }
-    }
+			if (accumulatorSize >= aggregateStates.length) {
+				aggregateStates = Arrays.copyOf(aggregateStates,
+						aggregateStates.length * 2);
+			}
 
-    void write(IFrameWriter writer) throws HyracksDataException {
-        ByteBuffer buffer = ctx.allocateFrame();
-        appender.reset(buffer, true);
+			aggregateStates[saIndex] = newState;
 
-        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);
-                    
-                    outputTupleBuilder.reset();
+			link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
 
-                    aggregator
-                            .outputFinalResult(outputTupleBuilder, 
-                                    storedKeysAccessor, tIndex,
-                                    aggregateStates[aIndex]);
-                    
-                    if(!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(), outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())){
-                        writer.nextFrame(buffer);
-                        appender.reset(buffer, true);
-                        if(!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(), outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                            throw new HyracksDataException("Cannot write the aggregation output into a frame.");
-                        }
-                    }
+		} else {
+			aggregator.aggregate(accessor, tIndex, null, 0,
+					aggregateStates[saIndex]);
+		}
+	}
 
-                }
-            }
-        }
-        if (appender.getTupleCount() != 0) {
-            writer.nextFrame(buffer);
-        }
-    }
+	void write(IFrameWriter writer) throws HyracksDataException {
+		ByteBuffer buffer = ctx.allocateFrame();
+		appender.reset(buffer, true);
 
-    void close() throws HyracksDataException {
-        for (AggregateState aState : aggregateStates) {
-            aState.close();
-        }
-    }
+		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);
+
+					// copy keys
+					outputTupleBuilder.reset();
+					for (int k = 0; k < storedKeys.length; k++) {
+						outputTupleBuilder.addField(storedKeysAccessor, tIndex,
+								storedKeys[k]);
+					}
+
+					aggregator
+							.outputFinalResult(outputTupleBuilder,
+									storedKeysAccessor, tIndex,
+									aggregateStates[aIndex]);
+
+					if (!appender.appendSkipEmptyField(
+							outputTupleBuilder.getFieldEndOffsets(),
+							outputTupleBuilder.getByteArray(), 0,
+							outputTupleBuilder.getSize())) {
+						writer.nextFrame(buffer);
+						appender.reset(buffer, true);
+						if (!appender.appendSkipEmptyField(
+								outputTupleBuilder.getFieldEndOffsets(),
+								outputTupleBuilder.getByteArray(), 0,
+								outputTupleBuilder.getSize())) {
+							throw new HyracksDataException(
+									"Cannot write the aggregation output into a frame.");
+						}
+					}
+
+				}
+			}
+		}
+		if (appender.getTupleCount() != 0) {
+			writer.nextFrame(buffer);
+		}
+	}
+
+	void close() throws HyracksDataException {
+		for (AggregateState aState : aggregateStates) {
+			aState.close();
+		}
+	}
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 6bd053f..29bd083 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -27,7 +27,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -44,74 +44,74 @@
  */
 public class HashSpillableTableFactory implements ISpillableTableFactory {
 
-    private static final long serialVersionUID = 1L;
-    private final ITuplePartitionComputerFactory tpcf;
-    private final int tableSize;
+	private static final long serialVersionUID = 1L;
+	private final ITuplePartitionComputerFactory tpcf;
+	private final int tableSize;
 
-    public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf,
-            int tableSize) {
-        this.tpcf = tpcf;
-        this.tableSize = tableSize;
-    }
+	public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf,
+			int tableSize) {
+		this.tpcf = tpcf;
+		this.tableSize = tableSize;
+	}
 
-    /*
-     * (non-Javadoc)
-     * 
-     * @see
-     * edu.uci.ics.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
-     * buildSpillableTable(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
-     * int[], edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
-     * edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
-     * edu.
-     * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
-     * [], edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
-     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int)
-     */
-    @Override
-    public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx,
-            final int[] keyFields,
-            IBinaryComparatorFactory[] comparatorFactories,
-            INormalizedKeyComputerFactory firstKeyNormalizerFactory,
-            IAggregatorDescriptorFactory aggregateFactory,
-            RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, final int framesLimit)
-            throws HyracksDataException {
-        final int[] storedKeys = new int[keyFields.length];
-        @SuppressWarnings("rawtypes")
-        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[keyFields.length];
-        for (int i = 0; i < keyFields.length; i++) {
-            storedKeys[i] = i;
-            storedKeySerDeser[i] = inRecordDescriptor.getFields()[keyFields[i]];
-        }
+	/*
+	 * (non-Javadoc)
+	 * 
+	 * @see
+	 * edu.uci.ics.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
+	 * buildSpillableTable(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+	 * int[], edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
+	 * edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
+	 * edu.
+	 * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
+	 * [], edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+	 * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int)
+	 */
+	@Override
+	public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx,
+			final int[] keyFields,
+			IBinaryComparatorFactory[] comparatorFactories,
+			INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+			IAggregatorDescriptorFactory aggregateFactory,
+			RecordDescriptor inRecordDescriptor,
+			RecordDescriptor outRecordDescriptor, final int framesLimit)
+			throws HyracksDataException {
+		final int[] storedKeys = new int[keyFields.length];
+		@SuppressWarnings("rawtypes")
+		ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[keyFields.length];
+		for (int i = 0; i < keyFields.length; i++) {
+			storedKeys[i] = i;
+			storedKeySerDeser[i] = inRecordDescriptor.getFields()[keyFields[i]];
+		}
 
-        RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
-        final FrameTupleAccessor storedKeysAccessor1;
-        final FrameTupleAccessor storedKeysAccessor2;
-        if (keyFields.length >= outRecordDescriptor.getFields().length) {
-            // for the case of zero-aggregations
-            ISerializerDeserializer<?>[] fields = outRecordDescriptor
-                    .getFields();
-            ITypeTrait[] types = outRecordDescriptor.getTypeTraits();
-            ISerializerDeserializer<?>[] newFields = new ISerializerDeserializer[fields.length + 1];
-            for (int i = 0; i < fields.length; i++)
-                newFields[i] = fields[i];
-            ITypeTrait[] newTypes = null;
-            if (types != null) {
-                newTypes = new ITypeTrait[types.length + 1];
-                for (int i = 0; i < types.length; i++)
-                    newTypes[i] = types[i];
-            }
-            internalRecordDescriptor = new RecordDescriptor(newFields, newTypes);
-        }
-        storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
-                internalRecordDescriptor);
-        storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
-                internalRecordDescriptor);
+		RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
+		final FrameTupleAccessor storedKeysAccessor1;
+		final FrameTupleAccessor storedKeysAccessor2;
+		if (keyFields.length >= outRecordDescriptor.getFields().length) {
+			// for the case of zero-aggregations
+			ISerializerDeserializer<?>[] fields = outRecordDescriptor
+					.getFields();
+			ITypeTraits[] types = outRecordDescriptor.getTypeTraits();
+			ISerializerDeserializer<?>[] newFields = new ISerializerDeserializer[fields.length + 1];
+			for (int i = 0; i < fields.length; i++)
+				newFields[i] = fields[i];
+			ITypeTraits[] newTypes = null;
+			if (types != null) {
+				newTypes = new ITypeTraits[types.length + 1];
+				for (int i = 0; i < types.length; i++)
+					newTypes[i] = types[i];
+			}
+			internalRecordDescriptor = new RecordDescriptor(newFields, newTypes);
+		}
+		storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
+				internalRecordDescriptor);
+		storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
+				internalRecordDescriptor);
 
-        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
+		final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+		for (int i = 0; i < comparatorFactories.length; ++i) {
+			comparators[i] = comparatorFactories[i].createBinaryComparator();
+		}
 
         final FrameTuplePairComparator ftpcPartial = new FrameTuplePairComparator(
                 keyFields, storedKeys, comparators);
@@ -119,22 +119,22 @@
         final FrameTuplePairComparator ftpcTuple = new FrameTuplePairComparator(
                 storedKeys, storedKeys, comparators);
 
-        final ITuplePartitionComputer tpc = tpcf.createPartitioner();
+		final ITuplePartitionComputer tpc = tpcf.createPartitioner();
 
-        final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null
-                : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+		final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null
+				: firstKeyNormalizerFactory.createNormalizedKeyComputer();
 
-        int[] keyFieldsInPartialResults = new int[keyFields.length];
-        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
-            keyFieldsInPartialResults[i] = i;
-        }
+		int[] keyFieldsInPartialResults = new int[keyFields.length];
+		for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+			keyFieldsInPartialResults[i] = i;
+		}
 
-        final IAggregatorDescriptor aggregator = aggregateFactory
-                .createAggregator(ctx, inRecordDescriptor, outRecordDescriptor,
-                        keyFields, keyFieldsInPartialResults);
+		final IAggregatorDescriptor aggregator = aggregateFactory
+				.createAggregator(ctx, inRecordDescriptor, outRecordDescriptor,
+						keyFields, keyFieldsInPartialResults);
 
-        final AggregateState aggregateState = aggregator
-                .createAggregateStates();
+		final AggregateState aggregateState = aggregator
+				.createAggregateStates();
 
         final ArrayTupleBuilder stateTupleBuilder;
         if (keyFields.length < outRecordDescriptor.getFields().length) {
@@ -152,7 +152,7 @@
 
             private int lastBufIndex;
 
-            private ByteBuffer outputFrame;
+			private ByteBuffer outputFrame;
             private FrameTupleAppender outputAppender;
 
             private FrameTupleAppender stateAppender = new FrameTupleAppender(
@@ -163,88 +163,88 @@
             private final TuplePointer storedTuplePointer = new TuplePointer();
             private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
 
-            /**
-             * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
-             * = Frame index in the "Frames" list, [1] = Tuple index in the
-             * frame, [2] = Poor man's normalized key for the tuple.
-             */
-            private int[] tPointers;
+			/**
+			 * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
+			 * = Frame index in the "Frames" list, [1] = Tuple index in the
+			 * frame, [2] = Poor man's normalized key for the tuple.
+			 */
+			private int[] tPointers;
 
-            @Override
-            public void sortFrames() {
-                int sfIdx = storedKeys[0];
-                int totalTCount = table.getTupleCount();
-                tPointers = new int[totalTCount * 3];
-                int ptr = 0;
+			@Override
+			public void sortFrames() {
+				int sfIdx = storedKeys[0];
+				int totalTCount = table.getTupleCount();
+				tPointers = new int[totalTCount * 3];
+				int ptr = 0;
 
-                for (int i = 0; i < tableSize; i++) {
-                    int entry = i;
-                    int offset = 0;
-                    do {
-                        table.getTuplePointer(entry, offset, storedTuplePointer);
-                        if (storedTuplePointer.frameIndex < 0)
-                            break;
-                        tPointers[ptr * 3] = entry;
-                        tPointers[ptr * 3 + 1] = offset;
-                        table.getTuplePointer(entry, offset, storedTuplePointer);
-                        int fIndex = storedTuplePointer.frameIndex;
-                        int tIndex = storedTuplePointer.tupleIndex;
-                        storedKeysAccessor1.reset(frames.get(fIndex));
-                        int tStart = storedKeysAccessor1
-                                .getTupleStartOffset(tIndex);
-                        int f0StartRel = storedKeysAccessor1
-                                .getFieldStartOffset(tIndex, sfIdx);
-                        int f0EndRel = storedKeysAccessor1.getFieldEndOffset(
-                                tIndex, sfIdx);
-                        int f0Start = f0StartRel + tStart
-                                + storedKeysAccessor1.getFieldSlotsLength();
-                        tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc
-                                .normalize(storedKeysAccessor1.getBuffer()
-                                        .array(), f0Start, f0EndRel
-                                        - f0StartRel);
-                        ptr++;
-                        offset++;
-                    } while (true);
-                }
-                /**
-                 * Sort using quick sort
-                 */
-                if (tPointers.length > 0) {
-                    sort(tPointers, 0, totalTCount);
-                }
-            }
+				for (int i = 0; i < tableSize; i++) {
+					int entry = i;
+					int offset = 0;
+					do {
+						table.getTuplePointer(entry, offset, storedTuplePointer);
+						if (storedTuplePointer.frameIndex < 0)
+							break;
+						tPointers[ptr * 3] = entry;
+						tPointers[ptr * 3 + 1] = offset;
+						table.getTuplePointer(entry, offset, storedTuplePointer);
+						int fIndex = storedTuplePointer.frameIndex;
+						int tIndex = storedTuplePointer.tupleIndex;
+						storedKeysAccessor1.reset(frames.get(fIndex));
+						int tStart = storedKeysAccessor1
+								.getTupleStartOffset(tIndex);
+						int f0StartRel = storedKeysAccessor1
+								.getFieldStartOffset(tIndex, sfIdx);
+						int f0EndRel = storedKeysAccessor1.getFieldEndOffset(
+								tIndex, sfIdx);
+						int f0Start = f0StartRel + tStart
+								+ storedKeysAccessor1.getFieldSlotsLength();
+						tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc
+								.normalize(storedKeysAccessor1.getBuffer()
+										.array(), f0Start, f0EndRel
+										- f0StartRel);
+						ptr++;
+						offset++;
+					} while (true);
+				}
+				/**
+				 * Sort using quick sort
+				 */
+				if (tPointers.length > 0) {
+					sort(tPointers, 0, totalTCount);
+				}
+			}
 
-            @Override
-            public void reset() {
-                lastBufIndex = -1;
-                tPointers = null;
-                table.reset();
-                aggregator.reset();
-            }
+			@Override
+			public void reset() {
+				lastBufIndex = -1;
+				tPointers = null;
+				table.reset();
+				aggregator.reset();
+			}
 
-            @Override
-            public boolean insert(FrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                if (lastBufIndex < 0)
-                    nextAvailableFrame();
-                int entry = tpc.partition(accessor, tIndex, tableSize);
-                boolean foundGroup = false;
-                int offset = 0;
-                do {
-                    table.getTuplePointer(entry, offset++, storedTuplePointer);
-                    if (storedTuplePointer.frameIndex < 0)
-                        break;
-                    storedKeysAccessor1.reset(frames
-                            .get(storedTuplePointer.frameIndex));
-                    int c = ftpcPartial.compare(accessor, tIndex,
-                            storedKeysAccessor1, storedTuplePointer.tupleIndex);
-                    if (c == 0) {
-                        foundGroup = true;
-                        break;
-                    }
-                } while (true);
+			@Override
+			public boolean insert(FrameTupleAccessor accessor, int tIndex)
+					throws HyracksDataException {
+				if (lastBufIndex < 0)
+					nextAvailableFrame();
+				int entry = tpc.partition(accessor, tIndex, tableSize);
+				boolean foundGroup = false;
+				int offset = 0;
+				do {
+					table.getTuplePointer(entry, offset++, storedTuplePointer);
+					if (storedTuplePointer.frameIndex < 0)
+						break;
+					storedKeysAccessor1.reset(frames
+							.get(storedTuplePointer.frameIndex));
+					int c = ftpcPartial.compare(accessor, tIndex,
+							storedKeysAccessor1, storedTuplePointer.tupleIndex);
+					if (c == 0) {
+						foundGroup = true;
+						break;
+					}
+				} while (true);
 
-                if (!foundGroup) {
+				if (!foundGroup) {
 
                     stateTupleBuilder.reset();
 
@@ -272,29 +272,29 @@
                     table.insert(entry, storedTuplePointer);
                 } else {
 
-                    aggregator.aggregate(accessor, tIndex, storedKeysAccessor1,
-                            storedTuplePointer.tupleIndex, aggregateState);
+					aggregator.aggregate(accessor, tIndex, storedKeysAccessor1,
+							storedTuplePointer.tupleIndex, aggregateState);
 
-                }
-                return true;
-            }
+				}
+				return true;
+			}
 
-            @Override
-            public List<ByteBuffer> getFrames() {
-                return frames;
-            }
+			@Override
+			public List<ByteBuffer> getFrames() {
+				return frames;
+			}
 
-            @Override
-            public int getFrameCount() {
-                return lastBufIndex;
-            }
+			@Override
+			public int getFrameCount() {
+				return lastBufIndex;
+			}
 
-            @Override
-            public void flushFrames(IFrameWriter writer, boolean isPartial)
-                    throws HyracksDataException {
-                if (outputFrame == null) {
-                    outputFrame = ctx.allocateFrame();
-                }
+			@Override
+			public void flushFrames(IFrameWriter writer, boolean isPartial)
+					throws HyracksDataException {
+				if (outputFrame == null) {
+					outputFrame = ctx.allocateFrame();
+				}
 
                 if (outputAppender == null) {
                     outputAppender = new FrameTupleAppender(
@@ -305,24 +305,24 @@
 
                 writer.open();
 
-                if (tPointers == null) {
-                    // Not sorted
-                    for (int i = 0; i < tableSize; ++i) {
-                        int entry = i;
-                        int offset = 0;
-                        do {
-                            table.getTuplePointer(entry, offset++,
-                                    storedTuplePointer);
-                            if (storedTuplePointer.frameIndex < 0)
-                                break;
-                            int bIndex = storedTuplePointer.frameIndex;
-                            int tIndex = storedTuplePointer.tupleIndex;
+				if (tPointers == null) {
+					// Not sorted
+					for (int i = 0; i < tableSize; ++i) {
+						int entry = i;
+						int offset = 0;
+						do {
+							table.getTuplePointer(entry, offset++,
+									storedTuplePointer);
+							if (storedTuplePointer.frameIndex < 0)
+								break;
+							int bIndex = storedTuplePointer.frameIndex;
+							int tIndex = storedTuplePointer.tupleIndex;
 
-                            storedKeysAccessor1.reset(frames.get(bIndex));
+							storedKeysAccessor1.reset(frames.get(bIndex));
 
                             outputTupleBuilder.reset();
 
-                            if (isPartial) {
+							if (isPartial) {
 
                                 aggregator.outputPartialResult(
                                         outputTupleBuilder,
@@ -378,7 +378,7 @@
 
                     outputTupleBuilder.reset();
 
-                    if (isPartial) {
+					if (isPartial) {
 
                         aggregator
                                 .outputPartialResult(outputTupleBuilder,
@@ -415,27 +415,27 @@
                 aggregator.close();
             }
 
-            @Override
-            public void close() {
-                lastBufIndex = -1;
-                tPointers = null;
-                table.close();
-                frames.clear();
-                aggregateState.close();
-            }
+			@Override
+			public void close() {
+				lastBufIndex = -1;
+				tPointers = null;
+				table.close();
+				frames.clear();
+				aggregateState.close();
+			}
 
-            /**
-             * Set the working frame to the next available frame in the frame
-             * list. There are two cases:<br>
-             * 1) If the next frame is not initialized, allocate a new frame. 2)
-             * When frames are already created, they are recycled.
-             * 
-             * @return Whether a new frame is added successfully.
-             */
-            private boolean nextAvailableFrame() {
-                // Return false if the number of frames is equal to the limit.
-                if (lastBufIndex + 1 >= framesLimit)
-                    return false;
+			/**
+			 * Set the working frame to the next available frame in the frame
+			 * list. There are two cases:<br>
+			 * 1) If the next frame is not initialized, allocate a new frame. 2)
+			 * When frames are already created, they are recycled.
+			 * 
+			 * @return Whether a new frame is added successfully.
+			 */
+			private boolean nextAvailableFrame() {
+				// Return false if the number of frames is equal to the limit.
+				if (lastBufIndex + 1 >= framesLimit)
+					return false;
 
                 if (frames.size() < framesLimit) {
                     // Insert a new frame
@@ -456,107 +456,107 @@
                 return true;
             }
 
-            private void sort(int[] tPointers, int offset, int length) {
-                int m = offset + (length >> 1);
-                int mTable = tPointers[m * 3];
-                int mRow = tPointers[m * 3 + 1];
-                int mNormKey = tPointers[m * 3 + 2];
+			private void sort(int[] tPointers, int offset, int length) {
+				int m = offset + (length >> 1);
+				int mTable = tPointers[m * 3];
+				int mRow = tPointers[m * 3 + 1];
+				int mNormKey = tPointers[m * 3 + 2];
 
-                table.getTuplePointer(mTable, mRow, storedTuplePointer);
-                int mFrame = storedTuplePointer.frameIndex;
-                int mTuple = storedTuplePointer.tupleIndex;
-                storedKeysAccessor1.reset(frames.get(mFrame));
+				table.getTuplePointer(mTable, mRow, storedTuplePointer);
+				int mFrame = storedTuplePointer.frameIndex;
+				int mTuple = storedTuplePointer.tupleIndex;
+				storedKeysAccessor1.reset(frames.get(mFrame));
 
-                int a = offset;
-                int b = a;
-                int c = offset + length - 1;
-                int d = c;
-                while (true) {
-                    while (b <= c) {
-                        int bTable = tPointers[b * 3];
-                        int bRow = tPointers[b * 3 + 1];
-                        int bNormKey = tPointers[b * 3 + 2];
-                        int cmp = 0;
-                        if (bNormKey != mNormKey) {
-                            cmp = ((((long) bNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
-                                    : 1;
-                        } else {
-                            table.getTuplePointer(bTable, bRow,
-                                    storedTuplePointer);
-                            int bFrame = storedTuplePointer.frameIndex;
-                            int bTuple = storedTuplePointer.tupleIndex;
-                            storedKeysAccessor2.reset(frames.get(bFrame));
-                            cmp = ftpcTuple.compare(storedKeysAccessor2,
-                                    bTuple, storedKeysAccessor1, mTuple);
-                        }
-                        if (cmp > 0) {
-                            break;
-                        }
-                        if (cmp == 0) {
-                            swap(tPointers, a++, b);
-                        }
-                        ++b;
-                    }
-                    while (c >= b) {
-                        int cTable = tPointers[c * 3];
-                        int cRow = tPointers[c * 3 + 1];
-                        int cNormKey = tPointers[c * 3 + 2];
-                        int cmp = 0;
-                        if (cNormKey != mNormKey) {
-                            cmp = ((((long) cNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
-                                    : 1;
-                        } else {
-                            table.getTuplePointer(cTable, cRow,
-                                    storedTuplePointer);
-                            int cFrame = storedTuplePointer.frameIndex;
-                            int cTuple = storedTuplePointer.tupleIndex;
-                            storedKeysAccessor2.reset(frames.get(cFrame));
-                            cmp = ftpcTuple.compare(storedKeysAccessor2,
-                                    cTuple, storedKeysAccessor1, mTuple);
-                        }
-                        if (cmp < 0) {
-                            break;
-                        }
-                        if (cmp == 0) {
-                            swap(tPointers, c, d--);
-                        }
-                        --c;
-                    }
-                    if (b > c)
-                        break;
-                    swap(tPointers, b++, c--);
-                }
+				int a = offset;
+				int b = a;
+				int c = offset + length - 1;
+				int d = c;
+				while (true) {
+					while (b <= c) {
+						int bTable = tPointers[b * 3];
+						int bRow = tPointers[b * 3 + 1];
+						int bNormKey = tPointers[b * 3 + 2];
+						int cmp = 0;
+						if (bNormKey != mNormKey) {
+							cmp = ((((long) bNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
+									: 1;
+						} else {
+							table.getTuplePointer(bTable, bRow,
+									storedTuplePointer);
+							int bFrame = storedTuplePointer.frameIndex;
+							int bTuple = storedTuplePointer.tupleIndex;
+							storedKeysAccessor2.reset(frames.get(bFrame));
+							cmp = ftpcTuple.compare(storedKeysAccessor2,
+									bTuple, storedKeysAccessor1, mTuple);
+						}
+						if (cmp > 0) {
+							break;
+						}
+						if (cmp == 0) {
+							swap(tPointers, a++, b);
+						}
+						++b;
+					}
+					while (c >= b) {
+						int cTable = tPointers[c * 3];
+						int cRow = tPointers[c * 3 + 1];
+						int cNormKey = tPointers[c * 3 + 2];
+						int cmp = 0;
+						if (cNormKey != mNormKey) {
+							cmp = ((((long) cNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1
+									: 1;
+						} else {
+							table.getTuplePointer(cTable, cRow,
+									storedTuplePointer);
+							int cFrame = storedTuplePointer.frameIndex;
+							int cTuple = storedTuplePointer.tupleIndex;
+							storedKeysAccessor2.reset(frames.get(cFrame));
+							cmp = ftpcTuple.compare(storedKeysAccessor2,
+									cTuple, storedKeysAccessor1, mTuple);
+						}
+						if (cmp < 0) {
+							break;
+						}
+						if (cmp == 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);
+				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);
-                }
-            }
+				if ((s = b - a) > 1) {
+					sort(tPointers, offset, s);
+				}
+				if ((s = d - c) > 1) {
+					sort(tPointers, n - s, s);
+				}
+			}
 
-            private void swap(int x[], int a, int b) {
-                for (int i = 0; i < 3; ++i) {
-                    int t = x[a * 3 + i];
-                    x[a * 3 + i] = x[b * 3 + i];
-                    x[b * 3 + i] = t;
-                }
-            }
+			private void swap(int x[], int a, int b) {
+				for (int i = 0; i < 3; ++i) {
+					int t = x[a * 3 + i];
+					x[a * 3 + i] = x[b * 3 + i];
+					x[b * 3 + i] = t;
+				}
+			}
 
-            private void vecswap(int x[], int a, int b, int n) {
-                for (int i = 0; i < n; i++, a++, b++) {
-                    swap(x, a, b);
-                }
-            }
+			private void vecswap(int x[], int a, int b, int n) {
+				for (int i = 0; i < n; i++, a++, b++) {
+					swap(x, a, b);
+				}
+			}
 
-        };
-    }
+		};
+	}
 
 }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
index 226642f..ee8489a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
@@ -27,145 +27,157 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class PreclusteredGroupWriter implements IFrameWriter {
-    private final int[] groupFields;
-    private final IBinaryComparator[] comparators;
-    private final IAggregatorDescriptor aggregator;
-    private final AggregateState aggregateState;
-    private final IFrameWriter writer;
-    private final ByteBuffer copyFrame;
-    private final FrameTupleAccessor inFrameAccessor;
-    private final FrameTupleAccessor copyFrameAccessor;
+	private final int[] groupFields;
+	private final IBinaryComparator[] comparators;
+	private final IAggregatorDescriptor aggregator;
+	private final AggregateState aggregateState;
+	private final IFrameWriter writer;
+	private final ByteBuffer copyFrame;
+	private final FrameTupleAccessor inFrameAccessor;
+	private final FrameTupleAccessor copyFrameAccessor;
 
-    private final ByteBuffer outFrame;
-    private final FrameTupleAppender appender;
-    private final ArrayTupleBuilder tupleBuilder;
+	private final ByteBuffer outFrame;
+	private final FrameTupleAppender appender;
+	private final ArrayTupleBuilder tupleBuilder;
 
-    private boolean first;
+	private boolean first;
 
-    public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields,
-            IBinaryComparator[] comparators, IAggregatorDescriptor aggregator,
-            RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, IFrameWriter writer) {
-        this.groupFields = groupFields;
-        this.comparators = comparators;
-        this.aggregator = aggregator;
-        this.aggregateState = aggregator.createAggregateStates();
-        this.writer = writer;
-        copyFrame = ctx.allocateFrame();
-        inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
-                inRecordDesc);
-        copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
-                inRecordDesc);
-        copyFrameAccessor.reset(copyFrame);
+	public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields,
+			IBinaryComparator[] comparators, IAggregatorDescriptor aggregator,
+			RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
+			IFrameWriter writer) {
+		this.groupFields = groupFields;
+		this.comparators = comparators;
+		this.aggregator = aggregator;
+		this.aggregateState = aggregator.createAggregateStates();
+		this.writer = writer;
+		copyFrame = ctx.allocateFrame();
+		inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+				inRecordDesc);
+		copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+				inRecordDesc);
+		copyFrameAccessor.reset(copyFrame);
 
-        outFrame = ctx.allocateFrame();
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(outFrame, true);
-        
-        tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
-    }
+		outFrame = ctx.allocateFrame();
+		appender = new FrameTupleAppender(ctx.getFrameSize());
+		appender.reset(outFrame, true);
 
-    @Override
-    public void open() throws HyracksDataException {
-        writer.open();
-        first = true;
-    }
+		tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
+	}
 
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        inFrameAccessor.reset(buffer);
-        int nTuples = inFrameAccessor.getTupleCount();
-        for (int i = 0; i < nTuples; ++i) {
-            if (first) {
+	@Override
+	public void open() throws HyracksDataException {
+		writer.open();
+		first = true;
+	}
 
-                tupleBuilder.reset();
-                
-                aggregator.init(tupleBuilder, inFrameAccessor, i, aggregateState);
+	@Override
+	public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+		inFrameAccessor.reset(buffer);
+		int nTuples = inFrameAccessor.getTupleCount();
+		for (int i = 0; i < nTuples; ++i) {
+			if (first) {
 
-                first = false;
+				tupleBuilder.reset();
+				for (int j = 0; j < groupFields.length; j++) {
+					tupleBuilder.addField(inFrameAccessor, i, j);
+				}
+				aggregator.init(tupleBuilder, inFrameAccessor, i,
+						aggregateState);
 
-            } else {
-                if (i == 0) {
-                    switchGroupIfRequired(copyFrameAccessor,
-                            copyFrameAccessor.getTupleCount() - 1,
-                            inFrameAccessor, i);
-                } else {
-                    switchGroupIfRequired(inFrameAccessor, i - 1,
-                            inFrameAccessor, i);
-                }
+				first = false;
 
-            }
-        }
-        FrameUtils.copy(buffer, copyFrame);
-    }
+			} else {
+				if (i == 0) {
+					switchGroupIfRequired(copyFrameAccessor,
+							copyFrameAccessor.getTupleCount() - 1,
+							inFrameAccessor, i);
+				} else {
+					switchGroupIfRequired(inFrameAccessor, i - 1,
+							inFrameAccessor, i);
+				}
 
-    private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor,
-            int prevTupleIndex, FrameTupleAccessor currTupleAccessor,
-            int currTupleIndex) throws HyracksDataException {
-        if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor,
-                currTupleIndex)) {
-            writeOutput(prevTupleAccessor, prevTupleIndex);
+			}
+		}
+		FrameUtils.copy(buffer, copyFrame);
+	}
 
-            tupleBuilder.reset();
-            aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex,
-                    aggregateState);
-        } else {
-            aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0,
-                    aggregateState);
-        }
-    }
+	private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor,
+			int prevTupleIndex, FrameTupleAccessor currTupleAccessor,
+			int currTupleIndex) throws HyracksDataException {
+		if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor,
+				currTupleIndex)) {
+			writeOutput(prevTupleAccessor, prevTupleIndex);
 
-    private void writeOutput(final FrameTupleAccessor lastTupleAccessor,
-            int lastTupleIndex) throws HyracksDataException {
+			tupleBuilder.reset();
+			for (int j = 0; j < groupFields.length; j++) {
+				tupleBuilder.addField(currTupleAccessor, currTupleIndex, j);
+			}
+			aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex,
+					aggregateState);
+		} else {
+			aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0,
+					aggregateState);
+		}
+	}
 
-        tupleBuilder.reset();
+	private void writeOutput(final FrameTupleAccessor lastTupleAccessor,
+			int lastTupleIndex) throws HyracksDataException {
 
-        aggregator.outputFinalResult(tupleBuilder,
-                lastTupleAccessor, lastTupleIndex, aggregateState);
-        
-        if(!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
-            FrameUtils.flushFrame(outFrame, writer);
-            appender.reset(outFrame, true);
-            if(!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize()))  {
-                throw new HyracksDataException(
-                        "The output cannot be fit into a frame.");
-            }
-        }
+		tupleBuilder.reset();
+		for (int j = 0; j < groupFields.length; j++) {
+			tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, j);
+		}
+		aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor,
+				lastTupleIndex, aggregateState);
 
-    }
+		if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
+				tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+			FrameUtils.flushFrame(outFrame, writer);
+			appender.reset(outFrame, true);
+			if (!appender.appendSkipEmptyField(
+					tupleBuilder.getFieldEndOffsets(),
+					tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+				throw new HyracksDataException(
+						"The output cannot be fit into a frame.");
+			}
+		}
 
-    private boolean sameGroup(FrameTupleAccessor a1, int t1Idx,
-            FrameTupleAccessor a2, int t2Idx) {
-        for (int i = 0; i < comparators.length; ++i) {
-            int fIdx = groupFields[i];
-            int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength()
-                    + a1.getFieldStartOffset(t1Idx, fIdx);
-            int l1 = a1.getFieldLength(t1Idx, fIdx);
-            int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength()
-                    + a2.getFieldStartOffset(t2Idx, fIdx);
-            int l2 = a2.getFieldLength(t2Idx, fIdx);
-            if (comparators[i].compare(a1.getBuffer().array(), s1, l1, a2
-                    .getBuffer().array(), s2, l2) != 0) {
-                return false;
-            }
-        }
-        return true;
-    }
+	}
 
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
+	private boolean sameGroup(FrameTupleAccessor a1, int t1Idx,
+			FrameTupleAccessor a2, int t2Idx) {
+		for (int i = 0; i < comparators.length; ++i) {
+			int fIdx = groupFields[i];
+			int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength()
+					+ a1.getFieldStartOffset(t1Idx, fIdx);
+			int l1 = a1.getFieldLength(t1Idx, fIdx);
+			int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength()
+					+ a2.getFieldStartOffset(t2Idx, fIdx);
+			int l2 = a2.getFieldLength(t2Idx, fIdx);
+			if (comparators[i].compare(a1.getBuffer().array(), s1, l1, a2
+					.getBuffer().array(), s2, l2) != 0) {
+				return false;
+			}
+		}
+		return true;
+	}
 
-    @Override
-    public void close() throws HyracksDataException {
-        if (!first) {
-            writeOutput(copyFrameAccessor,
-                    copyFrameAccessor.getTupleCount() - 1);
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(outFrame, writer);
-            }
-        }
-        aggregateState.close();
-        writer.close();
-    }
+	@Override
+	public void fail() throws HyracksDataException {
+		writer.fail();
+	}
+
+	@Override
+	public void close() throws HyracksDataException {
+		if (!first) {
+			writeOutput(copyFrameAccessor,
+					copyFrameAccessor.getTupleCount() - 1);
+			if (appender.getTupleCount() > 0) {
+				FrameUtils.flushFrame(outFrame, writer);
+			}
+		}
+		aggregateState.close();
+		writer.close();
+	}
 }
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks-examples/btree-example/btreeclient/pom.xml
index a95d17e..e0930b2 100644
--- a/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -2,8 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   <artifactId>btreeclient</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>btree-example</artifactId>
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 4711e9e..5a6d8cd 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -18,21 +18,21 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 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.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -63,7 +63,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -86,7 +86,7 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(options);
 
@@ -134,16 +134,16 @@
         // prepare insertion into primary index
         // tuples to be put into B-Tree shall have 4 fields
         int primaryFieldCount = 4;
-        ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
-        primaryTypeTraits[0] = new TypeTrait(4);
-        primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        primaryTypeTraits[2] = new TypeTrait(4);
-        primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+        primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
 
         // comparator factories for primary index
         IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[1];
-        primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-        
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
         ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(primaryTupleWriterFactory);
@@ -168,15 +168,15 @@
         // prepare insertion into secondary index
         // tuples to be put into B-Tree shall have 2 fields
         int secondaryFieldCount = 2;
-        ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
-        secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        secondaryTypeTraits[1] = new TypeTrait(4);
+        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
 
         // comparator factories for secondary index
         IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
-        secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-        secondaryComparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-        
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
         ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
@@ -202,7 +202,7 @@
         // distribute the records from the datagen via hashing to the bulk load
         // ops
         IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
-        hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
+        hashFactories[0] = PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY);
         IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
 
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 9790496..0b1cd6c 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -17,20 +17,21 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 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.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -60,7 +61,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -83,7 +84,7 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(options);
 
@@ -130,18 +131,18 @@
         int[] sortFields = { 2 };
         // comparators for sort fields
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+        comparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, options.sbSize, sortFields,
                 comparatorFactories, recDesc);
         JobHelper.createPartitionConstraint(spec, sorter, splitNCs);
 
         // tuples to be put into B-Tree shall have 4 fields
         int fieldCount = 4;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = new TypeTrait(4);
-        typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        typeTraits[2] = new TypeTrait(4);
-        typeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
 
         // create factories and providers for B-Tree
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
@@ -165,7 +166,7 @@
         // distribute the records from the datagen via hashing to the bulk load
         // ops
         IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
-        hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
+        hashFactories[0] = PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY);
         IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
 
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 4929243..aa73a62 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -19,18 +19,19 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -59,7 +60,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -76,7 +77,7 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(options);
 
@@ -95,16 +96,16 @@
         String[] splitNCs = options.ncs.split(",");
 
         int fieldCount = 4;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = new TypeTrait(4);
-        typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        typeTraits[2] = new TypeTrait(4);
-        typeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
 
         // comparators for btree
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-        
+        comparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         // create factories and providers for B-Tree
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index d77eb01..ad9ad3a 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -17,17 +17,17 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -56,7 +56,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -79,7 +79,7 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(options);
 
@@ -111,17 +111,17 @@
                 IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int primaryFieldCount = 4;
-        ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
-        primaryTypeTraits[0] = new TypeTrait(4);
-        primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        primaryTypeTraits[2] = new TypeTrait(4);
-        primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+        primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
 
         // comparators for sort fields and BTree fields
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[2];
-        comparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-        comparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-        
+        comparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        comparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         // create factories and providers for primary B-Tree
         TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
         ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(primaryTupleWriterFactory);
@@ -144,9 +144,9 @@
 
         // tuples to be put into B-Tree shall have 2 fields
         int secondaryFieldCount = 2;
-        ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
-        secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        secondaryTypeTraits[1] = new TypeTrait(4);
+        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 3a79df6..2719397 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -19,19 +19,19 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -60,7 +60,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -80,7 +80,7 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(options);
 
@@ -106,19 +106,19 @@
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int secondaryFieldCount = 2;
-        ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
-        secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        secondaryTypeTraits[1] = new TypeTrait(4);
+        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
 
         // comparators for sort fields and BTree fields
         IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
-        secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-        secondaryComparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-        
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         // comparators for primary index
         IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[1];
-        primaryComparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-        
+        primaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
         ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
@@ -131,11 +131,11 @@
                 IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE, });
 
         int primaryFieldCount = 4;
-        ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
-        primaryTypeTraits[0] = new TypeTrait(4);
-        primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-        primaryTypeTraits[2] = new TypeTrait(4);
-        primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+        primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -147,13 +147,13 @@
         // i.e. we will have a range condition on the first field only (implying
         // [-infinity, +infinity] for the second field)
         IBinaryComparatorFactory[] searchComparatorFactories = new IBinaryComparatorFactory[1];
-        searchComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-        
+        searchComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
         // build tuple containing low and high search keys
         ArrayTupleBuilder tb = new ArrayTupleBuilder(searchComparatorFactories.length * 2); // low
-                                                                                      // and
-                                                                                      // high
-                                                                                      // key
+        // and
+        // high
+        // key
         DataOutput dos = tb.getDataOutput();
 
         tb.reset();
diff --git a/hyracks-examples/btree-example/btreehelper/pom.xml b/hyracks-examples/btree-example/btreehelper/pom.xml
index 2615dc2..046d762 100644
--- a/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -2,8 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   <artifactId>btreehelper</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>btree-example</artifactId>
@@ -29,6 +27,11 @@
   		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-data-std</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
   <build>
     <plugins>
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index 35deb57..36e315d 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -58,8 +58,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
 
         final ByteBuffer outputFrame = ctx.allocateFrame();
         final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -82,7 +82,7 @@
                     appender.reset(outputFrame, true);
                     for (int i = 0; i < numRecords; i++) {
                         tb.reset();
-                        for (int j = 0; j < recDesc.getFields().length; j++) {
+                        for (int j = 0; j < recDesc.getFieldCount(); j++) {
                             genField(tb, j);
                         }
 
diff --git a/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-examples/hyracks-integration-tests/pom.xml
index 7012cde..d74b384 100644
--- a/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -2,8 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>hyracks-integration-tests</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-examples</artifactId>
@@ -80,5 +78,10 @@
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-data-std</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 38dd051..6ef1740 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -28,12 +28,13 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -76,7 +77,7 @@
 
     // field, type and key declarations for primary index
     private int primaryFieldCount = 6;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private int primaryKeyFieldCount = 1;
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
     private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -98,13 +99,13 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary index
-        primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         loadPrimaryIndexTest();
     }
@@ -131,7 +132,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index a043675..51dbb69 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -28,12 +28,13 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -76,7 +77,7 @@
 
     // field, type and key declarations for primary index
     private int primaryFieldCount = 6;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private int primaryKeyFieldCount = 1;
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
     private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -98,13 +99,13 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary index
-        primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         loadPrimaryIndexTest();
     }
@@ -131,7 +132,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index 51d3db1..a804bae 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -27,11 +27,12 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -73,7 +74,7 @@
 
     // field, type and key declarations for primary index
     private int primaryFieldCount = 6;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private int primaryKeyFieldCount = 1;
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
     private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -95,13 +96,13 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary index
-        primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         loadPrimaryIndexTest();
     }
@@ -128,7 +129,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 3b6b202..a16bbe4 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -28,12 +28,13 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -79,7 +80,7 @@
 
     // field, type and key declarations for primary index
     private int primaryFieldCount = 6;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private int primaryKeyFieldCount = 1;
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
     private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -100,7 +101,7 @@
 
     // field, type and key declarations for secondary indexes
     private int secondaryFieldCount = 2;
-    private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
+    private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
     private int secondaryKeyFieldCount = 2;
     private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
     private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
@@ -121,19 +122,19 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary index
-        primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         // field, type and key declarations for secondary indexes
-        secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-        secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         loadPrimaryIndexTest();
         loadSecondaryIndexTest();
@@ -162,7 +163,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
@@ -211,7 +213,8 @@
 
         // sort based on secondary keys
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, primaryRecDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                primaryRecDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         // load secondary index
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 1e070a8..ef1c6f6 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -28,12 +28,13 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -62,308 +63,238 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
 
-public class BTreeSecondaryIndexSearchOperatorTest extends
-		AbstractIntegrationTest {
-	static {
-		TestStorageManagerComponentHolder.init(8192, 20, 20);
-	}
+public class BTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
+    static {
+        TestStorageManagerComponentHolder.init(8192, 20, 20);
+    }
 
-	private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-	private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-	private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
+    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+    private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
 
-	private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
-			"ddMMyy-hhmmssSS");
-	private final static String sep = System.getProperty("file.separator");
+    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    private final static String sep = System.getProperty("file.separator");
 
-	// field, type and key declarations for primary index
-	private int primaryFieldCount = 6;
-	private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
-	private int primaryKeyFieldCount = 1;
-	private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-	private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
-			primaryTypeTraits);
-	private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
-			primaryTupleWriterFactory);
-	private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
-			primaryTupleWriterFactory);
+    // field, type and key declarations for primary index
+    private int primaryFieldCount = 6;
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+    private int primaryKeyFieldCount = 1;
+    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+    private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+    private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+            primaryTupleWriterFactory);
+    private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
 
-	private static String primaryBtreeName = "primary"
-			+ simpleDateFormat.format(new Date());
-	private static String primaryFileName = System
-			.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
+    private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
+    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
 
-	private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
-			new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-					primaryFileName))) });
+    private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
 
-	private RecordDescriptor primaryRecDesc = new RecordDescriptor(
-			new ISerializerDeserializer[] {
-					UTF8StringSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE });
+    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-	// field, type and key declarations for secondary indexes
-	private int secondaryFieldCount = 2;
-	private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
-	private int secondaryKeyFieldCount = 2;
-	private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
-	private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
-			secondaryTypeTraits);
-	private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
-			secondaryTupleWriterFactory);
-	private ITreeIndexFrameFactory secondaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
-			secondaryTupleWriterFactory);
+    // field, type and key declarations for secondary indexes
+    private int secondaryFieldCount = 2;
+    private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+    private int secondaryKeyFieldCount = 2;
+    private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
+    private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
+            secondaryTypeTraits);
+    private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+            secondaryTupleWriterFactory);
+    private ITreeIndexFrameFactory secondaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(secondaryTupleWriterFactory);
 
-	private static String secondaryBtreeName = "secondary"
-			+ simpleDateFormat.format(new Date());
-	private static String secondaryFileName = System
-			.getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
+    private static String secondaryBtreeName = "secondary" + simpleDateFormat.format(new Date());
+    private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
 
-	private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
-			new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-					secondaryFileName))) });
+    private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
 
-	private RecordDescriptor secondaryRecDesc = new RecordDescriptor(
-			new ISerializerDeserializer[] {
-					UTF8StringSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE });
+    private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-	@Before
-	public void setup() throws Exception {
-		// field, type and key declarations for primary index
-		primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+    @Before
+    public void setup() throws Exception {
+        // field, type and key declarations for primary index
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-		// field, type and key declarations for secondary indexes
-		secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-		secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-		secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        // field, type and key declarations for secondary indexes
+        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-		loadPrimaryIndexTest();
-		loadSecondaryIndexTest();
-	}
+        loadPrimaryIndexTest();
+        loadSecondaryIndexTest();
+    }
 
-	public void loadPrimaryIndexTest() throws Exception {
-		JobSpecification spec = new JobSpecification();
+    public void loadPrimaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
 
-		FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
-				new FileReference(new File("data/tpch0.001/orders-part1.tbl"))) };
-		IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
-				ordersSplits);
-		RecordDescriptor ordersDesc = new RecordDescriptor(
-				new ISerializerDeserializer[] {
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE });
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/orders-part1.tbl"))) };
+        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
-		FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
-				spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
-						new IValueParserFactory[] {
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE,
-								UTF8StringParserFactory.INSTANCE }, '|'),
-				ordersDesc);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				ordScanner, NC1_ID);
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
-		ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-				spec,
-				1000,
-				new int[] { 0 },
-				new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-				ordersDesc);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-				NC1_ID);
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
-		int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-		TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
-				spec, storageManager, indexRegistryProvider,
-				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
-				primaryLeafFrameFactory, primaryTypeTraits,
-				primaryComparatorFactories, fieldPermutation, 0.7f,
-				dataflowHelperFactory);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				primaryBtreeBulkLoad, NC1_ID);
+        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
+        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+                primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+                dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
 
-		spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
-				sorter, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
-		spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
-				primaryBtreeBulkLoad, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
 
-		spec.addRoot(primaryBtreeBulkLoad);
-		runTest(spec);
-	}
+        spec.addRoot(primaryBtreeBulkLoad);
+        runTest(spec);
+    }
 
-	public void loadSecondaryIndexTest() throws Exception {
-		JobSpecification spec = new JobSpecification();
+    public void loadSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
 
-		// build dummy tuple containing nothing
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
-		DataOutput dos = tb.getDataOutput();
+        // build dummy tuple containing nothing
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
 
-		tb.reset();
-		UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
-		tb.addFieldEndOffset();
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+        tb.addFieldEndOffset();
 
-		ISerializerDeserializer[] keyRecDescSers = {
-				UTF8StringSerializerDeserializer.INSTANCE,
-				UTF8StringSerializerDeserializer.INSTANCE };
-		RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
 
-		ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
-				spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
-				tb.getSize());
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				keyProviderOp, NC1_ID);
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
 
-		int[] lowKeyFields = null; // - infinity
-		int[] highKeyFields = null; // + infinity
+        int[] lowKeyFields = null; // - infinity
+        int[] highKeyFields = null; // + infinity
 
-		// scan primary index
-		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager,
-				indexRegistryProvider, primaryBtreeSplitProvider,
-				primaryInteriorFrameFactory, primaryLeafFrameFactory,
-				primaryTypeTraits, primaryComparatorFactories, true,
-				lowKeyFields, highKeyFields, true, true, dataflowHelperFactory);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				primaryBtreeSearchOp, NC1_ID);
+        // scan primary index
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+                primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+                highKeyFields, true, true, dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
-		// sort based on secondary keys
-		ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-				spec,
-				1000,
-				new int[] { 3, 0 },
-				new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-				primaryRecDesc);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-				NC1_ID);
+        // sort based on secondary keys
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                primaryRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
-		// load secondary index
-		int[] fieldPermutation = { 3, 0 };
-		TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
-				spec, storageManager, indexRegistryProvider,
-				secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
-				secondaryLeafFrameFactory, secondaryTypeTraits,
-				secondaryComparatorFactories, fieldPermutation, 0.7f,
-				dataflowHelperFactory);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				secondaryBtreeBulkLoad, NC1_ID);
+        // load secondary index
+        int[] fieldPermutation = { 3, 0 };
+        TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
+                secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
+                dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
 
-		spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
-				primaryBtreeSearchOp, 0);
-		spec.connect(new OneToOneConnectorDescriptor(spec),
-				primaryBtreeSearchOp, 0, sorter, 0);
-		spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
-				secondaryBtreeBulkLoad, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, sorter, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, secondaryBtreeBulkLoad, 0);
 
-		spec.addRoot(secondaryBtreeBulkLoad);
-		runTest(spec);
-	}
+        spec.addRoot(secondaryBtreeBulkLoad);
+        runTest(spec);
+    }
 
-	@Test
-	public void searchSecondaryIndexTest() throws Exception {
-		JobSpecification spec = new JobSpecification();
+    @Test
+    public void searchSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
 
-		// build tuple containing search keys (only use the first key as search
-		// key)
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
-		DataOutput dos = tb.getDataOutput();
+        // build tuple containing search keys (only use the first key as search
+        // key)
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
 
-		tb.reset();
-		// low key
-		UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
-		tb.addFieldEndOffset();
-		// high key
-		UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
-		tb.addFieldEndOffset();
+        tb.reset();
+        // low key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
+        tb.addFieldEndOffset();
+        // high key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
+        tb.addFieldEndOffset();
 
-		ISerializerDeserializer[] keyRecDescSers = {
-				UTF8StringSerializerDeserializer.INSTANCE,
-				UTF8StringSerializerDeserializer.INSTANCE };
-		RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
 
-		ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
-				spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
-				tb.getSize());
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				keyProviderOp, NC1_ID);
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
 
-		int[] secondaryLowKeyFields = { 0 };
-		int[] secondaryHighKeyFields = { 1 };
+        int[] secondaryLowKeyFields = { 0 };
+        int[] secondaryHighKeyFields = { 1 };
 
-		// search secondary index
-		BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, secondaryRecDesc, storageManager,
-				indexRegistryProvider, secondaryBtreeSplitProvider,
-				secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
-				secondaryTypeTraits, secondaryComparatorFactories, true,
-				secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-				dataflowHelperFactory);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				secondaryBtreeSearchOp, NC1_ID);
+        // search secondary index
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+                secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
+                secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryTypeTraits,
+                secondaryComparatorFactories, true, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+                dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
 
-		int[] primaryLowKeyFields = { 1 }; // second field from the tuples
-		// coming from secondary index
-		int[] primaryHighKeyFields = { 1 }; // second field from the tuples
-		// coming from secondary index
+        int[] primaryLowKeyFields = { 1 }; // second field from the tuples
+        // coming from secondary index
+        int[] primaryHighKeyFields = { 1 }; // second field from the tuples
+        // coming from secondary index
 
-		// search primary index
-		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager,
-				indexRegistryProvider, primaryBtreeSplitProvider,
-				primaryInteriorFrameFactory, primaryLeafFrameFactory,
-				primaryTypeTraits, primaryComparatorFactories, true,
-				primaryLowKeyFields, primaryHighKeyFields, true, true,
-				dataflowHelperFactory);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				primaryBtreeSearchOp, NC1_ID);
+        // search primary index
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+                primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, primaryLowKeyFields,
+                primaryHighKeyFields, true, true, dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
                 createTempFile().getAbsolutePath()) });
         IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-				NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-		spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
-				secondaryBtreeSearchOp, 0);
-		spec.connect(new OneToOneConnectorDescriptor(spec),
-				secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
-		spec.connect(new OneToOneConnectorDescriptor(spec),
-				primaryBtreeSearchOp, 0, printer, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
 
-		spec.addRoot(printer);
-		runTest(spec);
-	}
+        spec.addRoot(printer);
+        runTest(spec);
+    }
 
-	@AfterClass
-	public static void cleanup() throws Exception {
-		File primary = new File(primaryFileName);
-		primary.deleteOnExit();
-		File secondary = new File(secondaryFileName);
-		secondary.deleteOnExit();
-	}
+    @AfterClass
+    public static void cleanup() throws Exception {
+        File primary = new File(primaryFileName);
+        primary.deleteOnExit();
+        File secondary = new File(secondaryFileName);
+        secondary.deleteOnExit();
+    }
 }
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index abedf4a..fac141c 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -28,7 +28,7 @@
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
-import edu.uci.ics.hyracks.api.client.HyracksLocalConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -61,7 +61,10 @@
     @BeforeClass
     public static void init() throws Exception {
         CCConfig ccConfig = new CCConfig();
-        ccConfig.port = 39001;
+        ccConfig.clientNetIpAddress = "127.0.0.1";
+        ccConfig.clientNetPort = 39000;
+        ccConfig.clusterNetIpAddress = "127.0.0.1";
+        ccConfig.clusterNetPort = 39001;
         ccConfig.profileDumpPeriod = 10000;
         File outDir = new File("target/ClusterController");
         outDir.mkdirs();
@@ -75,6 +78,7 @@
         NCConfig ncConfig1 = new NCConfig();
         ncConfig1.ccHost = "localhost";
         ncConfig1.ccPort = 39001;
+        ncConfig1.clusterNetIPAddress = "127.0.0.1";
         ncConfig1.dataIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
         nc1 = new NodeControllerService(ncConfig1);
@@ -83,12 +87,13 @@
         NCConfig ncConfig2 = new NCConfig();
         ncConfig2.ccHost = "localhost";
         ncConfig2.ccPort = 39001;
+        ncConfig2.clusterNetIPAddress = "127.0.0.1";
         ncConfig2.dataIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
 
-        hcc = new HyracksLocalConnection(cc);
+        hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
         hcc.createApplication("test", null);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
index 042c919..1c5c014 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
@@ -27,8 +27,9 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -59,61 +60,40 @@
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MinMaxStringFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 
 /**
  *
  */
 public class AggregationTests extends AbstractIntegrationTest {
 
-    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
-             "data/tpch0.001/lineitem.tbl"))) });
+    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
+            new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
 
-    final RecordDescriptor desc = new RecordDescriptor(
-            new ISerializerDeserializer[] {
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE });
+    final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(
-            new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                    FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE, }, '|');
+    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+            UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+            IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+            FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, }, '|');
 
-    private AbstractSingleActivityOperatorDescriptor getPrinter(
-            JobSpecification spec, String prefix) throws IOException {
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+            throws IOException {
 
-        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
-                spec, new ConstantFileSplitProvider(new FileSplit[] {
-                        new FileSplit(NC1_ID, createTempFile()
-                                .getAbsolutePath()),
-                        new FileSplit(NC2_ID, createTempFile()
-                                .getAbsolutePath()) }), "\t");
+        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
+                new ConstantFileSplitProvider(new FileSplit[] {
+                        new FileSplit(NC1_ID, createTempFile().getAbsolutePath()),
+                        new FileSplit(NC2_ID, createTempFile().getAbsolutePath()) }), "\t");
 
         return printer;
     }
@@ -122,49 +102,38 @@
     public void singleKeySumInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -177,64 +146,34 @@
     public void singleKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                spec,
-                4,
-                keyFields,
-                null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
                 desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        
-        spec.connect(conn0, csvScanner, 0, sorter, 0);
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        int[] keyFields = new int[] { 0 };
+
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        spec.connect(conn1, sorter, 0, grouper, 0);
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -247,58 +186,41 @@
     public void singleKeySumExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(2, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        tableSize), true);
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(2, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -311,51 +233,38 @@
     public void singleKeyAvgInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -368,66 +277,34 @@
     public void singleKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                spec,
-                4,
-                keyFields,
-                null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                desc);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        
-        spec.connect(conn0, csvScanner, 0, sorter, 0);
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        spec.connect(conn1, sorter, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -440,61 +317,43 @@
     public void singleKeyAvgExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new CountFieldAggregatorFactory(false),
-                                new AvgFieldGroupAggregatorFactory(1, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(2, false),
-                                new AvgFieldMergeAggregatorFactory(3, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        tableSize), true);
+                                new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -507,49 +366,38 @@
     public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -562,64 +410,34 @@
     public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                spec,
-                4,
-                keyFields,
-                null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
                 desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        
-        spec.connect(conn0, csvScanner, 0, sorter, 0);
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec);
+        int[] keyFields = new int[] { 0 };
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        spec.connect(conn1, sorter, 0, grouper, 0);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -632,60 +450,42 @@
     public void singleKeyMinMaxStringExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, true) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(2, true,
-                                        true) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        tableSize), true);
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false),
+                        new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -698,53 +498,39 @@
     public void multiKeySumInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -757,67 +543,35 @@
     public void multiKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 8, 0 };
-        
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                spec,
-                4,
-                keyFields,
-                null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE},
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
                 desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE}));
-        
-        spec.connect(conn0, csvScanner, 0, sorter, 0);
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields, new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        int[] keyFields = new int[] { 8, 0 };
+
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        spec.connect(conn1, sorter, 0, grouper, 0);
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -830,64 +584,43 @@
     public void multiKeySumExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
-        int[] keys = new int[] { 0, 1 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(keyFields,
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                new MultiFieldsAggregatorFactory(keys,
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                        UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -900,55 +633,40 @@
     public void multiKeyAvgInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -961,69 +679,36 @@
     public void multiKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 8, 0 };
-        
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                spec,
-                4,
-                keyFields,
-                null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE},
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
                 desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE}));
-        
-        spec.connect(conn0, csvScanner, 0, sorter, 0);
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields, new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec);
+        int[] keyFields = new int[] { 8, 0 };
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        spec.connect(conn1, sorter, 0, grouper, 0);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgInmemGroupTest");
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1036,66 +721,46 @@
     public void multiKeyAvgExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new CountFieldAggregatorFactory(false),
-                                new AvgFieldGroupAggregatorFactory(1, false) }),
-                new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
                                 new IntSumFieldAggregatorFactory(3, false),
-                                new AvgFieldMergeAggregatorFactory(4, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                        UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+                                new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1108,53 +773,39 @@
     public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1167,67 +818,35 @@
     public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 8, 0 };
-        
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                spec,
-                4,
-                keyFields,
-                null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE},
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
                 desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        IConnectorDescriptor conn0 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE}));
-        
-        spec.connect(conn0, csvScanner, 0, sorter, 0);
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields, new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec);
+        int[] keyFields = new int[] { 8, 0 };
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
-        spec.connect(conn1, sorter, 0, grouper, 0);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringPreClusterGroupTest");
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1240,65 +859,44 @@
     public void multiKeyMinMaxStringExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, true) }),
-                new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new MinMaxStringFieldAggregatorFactory(3, true,
-                                        true) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                        UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+                new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(2, false),
+                        new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                                UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index 1c96755..ce72ec5 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -27,9 +27,10 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -69,7 +70,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -77,19 +79,19 @@
         PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
                 spec,
                 new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                 desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID);
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
 
@@ -100,7 +102,8 @@
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, sorter, 0);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -108,7 +111,8 @@
 
         IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn3, group, 0, sorter2, 0);
 
         IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
@@ -138,7 +142,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -146,19 +151,19 @@
         PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
                 spec,
                 new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                 desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID, NC2_ID);
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
@@ -169,7 +174,8 @@
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, sorter, 0);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -177,7 +183,8 @@
 
         IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn3, group, 0, sorter2, 0);
 
         IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
@@ -207,7 +214,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 3, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -215,19 +223,19 @@
         PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
                 spec,
                 new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                 desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID, NC2_ID);
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
@@ -238,7 +246,8 @@
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, sorter, 0);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -246,7 +255,8 @@
 
         IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn3, group, 0, sorter2, 0);
 
         IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 55a0c1c..9355110 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -26,8 +26,9 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -104,7 +105,8 @@
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, ordScanner, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -140,7 +142,8 @@
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { IntegerBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(IntegerPointable.FACTORY) }));
         spec.connect(conn1, ordScanner, 0, printer, 0);
 
         spec.addRoot(printer);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 57888c8..2c3fddf 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -26,8 +26,9 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -68,7 +69,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -78,10 +80,12 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                new int[] { 1 }, new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }),
-                sorter, 0, printer, 0);
+        spec.connect(
+                new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+                        new int[] { 1 }, new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), new int[] { 1 },
+                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+                                .of(UTF8StringPointable.FACTORY) }), sorter, 0, printer, 0);
 
         runTest(spec);
     }
@@ -110,8 +114,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 4, new int[] { 1, 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -121,11 +125,15 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                new int[] { 1, 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
-                        UTF8StringBinaryHashFunctionFactory.INSTANCE }), new int[] { 1, 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE }), sorter, 0, printer, 0);
+        spec.connect(
+                new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(new int[] {
+                        1, 0 }, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
+                        new IBinaryComparatorFactory[] {
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
+                printer, 0);
 
         runTest(spec);
     }
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 2fbf6d2..61d4696 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -32,8 +32,9 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -138,9 +139,13 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
-        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
-                new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -211,10 +216,17 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
-        GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 4, 10, 200, 1.2,
-                new int[] { 1 }, new int[] { 0 },
-                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+        GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(
+                spec,
+                4,
+                10,
+                200,
+                1.2,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -285,10 +297,17 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
-        HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
-                new int[] { 1 }, new int[] { 0 },
-                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+        HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(
+                spec,
+                5,
+                20,
+                200,
+                1.2,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -359,15 +378,18 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
-        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFields().length];
+        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
         for (int j = 0; j < nullWriterFactories.length; j++) {
             nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
         }
 
-        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 0 },
-                new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, true,
-                nullWriterFactories, 128);
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+                spec,
+                new int[] { 0 },
+                new int[] { 1 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, true, nullWriterFactories, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -438,16 +460,22 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
-        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFields().length];
+        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
         for (int j = 0; j < nullWriterFactories.length; j++) {
             nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
         }
 
-        GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
-                new int[] { 0 }, new int[] { 1 },
-                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, true,
-                nullWriterFactories);
+        GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(
+                spec,
+                5,
+                20,
+                200,
+                1.2,
+                new int[] { 0 },
+                new int[] { 1 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -518,16 +546,22 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
-        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFields().length];
+        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
         for (int j = 0; j < nullWriterFactories.length; j++) {
             nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
         }
 
-        HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
-                new int[] { 0 }, new int[] { 1 },
-                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, true,
-                nullWriterFactories);
+        HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(
+                spec,
+                5,
+                20,
+                200,
+                1.2,
+                new int[] { 0 },
+                new int[] { 1 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -600,9 +634,13 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
-        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
-                new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -612,12 +650,14 @@
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
         IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
         IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -679,10 +719,17 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
-        GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 3, 20, 100, 1.2,
-                new int[] { 1 }, new int[] { 0 },
-                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+        GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(
+                spec,
+                3,
+                20,
+                100,
+                1.2,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -692,12 +739,14 @@
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
         IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
         IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -759,10 +808,17 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
-        HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 3, 20, 100, 1.2,
-                new int[] { 1 }, new int[] { 0 },
-                new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
+        HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(
+                spec,
+                3,
+                20,
+                100,
+                1.2,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -772,12 +828,14 @@
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
         IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
         IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -839,9 +897,13 @@
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
-        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
-                new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, 128);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -851,12 +913,14 @@
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
         IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
         IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
@@ -924,9 +988,13 @@
         MaterializingOperatorDescriptor custMat = new MaterializingOperatorDescriptor(spec, custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custMat, NC1_ID, NC2_ID);
 
-        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
-                new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new int[] { 0 },
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -936,12 +1004,14 @@
 
         IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordPartConn, ordScanner, 0, ordMat, 0);
 
         IConnectorDescriptor custPartConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custPartConn, custScanner, 0, custMat, 0);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 135f1fe..6411390 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -30,7 +30,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
@@ -163,7 +164,7 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 4);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -237,7 +238,7 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 5);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -311,7 +312,7 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 6);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
index 6440db3..298c146 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -27,14 +27,15 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -99,7 +100,7 @@
 
     // Primary BTree index.
     private int primaryFieldCount = 2;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private int primaryKeyFieldCount = 1;
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
     private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
@@ -110,12 +111,12 @@
             IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
     // Inverted index BTree dictionary.
-    private ITypeTrait[] tokenTypeTraits = new ITypeTrait[1];
+    private ITypeTraits[] tokenTypeTraits = new ITypeTraits[1];
     private IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[1];
 
     // Inverted index stuff.
     private int invListElementFieldCount = 1;
-    private ITypeTrait[] invListsTypeTraits = new ITypeTrait[invListElementFieldCount];
+    private ITypeTraits[] invListsTypeTraits = new ITypeTraits[invListElementFieldCount];
     private IBinaryComparatorFactory[] invListsComparatorFactories = new IBinaryComparatorFactory[invListElementFieldCount];
     private RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
             UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -130,17 +131,17 @@
     @Before
     public void setup() throws Exception {
         // Field declarations and comparators for primary BTree index.
-        primaryTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
         // Field declarations and comparators for tokens.
-        tokenTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        tokenComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        tokenTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        tokenComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         // Field declarations and comparators for inverted lists.
-        invListsTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        invListsComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+        invListsTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        invListsComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
         loadPrimaryIndex();
         printPrimaryIndex();
@@ -232,8 +233,8 @@
     private IOperatorDescriptor createExternalSortOp(JobSpecification spec, int[] sortFields,
             RecordDescriptor outputRecDesc) {
         ExternalSortOperatorDescriptor externalSortOp = new ExternalSortOperatorDescriptor(spec, 1000, sortFields,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-                        IntegerBinaryComparatorFactory.INSTANCE }, outputRecDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, outputRecDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, externalSortOp, NC1_ID);
         return externalSortOp;
     }
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 74dc4a8..0c0df42 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -28,12 +28,14 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -80,7 +82,7 @@
     // field, type and key declarations for primary R-tree index
     private int primaryFieldCount = 5;
     private int primaryKeyFieldCount = 4;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
 
     private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
@@ -103,18 +105,18 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary R-tree index
-        primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
         primaryComparatorFactories[1] = primaryComparatorFactories[0];
         primaryComparatorFactories[2] = primaryComparatorFactories[0];
         primaryComparatorFactories[3] = primaryComparatorFactories[0];
 
         IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
-                .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
+                .createPrimitiveValueProviderFactories(primaryComparatorFactories.length, DoublePointable.FACTORY);
 
         primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
                 primaryValueProviderFactories);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index 62fc150..a4912e8 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -27,11 +27,12 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -77,7 +78,7 @@
     // field, type and key declarations for primary R-tree index
     private int primaryFieldCount = 5;
     private int primaryKeyFieldCount = 4;
-    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
     private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
 
     private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
@@ -100,18 +101,18 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary R-tree index
-        primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+        primaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = DoublePointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
         primaryComparatorFactories[1] = primaryComparatorFactories[0];
         primaryComparatorFactories[2] = primaryComparatorFactories[0];
         primaryComparatorFactories[3] = primaryComparatorFactories[0];
 
         IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
-                .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
+                .createPrimitiveValueProviderFactories(primaryComparatorFactories.length, DoublePointable.FACTORY);
 
         primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
                 primaryValueProviderFactories);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 300a726..025f675 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -28,13 +28,14 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
@@ -88,7 +89,7 @@
 
     // field, type and key declarations for primary B-tree index
     private int primaryBTreeFieldCount = 10;
-    private ITypeTrait[] primaryBTreeTypeTraits = new ITypeTrait[primaryBTreeFieldCount];
+    private ITypeTraits[] primaryBTreeTypeTraits = new ITypeTraits[primaryBTreeFieldCount];
     private int primaryBTreeKeyFieldCount = 1;
     private IBinaryComparatorFactory[] primaryBTreeComparatorFactories = new IBinaryComparatorFactory[primaryBTreeKeyFieldCount];
     private TypeAwareTupleWriterFactory primaryBTreeTupleWriterFactory = new TypeAwareTupleWriterFactory(
@@ -113,7 +114,7 @@
 
     // field, type and key declarations for secondary indexes
     private int secondaryFieldCount = 5;
-    private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
+    private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
     private int secondaryKeyFieldCount = 4;
     private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
     private IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = new IPrimitiveValueProviderFactory[secondaryKeyFieldCount];
@@ -138,25 +139,25 @@
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary B-tree index
-        primaryBTreeTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryBTreeTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryBTreeTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryBTreeTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryBTreeTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryBTreeTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        primaryBTreeTypeTraits[6] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryBTreeTypeTraits[7] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryBTreeTypeTraits[8] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryBTreeTypeTraits[9] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        primaryBTreeComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+        primaryBTreeTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[6] = DoublePointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[7] = DoublePointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[8] = DoublePointable.TYPE_TRAITS;
+        primaryBTreeTypeTraits[9] = DoublePointable.TYPE_TRAITS;
+        primaryBTreeComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
         // field, type and key declarations for secondary indexes
-        secondaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        secondaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        secondaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        secondaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
-        secondaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        secondaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+        secondaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
         secondaryComparatorFactories[1] = secondaryComparatorFactories[0];
         secondaryComparatorFactories[2] = secondaryComparatorFactories[0];
         secondaryComparatorFactories[3] = secondaryComparatorFactories[0];
@@ -166,7 +167,7 @@
         secondaryValueProviderFactories[3] = secondaryValueProviderFactories[0];
 
         IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = RTreeUtils
-                .comparatorFactoriesToPrimitiveValueProviderFactories(secondaryComparatorFactories);
+                .createPrimitiveValueProviderFactories(secondaryComparatorFactories.length, DoublePointable.FACTORY);
 
         secondaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(secondaryTupleWriterFactory,
                 secondaryValueProviderFactories);
@@ -203,7 +204,8 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index 8db3f48..66f7efc 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -19,7 +19,7 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
@@ -31,8 +31,9 @@
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -53,11 +54,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
@@ -98,7 +97,7 @@
 
         @Option(name = "-out-plain", usage = "Whether to output plain text (default: true)", required = false)
         public boolean outPlain = true;
-        
+
         @Option(name = "-algo", usage = "The algorithm to be used", required = true)
         public int algo;
     }
@@ -111,17 +110,15 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host,
-                options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job;
 
         for (int i = 0; i < 6; i++) {
             long start = System.currentTimeMillis();
-            job = createJob(parseFileSplits(options.inFileSplits),
-                    parseFileSplits(options.outFileSplits, i),
-                    options.htSize, options.sbSize, options.framesLimit,
-                    options.sortOutput, options.algo, options.outPlain);
+            job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits, i),
+                    options.htSize, options.sbSize, options.framesLimit, options.sortOutput, options.algo,
+                    options.outPlain);
 
             System.out.print(i + "\t" + (System.currentTimeMillis() - start));
             start = System.currentTimeMillis();
@@ -139,11 +136,9 @@
             String s = splits[i].trim();
             int idx = s.indexOf(':');
             if (idx < 0) {
-                throw new IllegalArgumentException("File split " + s
-                        + " not well formed");
+                throw new IllegalArgumentException("File split " + s + " not well formed");
             }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(
-                    new File(s.substring(idx + 1))));
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
         }
         return fSplits;
     }
@@ -155,70 +150,46 @@
             String s = splits[i].trim();
             int idx = s.indexOf(':');
             if (idx < 0) {
-                throw new IllegalArgumentException("File split " + s
-                        + " not well formed");
+                throw new IllegalArgumentException("File split " + s + " not well formed");
             }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(
-                    new File(s.substring(idx + 1) + "_" + count)));
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1) + "_"
+                    + count)));
         }
         return fSplits;
     }
 
-    private static JobSpecification createJob(FileSplit[] inSplits,
-            FileSplit[] outSplits, int htSize, int sbSize, int framesLimit,
-            boolean sortOutput, int alg, boolean outPlain) {
+    private static JobSpecification createJob(FileSplit[] inSplits, FileSplit[] outSplits, int htSize, int sbSize,
+            int framesLimit, boolean sortOutput, int alg, boolean outPlain) {
         JobSpecification spec = new JobSpecification();
-        IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(
-                inSplits);
+        IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(inSplits);
 
-        RecordDescriptor inDesc = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor inDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        FileScanOperatorDescriptor fileScanner = new FileScanOperatorDescriptor(
-                spec, splitsProvider, new DelimitedDataTupleParserFactory(
-                        new IValueParserFactory[] {
-                                IntegerParserFactory.INSTANCE,
-                                IntegerParserFactory.INSTANCE,
-                                IntegerParserFactory.INSTANCE,
-                                IntegerParserFactory.INSTANCE,
-                                IntegerParserFactory.INSTANCE,
-                                FloatParserFactory.INSTANCE,
-                                FloatParserFactory.INSTANCE,
-                                FloatParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE, }, '|'),
-                inDesc);
+        FileScanOperatorDescriptor fileScanner = new FileScanOperatorDescriptor(spec, splitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+                        IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+                        IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+                        FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, }, '|'), inDesc);
 
         createPartitionConstraint(spec, fileScanner, inSplits);
 
         // Output: each unique string with an integer count
-        RecordDescriptor outDesc = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        IntegerSerializerDeserializer.INSTANCE,
-                        // IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE,
+                // IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
 
         // Specify the grouping key, which will be the string extracted during
         // the scan.
@@ -229,156 +200,126 @@
         AbstractOperatorDescriptor grouper;
 
         switch (alg) {
-        case 0: // new external hash graph
-            grouper = new ExternalGroupOperatorDescriptor(
-                    spec,
-                    keys,
-                    framesLimit,
-                    new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE },
-                    new IntegerNormalizedKeyComputerFactory(),
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(
-                                    false) }),
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(
-                                    keys.length, false) }), outDesc,
-                    new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keys,
-                                    new IBinaryHashFunctionFactory[] {
-                                    // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                    IntegerBinaryHashFunctionFactory.INSTANCE }),
-                            htSize), false);
-            
-            createPartitionConstraint(spec, grouper, outSplits);
+            case 0: // new external hash graph
+                grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(spec, keys,
+                        framesLimit, new IBinaryComparatorFactory[] {
+                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
+                        new MultiFieldsAggregatorFactory(
+                                new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(keys.length,
+                                        false) }), outDesc, new HashSpillableTableFactory(
+                                new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                                // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }), htSize), false);
 
-            // Connect scanner with the grouper
-            IConnectorDescriptor scanGroupConnDef2 = new MToNPartitioningConnectorDescriptor(
-                    spec, new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] {
-                            // IntegerBinaryHashFunctionFactory.INSTANCE,
-                            IntegerBinaryHashFunctionFactory.INSTANCE }));
-            spec.connect(scanGroupConnDef2, fileScanner, 0, grouper, 0);
-            
-            break;
-        case 1: // External-sort + new-precluster
-            ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(
-                    spec, framesLimit, keys, new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE }, inDesc);
-            createPartitionConstraint(spec, sorter2, inSplits);
+                createPartitionConstraint(spec, grouper, outSplits);
 
-            // Connect scan operator with the sorter
-            IConnectorDescriptor scanSortConn2 = new MToNPartitioningConnectorDescriptor(
-                    spec, new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] {
-                            // IntegerBinaryHashFunctionFactory.INSTANCE,
-                            IntegerBinaryHashFunctionFactory.INSTANCE }));
-            spec.connect(scanSortConn2, fileScanner, 0, sorter2, 0);
+                // Connect scanner with the grouper
+                IConnectorDescriptor scanGroupConnDef2 = new MToNPartitioningConnectorDescriptor(spec,
+                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+                spec.connect(scanGroupConnDef2, fileScanner, 0, grouper, 0);
 
-            grouper = new PreclusteredGroupOperatorDescriptor(
-                    spec,
-                    keys,
-                    new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE },
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                    outDesc);
+                break;
+            case 1: // External-sort + new-precluster
+                ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(spec, framesLimit, keys,
+                        new IBinaryComparatorFactory[] {
+                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, inDesc);
+                createPartitionConstraint(spec, sorter2, inSplits);
 
-            createPartitionConstraint(spec, grouper, outSplits);
+                // Connect scan operator with the sorter
+                IConnectorDescriptor scanSortConn2 = new MToNPartitioningConnectorDescriptor(spec,
+                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+                spec.connect(scanSortConn2, fileScanner, 0, sorter2, 0);
 
-            // Connect sorter with the pre-cluster
-            OneToOneConnectorDescriptor sortGroupConn2 = new OneToOneConnectorDescriptor(
-                    spec);
-            spec.connect(sortGroupConn2, sorter2, 0, grouper, 0);
-            break;
-        case 2: // Inmem
-            grouper = new HashGroupOperatorDescriptor(
-                    spec,
-                    keys,
-                    new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] {
-                            // IntegerBinaryHashFunctionFactory.INSTANCE,
-                            IntegerBinaryHashFunctionFactory.INSTANCE }),
-                    new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE },
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                    outDesc, htSize);
+                grouper = new edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor(spec, keys,
+                        new IBinaryComparatorFactory[] {
+                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        new MultiFieldsAggregatorFactory(
+                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
+                        outDesc);
 
-            createPartitionConstraint(spec, grouper, outSplits);
+                createPartitionConstraint(spec, grouper, outSplits);
 
-            // Connect scanner with the grouper
-            IConnectorDescriptor scanConn2 = new MToNPartitioningConnectorDescriptor(
-                    spec, new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] {
-                            // IntegerBinaryHashFunctionFactory.INSTANCE,
-                            IntegerBinaryHashFunctionFactory.INSTANCE }));
-            spec.connect(scanConn2, fileScanner, 0, grouper, 0);
-            break;
-        default:
-            grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(
-                    spec,
-                    keys,
-                    framesLimit,
-                    new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE },
-                    new IntegerNormalizedKeyComputerFactory(),
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(
-                                    false) }),
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(
-                                    keys.length, false) }), outDesc,
-                    new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keys,
-                                    new IBinaryHashFunctionFactory[] {
-                                    // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                    IntegerBinaryHashFunctionFactory.INSTANCE }),
-                            htSize), false);
-            
-            createPartitionConstraint(spec, grouper, outSplits);
+                // Connect sorter with the pre-cluster
+                OneToOneConnectorDescriptor sortGroupConn2 = new OneToOneConnectorDescriptor(spec);
+                spec.connect(sortGroupConn2, sorter2, 0, grouper, 0);
+                break;
+            case 2: // Inmem
+                grouper = new HashGroupOperatorDescriptor(spec, keys, new FieldHashPartitionComputerFactory(keys,
+                        new IBinaryHashFunctionFactory[] {
+                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }),
+                        new IBinaryComparatorFactory[] {
+                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        new MultiFieldsAggregatorFactory(
+                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
+                        outDesc, htSize);
 
-            // Connect scanner with the grouper
-            IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(
-                    spec, new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] {
-                            // IntegerBinaryHashFunctionFactory.INSTANCE,
-                            IntegerBinaryHashFunctionFactory.INSTANCE }));
-            spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
+                createPartitionConstraint(spec, grouper, outSplits);
+
+                // Connect scanner with the grouper
+                IConnectorDescriptor scanConn2 = new MToNPartitioningConnectorDescriptor(spec,
+                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+                spec.connect(scanConn2, fileScanner, 0, grouper, 0);
+                break;
+            default:
+                grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(spec, keys,
+                        framesLimit, new IBinaryComparatorFactory[] {
+                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
+                        new MultiFieldsAggregatorFactory(
+                                new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(keys.length,
+                                        false) }), outDesc, new HashSpillableTableFactory(
+                                new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                                // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }), htSize), false);
+
+                createPartitionConstraint(spec, grouper, outSplits);
+
+                // Connect scanner with the grouper
+                IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(spec,
+                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+                spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
         }
 
-        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(
-                outSplits);
+        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
 
         AbstractSingleActivityOperatorDescriptor writer;
 
         if (outPlain)
-            writer = new PlainFileWriterOperatorDescriptor(spec,
-                    outSplitProvider, "|");
+            writer = new PlainFileWriterOperatorDescriptor(spec, outSplitProvider, "|");
         else
-            writer = new FrameFileWriterOperatorDescriptor(spec,
-                    outSplitProvider);
+            writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
 
         createPartitionConstraint(spec, writer, outSplits);
 
-        IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(
-                spec);
+        IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(groupOutConn, grouper, 0, writer, 0);
 
         spec.addRoot(writer);
         return spec;
     }
 
-    private static void createPartitionConstraint(JobSpecification spec,
-            IOperatorDescriptor op, FileSplit[] splits) {
+    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
         String[] parts = new String[splits.length];
         for (int i = 0; i < splits.length; ++i) {
             parts[i] = splits[i].getNodeName();
         }
-        PartitionConstraintHelper
-                .addAbsoluteLocationConstraint(spec, op, parts);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
     }
 }
\ No newline at end of file
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index 8cac822..1708259 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -20,7 +20,7 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
@@ -33,8 +33,9 @@
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
@@ -62,7 +63,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -94,7 +95,7 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits),
                 options.algo, options.htSize, options.sbSize, options.format);
@@ -140,19 +141,25 @@
         IOperatorDescriptor gBy;
         int[] keys = new int[] { 0 };
         if ("hash".equalsIgnoreCase(algo)) {
-            gBy = new HashGroupOperatorDescriptor(spec, keys, new FieldHashPartitionComputerFactory(keys,
-                    new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                    new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+            gBy = new HashGroupOperatorDescriptor(
+                    spec,
+                    keys,
+                    new FieldHashPartitionComputerFactory(keys,
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }),
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     new MultiFieldsAggregatorFactory(
                             new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                     groupResultDesc, htSize);
             createPartitionConstraint(spec, gBy, outSplits);
             IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
                     new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }));
             spec.connect(scanGroupConn, wordScanner, 0, gBy, 0);
         } else {
-            IBinaryComparatorFactory[] cfs = new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE };
+            IBinaryComparatorFactory[] cfs = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+                    .of(UTF8StringPointable.FACTORY) };
             IOperatorDescriptor sorter = "memsort".equalsIgnoreCase(algo) ? new InMemorySortOperatorDescriptor(spec,
                     keys, new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc)
                     : new ExternalSortOperatorDescriptor(spec, sbSize, keys,
@@ -161,11 +168,14 @@
 
             IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
                     new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }));
             spec.connect(scanSortConn, wordScanner, 0, sorter, 0);
 
-            gBy = new PreclusteredGroupOperatorDescriptor(spec, keys,
-                    new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+            gBy = new PreclusteredGroupOperatorDescriptor(
+                    spec,
+                    keys,
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     new MultiFieldsAggregatorFactory(
                             new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                     groupResultDesc);
diff --git a/hyracks-examples/text-example/texthelper/pom.xml b/hyracks-examples/text-example/texthelper/pom.xml
index 60c423d..b0bfcef 100644
--- a/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks-examples/text-example/texthelper/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.text</groupId>
   <artifactId>texthelper</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
@@ -23,6 +22,11 @@
   		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-data-std</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
   <build>
     <plugins>
diff --git a/hyracks-examples/tpch-example/tpchapp/pom.xml b/hyracks-examples/tpch-example/tpchapp/pom.xml
index c65aaf5..05256a3 100644
--- a/hyracks-examples/tpch-example/tpchapp/pom.xml
+++ b/hyracks-examples/tpch-example/tpchapp/pom.xml
@@ -2,8 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
   <artifactId>tpchapp</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>tpch-example</artifactId>
@@ -83,5 +81,10 @@
         <version>0.2.0-SNAPSHOT</version>
         <scope>compile</scope>
     </dependency>
+    <dependency>
+    	<groupId>edu.uci.ics.hyracks</groupId>
+    	<artifactId>hyracks-data-std</artifactId>
+    	<version>0.2.0-SNAPSHOT</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks-examples/tpch-example/tpchclient/pom.xml
index c7a138a..d9b5fce 100644
--- a/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -2,8 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
   <artifactId>tpchclient</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>tpch-example</artifactId>
@@ -17,6 +15,11 @@
   		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-data-std</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
   <build>
     <plugins>
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 9e88f8c..72533e7 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -20,7 +20,7 @@
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
@@ -38,8 +38,9 @@
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -68,7 +69,7 @@
         public String host;
 
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)", required = false)
-        public int port = 1099;
+        public int port = 1098;
 
         @Option(name = "-app", usage = "Hyracks Application name", required = true)
         public String app;
@@ -117,24 +118,16 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host,
-                options.port);
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
-        JobSpecification job = createJob(
-                parseFileSplits(options.inFileCustomerSplits),
-                parseFileSplits(options.inFileOrderSplits),
-                parseFileSplits(options.outFileSplits),
-                options.numJoinPartitions, options.algo,
-                options.graceInputSize, options.graceRecordsPerFrame,
-                options.graceFactor, options.memSize, options.tableSize,
-                options.hasGroupBy);
+        JobSpecification job = createJob(parseFileSplits(options.inFileCustomerSplits),
+                parseFileSplits(options.inFileOrderSplits), parseFileSplits(options.outFileSplits),
+                options.numJoinPartitions, options.algo, options.graceInputSize, options.graceRecordsPerFrame,
+                options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.createJob(
-                options.app,
-                job,
-                options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet
-                        .noneOf(JobFlag.class));
+        JobId jobId = hcc.createJob(options.app, job,
+                options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
@@ -148,104 +141,65 @@
             String s = splits[i].trim();
             int idx = s.indexOf(':');
             if (idx < 0) {
-                throw new IllegalArgumentException("File split " + s
-                        + " not well formed");
+                throw new IllegalArgumentException("File split " + s + " not well formed");
             }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(
-                    new File(s.substring(idx + 1))));
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
         }
         return fSplits;
     }
 
-    private static JobSpecification createJob(FileSplit[] customerSplits,
-            FileSplit[] orderSplits, FileSplit[] resultSplits,
-            int numJoinPartitions, String algo, int graceInputSize,
-            int graceRecordsPerFrame, double graceFactor, int memSize,
-            int tableSize, boolean hasGroupBy) throws HyracksDataException {
+    private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
+            FileSplit[] resultSplits, int numJoinPartitions, String algo, int graceInputSize, int graceRecordsPerFrame,
+            double graceFactor, int memSize, int tableSize, boolean hasGroupBy) throws HyracksDataException {
         JobSpecification spec = new JobSpecification();
 
-        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(
-                customerSplits);
-        RecordDescriptor custDesc = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
+        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(
-                orderSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
-        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
-                spec, ordersSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         createPartitionConstraint(spec, ordScanner, orderSplits);
 
-        FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(
-                spec, custSplitsProvider, new DelimitedDataTupleParserFactory(
-                        new IValueParserFactory[] {
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE,
-                                UTF8StringParserFactory.INSTANCE }, '|'),
-                custDesc);
+        FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
         createPartitionConstraint(spec, custScanner, customerSplits);
 
         IOperatorDescriptor join;
 
         if ("nestedloop".equalsIgnoreCase(algo)) {
-            join = new NestedLoopJoinOperatorDescriptor(spec,
-                    new JoinComparatorFactory(
-                            UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
-                    custOrderJoinDesc, memSize);
+            join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
+                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize);
 
         } else if ("gracehash".equalsIgnoreCase(algo)) {
             join = new GraceHashJoinOperatorDescriptor(
@@ -256,8 +210,9 @@
                     graceFactor,
                     new int[] { 0 },
                     new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                    new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                            .of(UTF8StringPointable.FACTORY) },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     custOrderJoinDesc);
 
         } else if ("hybridhash".equalsIgnoreCase(algo)) {
@@ -269,8 +224,9 @@
                     graceFactor,
                     new int[] { 0 },
                     new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                    new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                            .of(UTF8StringPointable.FACTORY) },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     custOrderJoinDesc);
 
         } else {
@@ -278,81 +234,71 @@
                     spec,
                     new int[] { 0 },
                     new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
-                    new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                            .of(UTF8StringPointable.FACTORY) },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     custOrderJoinDesc, 6000000);
         }
 
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, join,
-                numJoinPartitions);
+        PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
 
-        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 1);
 
-        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 0);
 
         IOperatorDescriptor endingOp = join;
 
         if (hasGroupBy) {
 
-            RecordDescriptor groupResultDesc = new RecordDescriptor(
-                    new ISerializerDeserializer[] {
-                            UTF8StringSerializerDeserializer.INSTANCE,
-                            IntegerSerializerDeserializer.INSTANCE });
+            RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                    UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
             HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(
                     spec,
                     new int[] { 6 },
-                    new FieldHashPartitionComputerFactory(
-                            new int[] { 6 },
-                            new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                    new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                    new FieldHashPartitionComputerFactory(new int[] { 6 },
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }),
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     new MultiFieldsAggregatorFactory(
                             new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                     groupResultDesc, 16);
             createPartitionConstraint(spec, gby, resultSplits);
 
-            IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(
-                    spec,
-                    new FieldHashPartitionComputerFactory(
-                            new int[] { 6 },
-                            new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
+            IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
+                    new FieldHashPartitionComputerFactory(new int[] { 6 },
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }));
             spec.connect(joinGroupConn, join, 0, gby, 0);
 
             endingOp = gby;
         }
 
-        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(
-                resultSplits);
-        FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(
-                spec, outSplitProvider);
+        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(resultSplits);
+        FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
         createPartitionConstraint(spec, writer, resultSplits);
 
-        IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(
-                spec);
+        IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(endingPrinterConn, endingOp, 0, writer, 0);
 
         spec.addRoot(writer);
         return spec;
     }
 
-    private static void createPartitionConstraint(JobSpecification spec,
-            IOperatorDescriptor op, FileSplit[] splits) {
+    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
         String[] parts = new String[splits.length];
         for (int i = 0; i < splits.length; ++i) {
             parts[i] = splits[i].getNodeName();
         }
-        PartitionConstraintHelper
-                .addAbsoluteLocationConstraint(spec, op, parts);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
     }
 
     static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -362,8 +308,7 @@
         private final int pos0;
         private final int pos1;
 
-        public JoinComparatorFactory(IBinaryComparatorFactory bFactory,
-                int pos0, int pos1) {
+        public JoinComparatorFactory(IBinaryComparatorFactory bFactory, int pos0, int pos1) {
             this.bFactory = bFactory;
             this.pos0 = pos0;
             this.pos1 = pos1;
@@ -371,8 +316,7 @@
 
         @Override
         public ITuplePairComparator createTuplePairComparator() {
-            return new JoinComparator(bFactory.createBinaryComparator(), pos0,
-                    pos1);
+            return new JoinComparator(bFactory.createBinaryComparator(), pos0, pos1);
         }
     }
 
@@ -382,16 +326,14 @@
         private final int field0;
         private final int field1;
 
-        public JoinComparator(IBinaryComparator bComparator, int field0,
-                int field1) {
+        public JoinComparator(IBinaryComparator bComparator, int field0, int field1) {
             this.bComparator = bComparator;
             this.field0 = field0;
             this.field1 = field1;
         }
 
         @Override
-        public int compare(IFrameTupleAccessor accessor0, int tIndex0,
-                IFrameTupleAccessor accessor1, int tIndex1) {
+        public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1) {
             int tStart0 = accessor0.getTupleStartOffset(tIndex0);
             int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
 
@@ -406,9 +348,8 @@
             int fEnd1 = accessor1.getFieldEndOffset(tIndex1, field1);
             int fLen1 = fEnd1 - fStart1;
 
-            int c = bComparator.compare(accessor0.getBuffer().array(), fStart0
-                    + fStartOffset0, fLen0, accessor1.getBuffer().array(),
-                    fStart1 + fStartOffset1, fLen1);
+            int c = bComparator.compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
+                    .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
             if (c != 0) {
                 return c;
             }
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index 9e76a3e..81e9f84 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -5,7 +5,7 @@
 import java.util.Properties;
 import java.util.Set;
 
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -15,7 +15,7 @@
 
 public class HyracksClient {
 
-    private static HyracksRMIConnection connection;
+    private static HyracksConnection connection;
     private static final String jobProfilingKey = "jobProfilingKey";
     Set<String> systemLibs;
 
@@ -25,7 +25,7 @@
 
     private void initialize(Properties properties) throws Exception {
         String clusterController = (String) properties.get(ConfigurationConstants.clusterControllerHost);
-        connection = new HyracksRMIConnection(clusterController, 1099);
+        connection = new HyracksConnection(clusterController, 1099);
         systemLibs = new HashSet<String>();
         for (String systemLib : ConfigurationConstants.systemLibs) {
             String systemLibPath = properties.getProperty(systemLib);
diff --git a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
index 8ed7bb7..4faa619 100644
--- a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
+++ b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
@@ -27,8 +27,11 @@
         @Option(name = "-n", required = false, usage = "Number of node controllers (default: 2)")
         public int n = 2;
 
-        @Option(name = "-cc-port", required = false, usage = "CC Port (default: 1099)")
-        public int ccPort = 1099;
+        @Option(name = "-cc-client-net-port", required = false, usage = "CC Port (default: 1098)")
+        public int ccClientNetPort = 1098;
+
+        @Option(name = "-cc-cluster-net-port", required = false, usage = "CC Port (default: 1099)")
+        public int ccClusterNetPort = 1099;
 
         @Option(name = "-cc-http-port", required = false, usage = "CC Port (default: 19001)")
         public int ccHttpPort = 19001;
@@ -46,7 +49,10 @@
         }
 
         CCConfig ccConfig = new CCConfig();
-        ccConfig.port = options.ccPort;
+        ccConfig.clusterNetIpAddress = "127.0.0.1";
+        ccConfig.clusterNetPort = options.ccClusterNetPort;
+        ccConfig.clientNetIpAddress = "127.0.0.1";
+        ccConfig.clientNetPort = options.ccClientNetPort;
         ccConfig.httpPort = options.ccHttpPort;
         HyracksCCProcess ccp = new HyracksCCProcess(ccConfig);
         ccp.start();
@@ -56,7 +62,9 @@
         HyracksNCProcess ncps[] = new HyracksNCProcess[options.n];
         for (int i = 0; i < options.n; ++i) {
             NCConfig ncConfig = new NCConfig();
-            ncConfig.ccHost = "localhost";
+            ncConfig.ccHost = "127.0.0.1";
+            ncConfig.ccPort = options.ccClusterNetPort;
+            ncConfig.clusterNetIPAddress = "127.0.0.1";
             ncConfig.nodeId = "nc" + i;
             ncConfig.dataIPAddress = "127.0.0.1";
             ncps[i] = new HyracksNCProcess(ncConfig);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index 8e88c18..4486205 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -21,7 +21,7 @@
 import java.util.Comparator;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.btree.api.IPrefixSlotManager;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.impls.FieldPrefixSlotManager;
@@ -41,9 +41,9 @@
     // them
     private int occurrenceThreshold;
 
-    private ITypeTrait[] typeTraits;
+    private ITypeTraits[] typeTraits;
 
-    public FieldPrefixCompressor(ITypeTrait[] typeTraits, float ratioThreshold, int occurrenceThreshold) {
+    public FieldPrefixCompressor(ITypeTraits[] typeTraits, float ratioThreshold, int occurrenceThreshold) {
         this.typeTraits = typeTraits;
         this.ratioThreshold = ratioThreshold;
         this.occurrenceThreshold = occurrenceThreshold;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 4075152..6314260 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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.IFileSplitProvider;
@@ -44,7 +44,7 @@
     public BTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
             IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory) {
         super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index 2be1641..8cde863 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -4,7 +4,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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.IFileSplitProvider;
@@ -24,7 +24,7 @@
     public BTreeUpdateSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
             IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
             ITupleUpdaterFactory tupleUpdaterFactory) {
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 56e0774..0aba70b 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -19,7 +19,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -77,7 +77,7 @@
     public BTreeFieldPrefixNSMLeafFrame(ITreeIndexTupleWriter tupleWriter) {
         this.tupleWriter = tupleWriter;
         this.frameTuple = new FieldPrefixTupleReference(tupleWriter.createTupleReference());
-        ITypeTrait[] typeTraits = ((TypeAwareTupleWriter) tupleWriter).getTypeTraits();
+        ITypeTraits[] typeTraits = ((TypeAwareTupleWriter) tupleWriter).getTypeTraits();
         this.framePrefixTuple = new FieldPrefixPrefixTupleReference(typeTraits);
         this.slotManager = new FieldPrefixSlotManager();
         this.compressor = new FieldPrefixCompressor(typeTraits, 0.001f, 2);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 57f1b79..4113e9f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -58,8 +58,6 @@
     private final static int MAX_RESTARTS = 10;
     private final static int rootPage = 1;
         
-    private boolean created = false;
-
     private final IFreePageManager freePageManager;
     private final IBufferCache bufferCache;    
     private final ITreeIndexFrameFactory interiorFrameFactory;
@@ -86,15 +84,11 @@
     public void create(int fileId) throws HyracksDataException {
         treeLatch.writeLock().lock();
         try {
-            if (created) {
-                return;
-            }
             ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
             ITreeIndexMetaDataFrame metaFrame = freePageManager.getMetaDataFrameFactory().createFrame();
             this.fileId = fileId;
             freePageManager.init(metaFrame, rootPage);
             initRoot(leafFrame, true);
-            created = true;
         } finally {
             treeLatch.writeLock().unlock();
         }
@@ -305,8 +299,9 @@
         insertUpdateOrDelete(tuple, ctx);
     }
     
-    private void insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+    private boolean insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
         ctx.leafFrame.setPage(node);
+        boolean restartOp = false;
         int targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(tuple);
         FrameOpSpaceStatus spaceStatus = ctx.leafFrame.hasSpaceInsert(tuple);
         switch (spaceStatus) {
@@ -324,7 +319,7 @@
                 ctx.splitKey.reset();
                 break;
             }
-            case INSUFFICIENT_SPACE: {
+            case INSUFFICIENT_SPACE: {            	
                 // Try compressing the page first and see if there is space available.
                 boolean reCompressed = ctx.leafFrame.compress();
                 if (reCompressed) {
@@ -336,31 +331,34 @@
                     ctx.leafFrame.insert(tuple, targetTupleIndex);
                     ctx.splitKey.reset();
                 } else {
-                    performLeafSplit(pageId, tuple, ctx);
+                	restartOp = performLeafSplit(pageId, tuple, ctx);
                 }
                 break;
             }
         }
         node.releaseWriteLatch();
         bufferCache.unpin(node);
+        return restartOp;
     }
     
-    private void performLeafSplit(int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
-        int rightSiblingPageId = ctx.leafFrame.getNextLeaf();
+    private boolean performLeafSplit(int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {    	
+    	// Lock is released in unsetSmPages(), after sm has fully completed.
+        if (!treeLatch.writeLock().tryLock()) {
+        	return true;
+        }
+    	int rightSiblingPageId = ctx.leafFrame.getNextLeaf();
         ICachedPage rightSibling = null;
         if (rightSiblingPageId > 0) {
             rightSibling = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightSiblingPageId),
                     false);
         }
-        // Lock is released in unsetSmPages(), after sm has fully completed.
-        treeLatch.writeLock().lock(); 
         try {
             int rightPageId = freePageManager.getFreePage(ctx.metaFrame);
             ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId),
                     true);
             rightNode.acquireWriteLatch();
             try {
-                IBTreeLeafFrame rightFrame = ctx.createLeafFrame();                
+                IBTreeLeafFrame rightFrame = ctx.createLeafFrame();
                 rightFrame.setPage(rightNode);
                 rightFrame.initBuffer((byte) 0);
                 rightFrame.setMultiComparator(cmp);
@@ -385,7 +383,7 @@
                 ctx.splitKey.setPages(pageId, rightPageId);
                 
                 if (rightSibling != null) {
-                    rightSibling.acquireWriteLatch();
+                	rightSibling.acquireWriteLatch();
                     try {
                         // Reuse rightFrame for modification.
                         rightFrame.setPage(rightSibling);
@@ -406,12 +404,14 @@
                 bufferCache.unpin(rightSibling);
             }
         }
+        return false;
     }
     
-    private void updateLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+    private boolean updateLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
         ctx.leafFrame.setPage(node);
         int oldTupleIndex = ctx.leafFrame.findUpdateTupleIndex(tuple);
         FrameOpSpaceStatus spaceStatus = ctx.leafFrame.hasSpaceUpdate(tuple, oldTupleIndex);
+        boolean restartOp = false;
         switch (spaceStatus) {
             case SUFFICIENT_INPLACE_SPACE: {
                 ctx.leafFrame.update(tuple, oldTupleIndex, true);
@@ -443,13 +443,14 @@
                     ctx.leafFrame.insert(tuple, targetTupleIndex);
                     ctx.splitKey.reset();
                 } else {
-                    performLeafSplit(pageId, tuple, ctx);
+                    restartOp = performLeafSplit(pageId, tuple, ctx);
                 }
                 break;
             }
         }
         node.releaseWriteLatch();
         bufferCache.unpin(node);
+        return restartOp;
     }
 
     private void insertInterior(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx)
@@ -506,95 +507,102 @@
         }
     }
 
-    private void deleteLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+    private boolean deleteLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
         ctx.leafFrame.setPage(node);
         int tupleIndex = ctx.leafFrame.findDeleteTupleIndex(tuple);
-
+        
         // Will this leaf become empty?
         if (ctx.leafFrame.getTupleCount() > 1) {
             // Leaf will not become empty.
             ctx.leafFrame.delete(tuple, tupleIndex);
             node.releaseWriteLatch();
             bufferCache.unpin(node);
-            return;
+            return false;
         }
         
         // Leaf will become empty. 
-        IBTreeLeafFrame siblingFrame = ctx.createLeafFrame();
+        IBTreeLeafFrame siblingFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
         siblingFrame.setMultiComparator(cmp);
         ICachedPage leftNode = null;
         ICachedPage rightNode = null;
         int nextLeaf = ctx.leafFrame.getNextLeaf();
         int prevLeaf = ctx.leafFrame.getPrevLeaf();
-        if (prevLeaf > 0) {
-            leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, prevLeaf), false);
+        // Try to get the tree latch, if it's already taken, then restart this operation
+        // to avoid latch deadlock.
+        if (!treeLatch.writeLock().tryLock()) {
+        	return true;
         }
         try {
-            if (nextLeaf > 0) {
-                rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeaf), false);
-            }
-            try {
-                treeLatch.writeLock().lock();
-                try {
-                    ctx.leafFrame.delete(tuple, tupleIndex);
-                    // To propagate the deletion we only need to make the
-                    // splitKey != null.
-                    // Reuse data to identify which key to delete in the parent.
-                    ctx.splitKey.initData(1);
-                } catch (Exception e) {
-                    // Don't propagate deletion.
-                    ctx.splitKey.reset();
-                    throw e;
-                }
+        	if (prevLeaf > 0) {
+        		leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, prevLeaf), false);
+        	}
+        	try {
+        		if (nextLeaf > 0) {
+        			rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeaf), false);
+        		}
+        		try {
+        			try {
+        				ctx.leafFrame.delete(tuple, tupleIndex);
+        				// To propagate the deletion we only need to make the
+        				// splitKey != null.
+        				// Reuse data to identify which key to delete in the parent.
+        				ctx.splitKey.initData(1);
+        			} catch (Exception e) {
+        				// Don't propagate deletion.
+        				ctx.splitKey.reset();
+        				throw e;
+        			}
 
-                // TODO: Tie together with logging.
-                ctx.leafFrame.setPageLsn(ctx.leafFrame.getPageLsn() + 1);
-                ctx.leafFrame.setLevel(freePageManager.getFreePageLevelIndicator());
+        			// TODO: Tie together with logging.
+        			ctx.leafFrame.setPageLsn(ctx.leafFrame.getPageLsn() + 1);
+        			ctx.leafFrame.setLevel(freePageManager.getFreePageLevelIndicator());
 
-                ctx.smPages.add(pageId);
-                ctx.leafFrame.setSmFlag(true);
+        			ctx.smPages.add(pageId);
+        			ctx.leafFrame.setSmFlag(true);
 
-                node.releaseWriteLatch();
-                bufferCache.unpin(node);
+        			node.releaseWriteLatch();
+        			bufferCache.unpin(node);
 
-                if (leftNode != null) {
-                    leftNode.acquireWriteLatch();
-                    try {
-                        siblingFrame.setPage(leftNode);
-                        siblingFrame.setNextLeaf(nextLeaf);
-                        // TODO: Tie together with logging.
-                        siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
-                    } finally {
-                        leftNode.releaseWriteLatch();
-                    }
-                }
+        			if (leftNode != null) {
+        				leftNode.acquireWriteLatch();
+        				try {
+        					siblingFrame.setPage(leftNode);
+        					siblingFrame.setNextLeaf(nextLeaf);
+        					// TODO: Tie together with logging.
+        					siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
+        				} finally {
+        					leftNode.releaseWriteLatch();
+        				}
+        			}
 
-                if (rightNode != null) {
-                    rightNode.acquireWriteLatch();
-                    try {
-                        siblingFrame.setPage(rightNode);
-                        siblingFrame.setPrevLeaf(prevLeaf);
-                        // TODO: Tie together with logging.
-                        siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
-                    } finally {
-                        rightNode.releaseWriteLatch();
-                    }
-                }
-                // Register pageId as a free.
-                ctx.freePages.add(pageId);
-            } catch (Exception e) {
-                treeLatch.writeLock().unlock();
-                throw e;
-            } finally {
-                if (rightNode != null) {
-                    bufferCache.unpin(rightNode);
-                }
-            }
-        } finally {
-            if (leftNode != null) {
-                bufferCache.unpin(leftNode);
-            }
+        			if (rightNode != null) {
+        				rightNode.acquireWriteLatch();
+        				try {
+        					siblingFrame.setPage(rightNode);
+        					siblingFrame.setPrevLeaf(prevLeaf);
+        					// TODO: Tie together with logging.
+        					siblingFrame.setPageLsn(siblingFrame.getPageLsn() + 1);
+        				} finally {
+        					rightNode.releaseWriteLatch();
+        				}
+        			}
+        			// Register pageId as a free.
+        			ctx.freePages.add(pageId);
+        		} finally {
+        			if (rightNode != null) {
+                		bufferCache.unpin(rightNode);
+                	}
+        		}
+        	} finally {
+        		if (leftNode != null) {
+        			bufferCache.unpin(leftNode);
+        		}
+        	}
+        } catch (Exception e) {
+        	treeLatch.writeLock().unlock();
+        	throw e;
         }
+        return false;
     }
 
     private void deleteInterior(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx)
@@ -665,7 +673,7 @@
         // this check performs an unprotected read in the page
         // the following could happen: TODO fill out
         boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
-        acquireLatch(node, ctx, unsafeIsLeaf);        
+        acquireLatch(node, ctx, unsafeIsLeaf);
         boolean smFlag = ctx.interiorFrame.getSmFlag();
         // re-check leafness after latching
         boolean isLeaf = ctx.interiorFrame.isLeaf();
@@ -674,7 +682,6 @@
         // structure modification
         ctx.pageLsns.add(ctx.interiorFrame.getPageLsn());
         try {
-
             // latch coupling, note: parent should never be write latched,
             // otherwise something is wrong.
             if (parent != null) {
@@ -742,8 +749,6 @@
                     } // end while
                 } else { // smFlag
                     ctx.opRestarts++;
-                    System.out.println("ONGOING SM ON PAGE " + pageId + " AT LEVEL " + ctx.interiorFrame.getLevel()
-                            + ", RESTARTS: " + ctx.opRestarts);
                     releaseLatch(node, ctx, unsafeIsLeaf);
                     bufferCache.unpin(node);
 
@@ -752,9 +757,9 @@
                     // instead we just immediately release the lock. this is
                     // inefficient but still correct and will not cause
                     // latch-deadlock
-                    treeLatch.readLock().lock();
-                    treeLatch.readLock().unlock();
-
+                    treeLatch.writeLock().lock();
+                    treeLatch.writeLock().unlock();
+                    
                     // unwind recursion and restart operation, find lowest page
                     // with a pageLsn as seen by this operation during descent
                     ctx.pageLsns.removeLast(); // pop current page lsn
@@ -763,17 +768,19 @@
                     ctx.pageLsns.add(RESTART_OP);
                 }
             } else { // isLeaf and !smFlag
-                switch (ctx.op) {
+                // We may have to restart an op to avoid latch deadlock.
+            	boolean restartOp = false;
+            	switch (ctx.op) {
                     case INSERT: {
-                        insertLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+                        restartOp = insertLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
                         break;
                     }
                     case UPDATE: {
-                        updateLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+                    	restartOp = updateLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
                         break;
                     }
                     case DELETE: {
-                        deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+                    	restartOp = deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
                         break;
                     }
                     case SEARCH: {
@@ -782,22 +789,27 @@
                         break;
                     }
                 }
+            	if (restartOp) {
+            		ctx.pageLsns.removeLast();
+                    ctx.pageLsns.add(RESTART_OP);
+            	}
             }
         } catch (TreeIndexException e) {
-            if (!ctx.exceptionHandled) {
+        	if (!ctx.exceptionHandled) {
                 releaseLatch(node, ctx, unsafeIsLeaf);
                 bufferCache.unpin(node);
                 ctx.exceptionHandled = true;
             }
             throw e;
         } catch (PageAllocationException e) {
-            if (!ctx.exceptionHandled) {
+        	if (!ctx.exceptionHandled) {
                 releaseLatch(node, ctx, unsafeIsLeaf);
                 bufferCache.unpin(node);
                 ctx.exceptionHandled = true;
             }
             throw e;
         } catch (Exception e) {
+        	e.printStackTrace();
             releaseLatch(node, ctx, unsafeIsLeaf);
             bufferCache.unpin(node);
             BTreeException wrappedException = new BTreeException(e);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
index 621281a..80ac173 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
@@ -15,14 +15,14 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
 
 public class FieldPrefixPrefixTupleReference extends TypeAwareTupleReference {
 
-    public FieldPrefixPrefixTupleReference(ITypeTrait[] typeTraits) {
+    public FieldPrefixPrefixTupleReference(ITypeTraits[] typeTraits) {
         super(typeTraits);
     }
 
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
index 625fa02..f6310af 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
@@ -1,7 +1,7 @@
 package edu.uci.ics.hyracks.storage.am.btree.util;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrameFactory;
@@ -20,7 +20,7 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
 public class BTreeUtils {
-    public static BTree createBTree(IBufferCache bufferCache, int btreeFileId, ITypeTrait[] typeTraits, IBinaryComparator[] cmps, BTreeLeafFrameType leafType) throws BTreeException {
+    public static BTree createBTree(IBufferCache bufferCache, int btreeFileId, ITypeTraits[] typeTraits, IBinaryComparator[] cmps, BTreeLeafFrameType leafType) throws BTreeException {
     	MultiComparator cmp = new MultiComparator(cmps);
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index 459ab57..ff4d078 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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;
@@ -40,7 +40,7 @@
 	protected final IStorageManagerInterface storageManager;
 	protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
 
-	protected final ITypeTrait[] typeTraits;
+	protected final ITypeTraits[] typeTraits;
 	protected final IIndexDataflowHelperFactory dataflowHelperFactory;
 
 	public AbstractTreeIndexOperatorDescriptor(JobSpecification spec,
@@ -49,7 +49,7 @@
 			IIndexRegistryProvider<IIndex> indexRegistryProvider,
 			IFileSplitProvider fileSplitProvider,
 			ITreeIndexFrameFactory interiorFrameFactory,
-			ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+			ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
 			IBinaryComparatorFactory[] comparatorFactories,
 			IIndexDataflowHelperFactory dataflowHelperFactory) {
 		super(spec, inputArity, outputArity);
@@ -77,7 +77,7 @@
 	}
 
 	@Override
-	public ITypeTrait[] getTreeIndexTypeTraits() {
+	public ITypeTraits[] getTreeIndexTypeTraits() {
 		return typeTraits;
 	}
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
index 9bbca56..e415cbb 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 
 public interface ITreeIndexOperatorDescriptor extends IIndexOperatorDescriptor {
@@ -26,6 +26,6 @@
 
 	public ITreeIndexFrameFactory getTreeIndexLeafFactory();
 
-	public ITypeTrait[] getTreeIndexTypeTraits();
+	public ITypeTraits[] getTreeIndexTypeTraits();
 	
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index c5b5355..af30105 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -35,7 +35,7 @@
     public TreeIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
             IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
-            ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
             float fillFactor, IIndexDataflowHelperFactory dataflowHelperFactory) {
         super(spec, 1, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 8cc83f7..8d56d37 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -18,7 +18,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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.IFileSplitProvider;
@@ -35,7 +35,7 @@
 			IIndexRegistryProvider<IIndex> indexRegistryProvider,
 			IFileSplitProvider fileSplitProvider,
 			ITreeIndexFrameFactory interiorFrameFactory,
-			ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+			ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
 			IIndexDataflowHelperFactory dataflowHelperFactory) {
 		super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider,
 				fileSplitProvider, interiorFrameFactory, leafFrameFactory,
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 50fb3ea..e08f5d6 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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.IFileSplitProvider;
@@ -41,7 +41,7 @@
 			IIndexRegistryProvider<IIndex> indexRegistryProvider,
 			IFileSplitProvider fileSplitProvider,
 			ITreeIndexFrameFactory interiorFrameFactory,
-			ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+			ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
 			IBinaryComparatorFactory[] comparatorFactories,
 			int[] fieldPermutation, IndexOp op,
 			IIndexDataflowHelperFactory dataflowHelperFactory) {
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 96a2a94..4d718a8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -36,7 +36,7 @@
     public TreeIndexStatsOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
             IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
-            ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
             IIndexDataflowHelperFactory dataflowHelperFactory) {
         super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
index 72a36a1..4776bdd 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
@@ -17,7 +17,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 
@@ -29,11 +29,11 @@
     protected int nullFlagsBytes;
     protected int dataStartOff;
 
-    protected ITypeTrait[] typeTraits;
+    protected ITypeTraits[] typeTraits;
     protected VarLenIntEncoderDecoder encDec = new VarLenIntEncoderDecoder();
     protected int[] decodedFieldSlots;
 
-    public TypeAwareTupleReference(ITypeTrait[] typeTraits) {
+    public TypeAwareTupleReference(ITypeTraits[] typeTraits) {
         this.typeTraits = typeTraits;
         this.fieldStartIndex = 0;
         setFieldCount(typeTraits.length);
@@ -50,12 +50,11 @@
         int end = fieldStartIndex + fieldCount;
         encDec.reset(buf.array(), tupleStartOff + nullFlagsBytes);
         for (int i = fieldStartIndex; i < end; i++) {
-            int staticDataLen = typeTraits[i].getStaticallyKnownDataLength();
-            if (staticDataLen == ITypeTrait.VARIABLE_LENGTH) {
+            if (!typeTraits[i].isFixedLength()) {
                 cumul += encDec.decode();
                 decodedFieldSlots[field++] = cumul;
             } else {
-                cumul += staticDataLen;
+                cumul += typeTraits[i].getFixedLength();
                 decodedFieldSlots[field++] = cumul;
             }
         }
@@ -119,8 +118,8 @@
         return (int) Math.ceil(fieldCount / 8.0);
     }
 
-	@Override
-	public int getTupleSize() {
-		return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount-1];
-	}
+    @Override
+    public int getTupleSize() {
+        return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount - 1];
+    }
 }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
index 6b33712..fe52608 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
@@ -17,17 +17,17 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 
 public class TypeAwareTupleWriter implements ITreeIndexTupleWriter {
 
-    protected ITypeTrait[] typeTraits;
+    protected ITypeTraits[] typeTraits;
     protected VarLenIntEncoderDecoder encDec = new VarLenIntEncoderDecoder();
 
-    public TypeAwareTupleWriter(ITypeTrait[] typeTraits) {
+    public TypeAwareTupleWriter(ITypeTraits[] typeTraits) {
         this.typeTraits = typeTraits;
     }
 
@@ -65,13 +65,13 @@
         int nullFlagsBytes = getNullFlagsBytes(tuple);
         // write null indicator bits
         for (int i = 0; i < nullFlagsBytes; i++) {
-        	targetBuf[runner++] = (byte) 0;
+            targetBuf[runner++] = (byte) 0;
         }
 
         // write field slots for variable length fields
         encDec.reset(targetBuf, runner);
         for (int i = 0; i < tuple.getFieldCount(); i++) {
-            if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+            if (!typeTraits[i].isFixedLength()) {
                 encDec.encode(tuple.getFieldLength(i));
             }
         }
@@ -81,14 +81,13 @@
         for (int i = 0; i < tuple.getFieldCount(); i++) {
             int s = tuple.getFieldStart(i);
             int l = tuple.getFieldLength(i);
-        	System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner,
-                    tuple.getFieldLength(i));
+            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
             runner += tuple.getFieldLength(i);
         }
 
         return runner - targetOff;
     }
-    
+
     @Override
     public int writeTupleFields(ITupleReference tuple, int startField, int numFields, ByteBuffer targetBuf,
             int targetOff) {
@@ -102,7 +101,7 @@
         // write field slots for variable length fields
         encDec.reset(targetBuf.array(), runner);
         for (int i = startField; i < startField + numFields; i++) {
-            if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+            if (!typeTraits[i].isFixedLength()) {
                 encDec.encode(tuple.getFieldLength(i));
             }
         }
@@ -124,7 +123,7 @@
     protected int getFieldSlotsBytes(ITupleReference tuple) {
         int fieldSlotBytes = 0;
         for (int i = 0; i < tuple.getFieldCount(); i++) {
-            if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+            if (!typeTraits[i].isFixedLength()) {
                 fieldSlotBytes += encDec.getBytesRequired(tuple.getFieldLength(i));
             }
         }
@@ -138,18 +137,18 @@
     protected int getFieldSlotsBytes(ITupleReference tuple, int startField, int numFields) {
         int fieldSlotBytes = 0;
         for (int i = startField; i < startField + numFields; i++) {
-            if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+            if (!typeTraits[i].isFixedLength()) {
                 fieldSlotBytes += encDec.getBytesRequired(tuple.getFieldLength(i));
             }
         }
         return fieldSlotBytes;
     }
 
-    public ITypeTrait[] getTypeTraits() {
+    public ITypeTraits[] getTypeTraits() {
         return typeTraits;
     }
 
-    public void setTypeTraits(ITypeTrait[] typeTraits) {
+    public void setTypeTraits(ITypeTraits[] typeTraits) {
         this.typeTraits = typeTraits;
     }
 }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
index 28e2535..9e6ba6f 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.tuples;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
 
@@ -23,9 +23,9 @@
 		ITreeIndexTupleWriterFactory {
 
 	private static final long serialVersionUID = 1L;
-	private ITypeTrait[] typeTraits;
+	private ITypeTraits[] typeTraits;
 
-	public TypeAwareTupleWriterFactory(ITypeTrait[] typeTraits) {
+	public TypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
 		this.typeTraits = typeTraits;
 	}
 
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
index 499b60b..1dc753e 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
@@ -16,14 +16,14 @@
 package edu.uci.ics.hyracks.storage.am.invertedindex.api;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptor;
 
 public interface IInvertedIndexOperatorDescriptor extends ITreeIndexOperatorDescriptor {
     public IBinaryComparatorFactory[] getInvListsComparatorFactories();
 
-    public ITypeTrait[] getInvListsTypeTraits();
+    public ITypeTraits[] getInvListsTypeTraits();
     
     public IFileSplitProvider getInvListsFileSplitProvider();
 }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
index a8860e2..eab266d 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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;
@@ -45,21 +45,21 @@
     // Btree.
     protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
     protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
-    protected final ITypeTrait[] btreeTypeTraits;
+    protected final ITypeTraits[] btreeTypeTraits;
     protected final IBinaryComparatorFactory[] btreeComparatorFactories;
     protected final IIndexDataflowHelperFactory btreeDataflowHelperFactory;
     protected final IFileSplitProvider btreeFileSplitProvider;
 
     // Inverted index.
-    protected final ITypeTrait[] invListsTypeTraits;
+    protected final ITypeTraits[] invListsTypeTraits;
     protected final IBinaryComparatorFactory[] invListComparatorFactories;
     protected final IFileSplitProvider invListsFileSplitProvider;
 
     public AbstractInvertedIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
             RecordDescriptor recDesc, IStorageManagerInterface storageManager,
             IFileSplitProvider btreeFileSplitProvider, IFileSplitProvider invListsFileSplitProvider,
-            IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTrait[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTrait[] invListsTypeTraits,
+            IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories,
             IIndexDataflowHelperFactory btreeDataflowHelperFactory) {
         super(spec, inputArity, outputArity);
@@ -103,7 +103,7 @@
     }
 
     @Override
-    public ITypeTrait[] getTreeIndexTypeTraits() {
+    public ITypeTraits[] getTreeIndexTypeTraits() {
         return btreeTypeTraits;
     }
 
@@ -133,7 +133,7 @@
     }
 
     @Override
-    public ITypeTrait[] getInvListsTypeTraits() {
+    public ITypeTraits[] getInvListsTypeTraits() {
         return invListsTypeTraits;
     }
 
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 0af27d6..d00bea6 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -59,7 +59,7 @@
     public void open() throws HyracksDataException {
         accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
         writeBuffer = ctx.allocateFrame();
-        builder = new ArrayTupleBuilder(outputRecDesc.getFields().length);
+        builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
         builderDos = builder.getDataOutput();
         appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(writeBuffer, true);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index 05f8b7d..e91aa08 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -36,8 +36,8 @@
     public InvertedIndexBulkLoadOperatorDescriptor(JobSpecification spec, int[] fieldPermutation,
             IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
             IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
-            ITypeTrait[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
-            ITypeTrait[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
             IIndexDataflowHelperFactory btreeDataflowHelperFactory) {
         super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
                 indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
index f1704cc..bf9899f 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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;
@@ -43,8 +43,8 @@
     public InvertedIndexSearchOperatorDescriptor(JobSpecification spec,
             int queryField, IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
             IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
-            ITypeTrait[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
-            ITypeTrait[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
             IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
             IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc) {
         super(spec, 1, 1, recDesc, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
index 03fc9a1..643c105 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
 
@@ -26,10 +26,10 @@
     private byte[] targetBuf;
     private int pos;
 
-    public FixedSizeElementInvertedListBuilder(ITypeTrait[] invListFields) {
+    public FixedSizeElementInvertedListBuilder(ITypeTraits[] invListFields) {
         int tmp = 0;
         for (int i = 0; i < invListFields.length; i++) {
-            tmp += invListFields[i].getStaticallyKnownDataLength();
+            tmp += invListFields[i].getFixedLength();
         }
         listElementSize = tmp;
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
index f7ef56e..4158019 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
@@ -5,7 +5,7 @@
 import java.io.DataInputStream;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -32,7 +32,7 @@
     private ICachedPage[] pages = new ICachedPage[10];
     private int[] elementIndexes = new int[10];
 
-    public FixedSizeElementInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTrait[] invListFields) {
+    public FixedSizeElementInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields) {
         this.bufferCache = bufferCache;
         this.fileId = fileId;
         this.currentElementIx = 0;
@@ -40,7 +40,7 @@
 
         int tmp = 0;
         for (int i = 0; i < invListFields.length; i++) {
-            tmp += invListFields[i].getStaticallyKnownDataLength();
+            tmp += invListFields[i].getFixedLength();
         }
         elementSize = tmp;
         this.currentOff = -elementSize;
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
index 9858eb0..cbedc45 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
@@ -19,29 +19,29 @@
 
 import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public class FixedSizeFrameTupleAccessor implements IFrameTupleAccessor {
 
     private final int frameSize;
     private ByteBuffer buffer;
 
-    private final ITypeTrait[] fields;
+    private final ITypeTraits[] fields;
     private final int[] fieldStartOffsets;
     private final int tupleSize;
 
-    public FixedSizeFrameTupleAccessor(int frameSize, ITypeTrait[] fields) {
+    public FixedSizeFrameTupleAccessor(int frameSize, ITypeTraits[] fields) {
         this.frameSize = frameSize;
         this.fields = fields;
         this.fieldStartOffsets = new int[fields.length];
         this.fieldStartOffsets[0] = 0;
         for (int i = 1; i < fields.length; i++) {
-            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + fields[i - 1].getStaticallyKnownDataLength();
+            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + fields[i - 1].getFixedLength();
         }
 
         int tmp = 0;
         for (int i = 0; i < fields.length; i++) {
-            tmp += fields[i].getStaticallyKnownDataLength();
+            tmp += fields[i].getFixedLength();
         }
         tupleSize = tmp;
     }
@@ -58,12 +58,12 @@
 
     @Override
     public int getFieldEndOffset(int tupleIndex, int fIdx) {
-        return getTupleStartOffset(tupleIndex) + fieldStartOffsets[fIdx] + fields[fIdx].getStaticallyKnownDataLength();
+        return getTupleStartOffset(tupleIndex) + fieldStartOffsets[fIdx] + fields[fIdx].getFixedLength();
     }
 
     @Override
     public int getFieldLength(int tupleIndex, int fIdx) {
-        return fields[fIdx].getStaticallyKnownDataLength();
+        return fields[fIdx].getFixedLength();
     }
 
     @Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
index edc2304..489ec2e 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
@@ -18,7 +18,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public class FixedSizeFrameTupleAppender {
 
@@ -29,11 +29,11 @@
     private int tupleCount;
     private int tupleDataEndOffset;
 
-    public FixedSizeFrameTupleAppender(int frameSize, ITypeTrait[] fields) {
+    public FixedSizeFrameTupleAppender(int frameSize, ITypeTraits[] fields) {
         this.frameSize = frameSize;
         int tmp = 0;
         for (int i = 0; i < fields.length; i++) {
-            tmp += fields[i].getStaticallyKnownDataLength();
+            tmp += fields[i].getFixedLength();
         }
         tupleSize = tmp;
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
index 248b81e..0656d69 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
@@ -15,22 +15,22 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public class FixedSizeTupleReference implements ITupleReference {
 
-    private final ITypeTrait[] typeTraits;
+    private final ITypeTraits[] typeTraits;
     private final int[] fieldStartOffsets;
     private byte[] data;
     private int startOff;
 
-    public FixedSizeTupleReference(ITypeTrait[] typeTraits) {
+    public FixedSizeTupleReference(ITypeTraits[] typeTraits) {
         this.typeTraits = typeTraits;
         this.fieldStartOffsets = new int[typeTraits.length];
         this.fieldStartOffsets[0] = 0;
         for (int i = 1; i < typeTraits.length; i++) {
-            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + typeTraits[i - 1].getStaticallyKnownDataLength();
+            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + typeTraits[i - 1].getFixedLength();
         }
     }
 
@@ -51,7 +51,7 @@
 
     @Override
     public int getFieldLength(int fIdx) {
-        return typeTraits[fIdx].getStaticallyKnownDataLength();
+        return typeTraits[fIdx].getFixedLength();
     }
 
     @Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 1118d04..986e57b 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -18,7 +18,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -57,12 +57,12 @@
     private int rootPageId = 0;
     private IBufferCache bufferCache;
     private int fileId;
-    private final ITypeTrait[] invListTypeTraits;
+    private final ITypeTraits[] invListTypeTraits;
     private final MultiComparator invListCmp;
     private final int numTokenFields;
     private final int numInvListKeys;
 
-    public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTrait[] invListTypeTraits, MultiComparator invListCmp) {
+    public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTraits[] invListTypeTraits, MultiComparator invListCmp) {
         this.bufferCache = bufferCache;
         this.btree = btree;
         this.invListCmp = invListCmp;
@@ -230,7 +230,7 @@
         return invListCmp;
     }
     
-    public ITypeTrait[] getTypeTraits() {
+    public ITypeTraits[] getTypeTraits() {
         return invListTypeTraits;
     }
 
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index dd09ef7..b24d416 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -24,10 +24,10 @@
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -72,14 +72,14 @@
 
     protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
             new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
-    protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFields().length);
+    protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFieldCount());
     protected DataOutput queryTokenDos = queryTokenBuilder.getDataOutput();
     protected FrameTupleAppender queryTokenAppender;
     protected ByteBuffer queryTokenFrame;
 
     protected final InvertedIndex invIndex;
     protected final IBinaryTokenizer queryTokenizer;
-    protected final ITypeTrait[] invListFieldsWithCount;
+    protected final ITypeTraits[] invListFieldsWithCount;
     protected int occurrenceThreshold;
 
     protected final int cursorCacheSize = 10;
@@ -95,15 +95,15 @@
         interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
 
         btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
-        ITypeTrait[] invListFields = invIndex.getTypeTraits();
-        invListFieldsWithCount = new TypeTrait[invListFields.length + 1];
+        ITypeTraits[] invListFields = invIndex.getTypeTraits();
+        invListFieldsWithCount = new ITypeTraits[invListFields.length + 1];
         int tmp = 0;
         for (int i = 0; i < invListFields.length; i++) {
             invListFieldsWithCount[i] = invListFields[i];
-            tmp += invListFields[i].getStaticallyKnownDataLength();
+            tmp += invListFields[i].getFixedLength();
         }
         // using an integer for counting occurrences
-        invListFieldsWithCount[invListFields.length] = new TypeTrait(4);
+        invListFieldsWithCount[invListFields.length] = IntegerPointable.TYPE_TRAITS;
         invListKeyLength = tmp;
 
         resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
@@ -126,7 +126,7 @@
 
         queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
         queryTokenFrame = ctx.allocateFrame();
-        
+
         btreeAccessor = invIndex.getBTree().createAccessor();
         currentNumResults = 0;
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
index bbb32d6..be6bfb0 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
@@ -19,60 +19,59 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
-public abstract class AbstractUTF8StringBinaryTokenizer implements
-		IBinaryTokenizer {
+public abstract class AbstractUTF8StringBinaryTokenizer implements IBinaryTokenizer {
 
-	protected byte[] data;
-	protected int start;
-	protected int length;
-	protected int tokenLength;
-	protected int index;
-	protected int utf8Length;
+    protected byte[] data;
+    protected int start;
+    protected int length;
+    protected int tokenLength;
+    protected int index;
+    protected int utf8Length;
 
-	protected final IntArray tokensStart;
-	protected final IntArray tokensLength;
-	protected final IToken token;
+    protected final IntArray tokensStart;
+    protected final IntArray tokensLength;
+    protected final IToken token;
 
-	protected final boolean ignoreTokenCount;
-	protected final boolean sourceHasTypeTag;
+    protected final boolean ignoreTokenCount;
+    protected final boolean sourceHasTypeTag;
 
-	public AbstractUTF8StringBinaryTokenizer(boolean ignoreTokenCount,
-			boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
-		this.ignoreTokenCount = ignoreTokenCount;
-		this.sourceHasTypeTag = sourceHasTypeTag;
-		if (!ignoreTokenCount) {
-			tokensStart = new IntArray();
-			tokensLength = new IntArray();
-		} else {
-			tokensStart = null;
-			tokensLength = null;
-		}
-		token = tokenFactory.createToken();
-	}
+    public AbstractUTF8StringBinaryTokenizer(boolean ignoreTokenCount, boolean sourceHasTypeTag,
+            ITokenFactory tokenFactory) {
+        this.ignoreTokenCount = ignoreTokenCount;
+        this.sourceHasTypeTag = sourceHasTypeTag;
+        if (!ignoreTokenCount) {
+            tokensStart = new IntArray();
+            tokensLength = new IntArray();
+        } else {
+            tokensStart = null;
+            tokensLength = null;
+        }
+        token = tokenFactory.createToken();
+    }
 
-	@Override
-	public IToken getToken() {
-		return token;
-	}
+    @Override
+    public IToken getToken() {
+        return token;
+    }
 
-	@Override
-	public void reset(byte[] data, int start, int length) {
-		this.start = start;
-		index = this.start;
-		if (sourceHasTypeTag) {
-			index++; // skip type tag
-		}
-		utf8Length = StringUtils.getUTFLen(data, index);
-		index += 2; // skip utf8 length indicator
-		this.data = data;
-		this.length = length + start;
+    @Override
+    public void reset(byte[] data, int start, int length) {
+        this.start = start;
+        index = this.start;
+        if (sourceHasTypeTag) {
+            index++; // skip type tag
+        }
+        utf8Length = UTF8StringPointable.getUTFLen(data, index);
+        index += 2; // skip utf8 length indicator
+        this.data = data;
+        this.length = length + start;
 
-		tokenLength = 0;
-		if (!ignoreTokenCount) {
-			tokensStart.reset();
-			tokensLength.reset();
-		}
-	}
+        tokenLength = 0;
+        if (!ignoreTokenCount) {
+            tokensStart.reset();
+            tokensLength.reset();
+        }
+    }
 }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
index 209c939..65afa65 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
@@ -21,7 +21,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public abstract class AbstractUTF8Token implements IToken {
     public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
@@ -58,9 +58,9 @@
         int lowerCaseUTF8Len = 0;
         int pos = start;
         for (int i = 0; i < size; i++) {
-            char c = Character.toLowerCase(StringUtils.charAt(data, pos));
-            lowerCaseUTF8Len += StringUtils.getModifiedUTF8Len(c);
-            pos += StringUtils.charSize(data, pos);
+            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+            lowerCaseUTF8Len += UTF8StringPointable.getModifiedUTF8Len(c);
+            pos += UTF8StringPointable.charSize(data, pos);
         }
         return lowerCaseUTF8Len;
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
index 83d0c75..9dacde6 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
@@ -19,7 +19,7 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class DelimitedUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
 
@@ -31,8 +31,8 @@
     @Override
     public boolean hasNext() {
         // skip delimiters
-        while (index < length && isSeparator(StringUtils.charAt(data, index))) {
-            index += StringUtils.charSize(data, index);
+        while (index < length && isSeparator(UTF8StringPointable.charAt(data, index))) {
+            index += UTF8StringPointable.charSize(data, index);
         }
         return index < length;
     }
@@ -45,8 +45,8 @@
     public void next() {
         tokenLength = 0;
         int currentTokenStart = index;
-        while (index < length && !isSeparator(StringUtils.charAt(data, index))) {
-            index += StringUtils.charSize(data, index);
+        while (index < length && !isSeparator(UTF8StringPointable.charAt(data, index))) {
+            index += UTF8StringPointable.charSize(data, index);
             tokenLength++;
         }
         int tokenCount = 1;
@@ -60,12 +60,12 @@
                     int currLength = 0;
                     while (currLength < tokenLength) {
                         // case insensitive comparison
-                        if (Character.toLowerCase(StringUtils.charAt(data, currentTokenStart + offset)) != Character
-                                .toLowerCase(StringUtils.charAt(data, tokenStart + offset))) {
+                        if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
+                                .toLowerCase(UTF8StringPointable.charAt(data, tokenStart + offset))) {
                             tokenCount--;
                             break;
                         }
-                        offset += StringUtils.charSize(data, currentTokenStart + offset);
+                        offset += UTF8StringPointable.charSize(data, currentTokenStart + offset);
                         currLength++;
                     }
                 }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
index 43f89c7..b7bb828 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
@@ -22,7 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class HashedUTF8NGramToken extends UTF8NGramToken {
     public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
@@ -45,9 +45,9 @@
         int numRegGrams = tokenLength - numPreChars - numPostChars;
         int pos = start;
         for (int i = 0; i < numRegGrams; i++) {
-            hash ^= Character.toLowerCase(StringUtils.charAt(data, pos));
+            hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
             hash *= GOLDEN_RATIO_32;
-            pos += StringUtils.charSize(data, pos);
+            pos += UTF8StringPointable.charSize(data, pos);
         }
 
         // post chars
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
index 747b65d..42ed053 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
@@ -22,7 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class HashedUTF8WordToken extends UTF8WordToken {
 
@@ -46,10 +46,11 @@
         }
         int offset = 0;
         for (int i = 0; i < tokenLength; i++) {
-            if (StringUtils.charAt(t.getData(), t.getStart() + offset) != StringUtils.charAt(data, start + offset)) {
+            if (UTF8StringPointable.charAt(t.getData(), t.getStart() + offset) != UTF8StringPointable.charAt(data,
+                    start + offset)) {
                 return false;
             }
-            offset += StringUtils.charSize(data, start + offset);
+            offset += UTF8StringPointable.charSize(data, start + offset);
         }
         return true;
     }
@@ -67,9 +68,9 @@
         int pos = start;
         hash = GOLDEN_RATIO_32;
         for (int i = 0; i < tokenLength; i++) {
-            hash ^= Character.toLowerCase(StringUtils.charAt(data, pos));
+            hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
             hash *= GOLDEN_RATIO_32;
-            pos += StringUtils.charSize(data, pos);
+            pos += UTF8StringPointable.charSize(data, pos);
         }
         hash += tokenCount;
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
index bdbf6f8..fdfc02f 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
@@ -19,7 +19,7 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class NGramUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
 
@@ -62,7 +62,7 @@
 
         concreteToken.setNumPrePostChars(numPreChars, numPostChars);
         if (numPreChars == 0) {
-            index += StringUtils.charSize(data, index);
+            index += UTF8StringPointable.charSize(data, index);
         }
 
         // compute token count
@@ -73,14 +73,14 @@
                 tokenCount++; // assume found
                 int offset = 0;
                 for (int j = 0; j < gramLength; j++) {
-                    if (Character.toLowerCase(StringUtils.charAt(data, currentTokenStart + offset)) != Character
-                            .toLowerCase(StringUtils.charAt(data, tmpIndex + offset))) {
+                    if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
+                            .toLowerCase(UTF8StringPointable.charAt(data, tmpIndex + offset))) {
                         tokenCount--;
                         break;
                     }
-                    offset += StringUtils.charSize(data, tmpIndex + offset);
+                    offset += UTF8StringPointable.charSize(data, tmpIndex + offset);
                 }
-                tmpIndex += StringUtils.charSize(data, tmpIndex);
+                tmpIndex += UTF8StringPointable.charSize(data, tmpIndex);
             }
         }
 
@@ -98,7 +98,7 @@
         int end = pos + utf8Length;
         while (pos < end) {
             numChars++;
-            pos += StringUtils.charSize(data, pos);
+            pos += UTF8StringPointable.charSize(data, pos);
         }
 
         if (usePrePost) {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
index 1b124dc..59cadc8 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
@@ -22,6 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
 public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
@@ -67,9 +68,9 @@
 
         int pos = start;
         for (int i = 0; i < numRegChars; i++) {
-            char c = Character.toLowerCase(StringUtils.charAt(data, pos));
+            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
             StringUtils.writeCharAsModifiedUTF8(c, dos);
-            pos += StringUtils.charSize(data, pos);
+            pos += UTF8StringPointable.charSize(data, pos);
         }
 
         // post chars
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
index 2a74145..97a1e12 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
@@ -22,6 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
 public class UTF8WordToken extends AbstractUTF8Token {
@@ -38,9 +39,9 @@
         StringUtils.writeUTF8Len(tokenUTF8Len, dos);
         int pos = start;
         for (int i = 0; i < tokenLength; i++) {
-            char c = Character.toLowerCase(StringUtils.charAt(data, pos));
+            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
             StringUtils.writeCharAsModifiedUTF8(c, dos);
-            pos += StringUtils.charSize(data, pos);
+            pos += UTF8StringPointable.charSize(data, pos);
         }
     }
 }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
index 5a62f51..a1d1f06 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
@@ -15,33 +15,33 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.util;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 
 public class InvertedIndexUtils {
-	// Type traits to be appended to the token type trait which finally form the BTree field type traits.
-	private static final ITypeTrait[] btreeValueTypeTraits = new ITypeTrait[4];
-	static {
-		// startPageId
-		btreeValueTypeTraits[0] = new TypeTrait(4);
+    // Type traits to be appended to the token type trait which finally form the BTree field type traits.
+    private static final ITypeTraits[] btreeValueTypeTraits = new ITypeTraits[4];
+    static {
+        // startPageId
+        btreeValueTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
         // endPageId
-		btreeValueTypeTraits[1] = new TypeTrait(4);
+        btreeValueTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
         // startOff
-		btreeValueTypeTraits[2] = new TypeTrait(4);
+        btreeValueTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
         // numElements
-		btreeValueTypeTraits[3] = new TypeTrait(4);
-	}
-	
-	public static ITypeTrait[] getBTreeTypeTraits(ITypeTrait[] tokenTypeTraits) {
-		ITypeTrait[] btreeTypeTraits = new ITypeTrait[tokenTypeTraits.length + btreeValueTypeTraits.length];
-		// Set key type traits.
-		for (int i = 0; i < tokenTypeTraits.length; i++) {
-		    btreeTypeTraits[i] = tokenTypeTraits[i];
-		}
-		// Set value type traits.
-		for (int i = 0; i < btreeValueTypeTraits.length; i++) {
-		    btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
-		}
-		return btreeTypeTraits;
-	}
+        btreeValueTypeTraits[3] = IntegerPointable.TYPE_TRAITS;
+    }
+
+    public static ITypeTraits[] getBTreeTypeTraits(ITypeTraits[] tokenTypeTraits) {
+        ITypeTraits[] btreeTypeTraits = new ITypeTraits[tokenTypeTraits.length + btreeValueTypeTraits.length];
+        // Set key type traits.
+        for (int i = 0; i < tokenTypeTraits.length; i++) {
+            btreeTypeTraits[i] = tokenTypeTraits[i];
+        }
+        // Set value type traits.
+        for (int i = 0; i < btreeValueTypeTraits.length; i++) {
+            btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
+        }
+        return btreeTypeTraits;
+    }
 }
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 586324f..8cb4e55 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -19,7 +19,7 @@
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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.IFileSplitProvider;
@@ -39,7 +39,7 @@
     public RTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
             IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory) {
         super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
index 96d739c..f3e6be2 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
@@ -19,8 +19,9 @@
 import java.util.Collections;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
@@ -42,9 +43,9 @@
 public class RTreeNSMInteriorFrame extends RTreeNSMFrame implements IRTreeInteriorFrame {
 
     private static final int childPtrSize = 4;
-    private static IBinaryComparator childPtrCmp = IntegerBinaryComparatorFactory.INSTANCE
-			.createBinaryComparator();
-    
+    private static IBinaryComparator childPtrCmp = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY)
+            .createBinaryComparator();
+
     public RTreeNSMInteriorFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders) {
         super(tupleWriter, keyValueProviders);
         frameTuple.setFieldCount(keyValueProviders.length);
@@ -269,11 +270,11 @@
     }
 
     private int pointerCmp(ITupleReference tupleA, ITupleReference tupleB, MultiComparator cmp) {
-    	return childPtrCmp.compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1),
-                getChildPointerOff(tupleA), childPtrSize, tupleB.getFieldData(cmp.getKeyFieldCount() - 1),
-                getChildPointerOff(tupleB), childPtrSize);
+        return childPtrCmp
+                .compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleA), childPtrSize,
+                        tupleB.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleB), childPtrSize);
     }
-    
+
     @Override
     public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException {
         RTreeSplitKey rTreeSplitKey = ((RTreeSplitKey) splitKey);
@@ -294,10 +295,10 @@
             for (int k = 0; k < getTupleCount(); ++k) {
 
                 frameTuple.resetByTupleIndex(this, k);
-                double LowerKey = keyValueProviders[i].getValue(frameTuple.getFieldData(i),
-                        frameTuple.getFieldStart(i));
-                double UpperKey = keyValueProviders[j].getValue(frameTuple.getFieldData(j),
-                        frameTuple.getFieldStart(j));
+                double LowerKey = keyValueProviders[i]
+                        .getValue(frameTuple.getFieldData(i), frameTuple.getFieldStart(i));
+                double UpperKey = keyValueProviders[j]
+                        .getValue(frameTuple.getFieldData(j), frameTuple.getFieldStart(j));
 
                 tupleEntries1.add(k, LowerKey);
                 tupleEntries2.add(k, UpperKey);
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
index 244c69e..898ebf8 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
@@ -17,18 +17,18 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 
 public class RTreeTypeAwareTupleWriter extends TypeAwareTupleWriter {
 
-    public RTreeTypeAwareTupleWriter(ITypeTrait[] typeTraits) {
+    public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits) {
         super(typeTraits);
     }
 
-    public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {        
-    	int runner = targetOff;
+    public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {
+        int runner = targetOff;
         int nullFlagsBytes = getNullFlagsBytes(refs.length);
         // write null indicator bits
         for (int i = 0; i < nullFlagsBytes; i++) {
@@ -39,7 +39,7 @@
         // since the r-tree has fixed length keys, we don't actually need this?
         encDec.reset(targetBuf.array(), runner);
         for (int i = startField; i < startField + refs.length; i++) {
-            if (typeTraits[i].getStaticallyKnownDataLength() == ITypeTrait.VARIABLE_LENGTH) {
+            if (!typeTraits[i].isFixedLength()) {
                 encDec.encode(refs[i].getFieldLength(i));
             }
         }
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
index a27d8cd..24b2b53 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
@@ -15,16 +15,16 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.tuples;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
 
 public class RTreeTypeAwareTupleWriterFactory implements ITreeIndexTupleWriterFactory {
 
     private static final long serialVersionUID = 1L;
-    private ITypeTrait[] typeTraits;
+    private ITypeTraits[] typeTraits;
 
-    public RTreeTypeAwareTupleWriterFactory(ITypeTrait[] typeTraits) {
+    public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
         this.typeTraits = typeTraits;
     }
 
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
index b312116..74e4840 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
@@ -15,88 +15,16 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.util;
 
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.FloatBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.FloatBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.FloatPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.IntegerPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.data.PointablePrimitiveValueProviderFactory;
 
 public class RTreeUtils {
-    public static IPrimitiveValueProvider comparatorToPrimitiveValueProvider(IBinaryComparator cmp) {
-        if (cmp instanceof IntegerBinaryComparator) {
-            return IntegerPrimitiveValueProviderFactory.INSTANCE.createPrimitiveValueProvider();
+    public static IPrimitiveValueProviderFactory[] createPrimitiveValueProviderFactories(int len, IPointableFactory pf) {
+        IPrimitiveValueProviderFactory[] pvpfs = new IPrimitiveValueProviderFactory[len];
+        for (int i = 0; i < len; ++i) {
+            pvpfs[i] = new PointablePrimitiveValueProviderFactory(pf);
         }
-        if (cmp instanceof FloatBinaryComparator) {
-            return FloatPrimitiveValueProviderFactory.INSTANCE.createPrimitiveValueProvider();
-        }
-        if (cmp instanceof DoubleBinaryComparator) {
-            return DoublePrimitiveValueProviderFactory.INSTANCE.createPrimitiveValueProvider();
-        }
-        throw new UnsupportedOperationException(
-                "Converting binary comparator to primitive value provider not implemented for: " + cmp.toString());
-    }
-
-    public static IPrimitiveValueProvider[] comparatorsToPrimitiveValueProviders(IBinaryComparator[] cmps) {
-        IPrimitiveValueProvider[] primitiveValueProviders = new IPrimitiveValueProvider[cmps.length];
-        for (int i = 0; i < cmps.length; i++) {
-            primitiveValueProviders[i] = comparatorToPrimitiveValueProvider(cmps[i]);
-        }
-        return primitiveValueProviders;
-    }
-
-    public static IPrimitiveValueProviderFactory comparatorToPrimitiveValueProviderFactory(IBinaryComparator cmp) {
-        if (cmp instanceof IntegerBinaryComparator) {
-            return IntegerPrimitiveValueProviderFactory.INSTANCE;
-        }
-        if (cmp instanceof FloatBinaryComparator) {
-            return FloatPrimitiveValueProviderFactory.INSTANCE;
-        }
-        if (cmp instanceof DoubleBinaryComparator) {
-            return DoublePrimitiveValueProviderFactory.INSTANCE;
-        }
-        throw new UnsupportedOperationException(
-                "Converting binary comparator to primitive value provider factory not implemented for: "
-                        + cmp.toString());
-    }
-
-    public static IPrimitiveValueProviderFactory[] comparatorsToPrimitiveValueProviderFactories(IBinaryComparator[] cmps) {
-        IPrimitiveValueProviderFactory[] primitiveValueProviders = new IPrimitiveValueProviderFactory[cmps.length];
-        for (int i = 0; i < cmps.length; i++) {
-            primitiveValueProviders[i] = comparatorToPrimitiveValueProviderFactory(cmps[i]);
-        }
-        return primitiveValueProviders;
-    }
-
-    public static IPrimitiveValueProviderFactory comparatorFactoryToPrimitiveValueProviderFactory(
-            IBinaryComparatorFactory cmpFactory) {
-        if (cmpFactory instanceof IntegerBinaryComparatorFactory) {
-            return IntegerPrimitiveValueProviderFactory.INSTANCE;
-        }
-        if (cmpFactory instanceof FloatBinaryComparatorFactory) {
-            return FloatPrimitiveValueProviderFactory.INSTANCE;
-        }
-        if (cmpFactory instanceof DoubleBinaryComparatorFactory) {
-            return DoublePrimitiveValueProviderFactory.INSTANCE;
-        }
-        throw new UnsupportedOperationException(
-                "Converting binary comparator factory to primitive value provider factory not implemented for: "
-                        + cmpFactory.toString());
-    }
-
-    public static IPrimitiveValueProviderFactory[] comparatorFactoriesToPrimitiveValueProviderFactories(
-            IBinaryComparatorFactory[] cmpFactories) {
-        IPrimitiveValueProviderFactory[] primitiveValueProviders = new IPrimitiveValueProviderFactory[cmpFactories.length];
-        for (int i = 0; i < cmpFactories.length; i++) {
-            primitiveValueProviders[i] = comparatorFactoryToPrimitiveValueProviderFactory(cmpFactories[i]);
-        }
-        return primitiveValueProviders;
+        return pvpfs;
     }
 }
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
index c14a11d..5bb86b9 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
@@ -22,12 +22,13 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
@@ -63,9 +64,9 @@
 
         // Declare fields.
         int fieldCount = 2;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
         // Declare field serdes.
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
@@ -73,7 +74,7 @@
         // Declare keys.
         int keyFieldCount = 1;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
 
         BTree btree = BTreeUtils
                 .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -140,10 +141,10 @@
 
         // Declare fields.
         int fieldCount = 3;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        typeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
         // Declare field serdes.
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -151,8 +152,8 @@
         // declare keys
         int keyFieldCount = 2;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+        cmps[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
 
         BTree btree = BTreeUtils
                 .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -219,9 +220,9 @@
 
         // Declare fields.
         int fieldCount = 2;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
         // Declare field serdes.
         ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE };
@@ -229,7 +230,7 @@
         // Declare keys.
         int keyFieldCount = 1;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
 
         BTree btree = BTreeUtils
                 .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -298,9 +299,9 @@
 
         // Declare fields.
         int fieldCount = 2;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
         // Declare field serdes.
         ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE };
@@ -308,7 +309,7 @@
         // Declare keys.
         int keyFieldCount = 1;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
 
         BTree btree = BTreeUtils
                 .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -399,9 +400,9 @@
 
         // Declare fields.
         int fieldCount = 2;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
-        typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
         // Declare field serdes.
         ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE };
@@ -409,7 +410,7 @@
         // Declare keys.
         int keyFieldCount = 1;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
 
         BTree btree = BTreeUtils
                 .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
@@ -483,10 +484,10 @@
         }
         // Declare fields.
         int fieldCount = 3;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
-        typeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
         // Declare field serdes.
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -494,8 +495,8 @@
         // declare keys
         int keyFieldCount = 2;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+        cmps[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
 
         BTree btree = BTreeUtils
                 .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index ac85afd..e8aa00a 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -27,16 +27,16 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
@@ -96,17 +96,17 @@
 
         // declare fields
         int fieldCount = 3;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = new TypeTrait(4);
-        typeTraits[1] = new TypeTrait(4);
-        typeTraits[2] = new TypeTrait(4);
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
 
         // declare keys
         int keyFieldCount = 3;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-        cmps[2] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+        cmps[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+        cmps[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
         MultiComparator cmp = new MultiComparator(cmps);
 
         // just for printing
@@ -120,7 +120,7 @@
         try {
 
             ITreeIndexTupleWriter tupleWriter = new TypeAwareTupleWriter(typeTraits);
-            BTreeFieldPrefixNSMLeafFrame frame = new BTreeFieldPrefixNSMLeafFrame(tupleWriter);            
+            BTreeFieldPrefixNSMLeafFrame frame = new BTreeFieldPrefixNSMLeafFrame(tupleWriter);
             frame.setPage(page);
             frame.initBuffer((byte) 0);
             frame.setMultiComparator(cmp);
@@ -148,7 +148,7 @@
                 int a = rnd.nextInt() % smallMax;
                 int b = rnd.nextInt() % smallMax;
                 int c = i;
-                
+
                 ITupleReference tuple = createTuple(ctx, a, b, c, false);
                 try {
                     int targetTupleIndex = frame.findInsertTupleIndex(tuple);
@@ -213,19 +213,19 @@
             bufferCache.unpin(page);
         }
     }
-    
+
     public int getPageSize() {
         return PAGE_SIZE;
     }
-    
+
     public int getNumPages() {
         return NUM_PAGES;
     }
-    
+
     public int getHyracksFrameSize() {
         return HYRACKS_FRAME_SIZE;
     }
-    
+
     public int getMaxOpenFiles() {
         return MAX_OPEN_FILES;
     }
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 922c715..4031e7f 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -29,13 +29,13 @@
 
 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.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
@@ -60,434 +60,390 @@
 import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 
 public class RangeSearchCursorTest extends AbstractBTreeTest {
-	// Declare fields
-	int fieldCount = 2;
-	ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+    // Declare fields
+    int fieldCount = 2;
+    ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
 
-	TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
-			typeTraits);
-	ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-	ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+    TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+    ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+    ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-	Random rnd = new Random(50);
+    Random rnd = new Random(50);
 
-	@Before
-	public void setUp() throws HyracksDataException {
-		super.setUp();
-	    typeTraits[0] = new TypeTrait(4);
-		typeTraits[1] = new TypeTrait(4);
-	}
+    @Before
+    public void setUp() throws HyracksDataException {
+        super.setUp();
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+    }
 
-	@Test
-	public void uniqueIndexTest() throws Exception {
+    @Test
+    public void uniqueIndexTest() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
         }
 
-		// declare keys
-		int keyFieldCount = 1;
-		IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+        // declare keys
+        int keyFieldCount = 1;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
-		ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
-	            tupleWriterFactory);
-	    ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
-	            tupleWriterFactory);
-		
-	    IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
-	    IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
-	    
-		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
-		
-		BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
-		btree.create(btreeFileId);
-		btree.open(btreeFileId);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
 
-		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
-	    ArrayTupleReference tuple = new ArrayTupleReference();
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
 
-	    ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
 
-		// generate keys
-		int numKeys = 50;
-		int maxKey = 1000;
-		TreeSet<Integer> uniqueKeys = new TreeSet<Integer>();
-		ArrayList<Integer> keys = new ArrayList<Integer>();
-		while (uniqueKeys.size() < numKeys) {
-			int key = rnd.nextInt() % maxKey;
-			uniqueKeys.add(key);
-		}
-		for (Integer i : uniqueKeys) {
-			keys.add(i);
-		}
+        BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        btree.create(btreeFileId);
+        btree.open(btreeFileId);
 
-		// insert keys into btree
-		for (int i = 0; i < keys.size(); i++) {
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
 
-		    TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);		    
-			tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
-			try {
-			    indexAccessor.insert(tuple);
-			} catch (BTreeException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        TreeSet<Integer> uniqueKeys = new TreeSet<Integer>();
+        ArrayList<Integer> keys = new ArrayList<Integer>();
+        while (uniqueKeys.size() < numKeys) {
+            int key = rnd.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
 
-		// btree.printTree(leafFrame, interiorFrame, recDescSers);
+        // insert keys into btree
+        for (int i = 0; i < keys.size(); i++) {
 
-		int minSearchKey = -100;
-		int maxSearchKey = 100;
+            TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
+            tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
 
-		// forward searches
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, false, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, false, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, true, false);
+            try {
+                indexAccessor.insert(tuple);
+            } catch (BTreeException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		// backward searches
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, false, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, false, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, true, false);
+        // btree.printTree(leafFrame, interiorFrame, recDescSers);
 
-		btree.close();
-	}
+        int minSearchKey = -100;
+        int maxSearchKey = 100;
 
-	@Test
-	public void nonUniqueIndexTest() throws Exception {
+        // forward searches
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+
+        // backward searches
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, false, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+
+        btree.close();
+    }
+
+    @Test
+    public void nonUniqueIndexTest() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
         }
 
-		// declare keys
-		int keyFieldCount = 2;
-		IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-		cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
+        // declare keys
+        int keyFieldCount = 2;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
-                tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
-                tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
 
-        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
-        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
-        
-		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
-		
-		BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
-		btree.create(btreeFileId);
-		btree.open(btreeFileId);
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
 
-		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
+
+        BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        btree.create(btreeFileId);
+        btree.open(btreeFileId);
+
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
         ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
-		// generate keys
-		int numKeys = 50;
-		int maxKey = 10;
-		ArrayList<Integer> keys = new ArrayList<Integer>();
-		for (int i = 0; i < numKeys; i++) {
-			int k = rnd.nextInt() % maxKey;
-			keys.add(k);
-		}
-		Collections.sort(keys);
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 10;
+        ArrayList<Integer> keys = new ArrayList<Integer>();
+        for (int i = 0; i < numKeys; i++) {
+            int k = rnd.nextInt() % maxKey;
+            keys.add(k);
+        }
+        Collections.sort(keys);
 
-		// insert keys into btree
-		for (int i = 0; i < keys.size(); i++) {
+        // insert keys into btree
+        for (int i = 0; i < keys.size(); i++) {
 
-		    TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);          
+            TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
             tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
 
-			try {
-			    indexAccessor.insert(tuple);
-			} catch (BTreeException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (BTreeException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		// btree.printTree(leafFrame, interiorFrame, recDescSers);
+        // btree.printTree(leafFrame, interiorFrame, recDescSers);
 
-		int minSearchKey = -100;
-		int maxSearchKey = 100;
+        int minSearchKey = -100;
+        int maxSearchKey = 100;
 
-		// forward searches
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, false, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, false, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, true, false);
+        // forward searches
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
 
-		// backward searches
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, false, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, false, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, true, false);
+        // backward searches
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, false, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
 
-		btree.close();
-	}
+        btree.close();
+    }
 
-	@Test
-	public void nonUniqueFieldPrefixIndexTest() throws Exception {
+    @Test
+    public void nonUniqueFieldPrefixIndexTest() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
         }
 
-		// declare keys
-		int keyFieldCount = 2;
-		IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-		cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE	;			
+        // declare keys
+        int keyFieldCount = 2;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
-                tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
-                tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
 
-        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
-        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
 
-		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);		
-		
-		BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
-		btree.create(btreeFileId);
-		btree.open(btreeFileId);
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
 
-		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        btree.create(btreeFileId);
+        btree.open(btreeFileId);
+
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
         ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
-		// generate keys
-		int numKeys = 50;
-		int maxKey = 10;
-		ArrayList<Integer> keys = new ArrayList<Integer>();
-		for (int i = 0; i < numKeys; i++) {
-			int k = rnd.nextInt() % maxKey;
-			keys.add(k);
-		}
-		Collections.sort(keys);
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 10;
+        ArrayList<Integer> keys = new ArrayList<Integer>();
+        for (int i = 0; i < numKeys; i++) {
+            int k = rnd.nextInt() % maxKey;
+            keys.add(k);
+        }
+        Collections.sort(keys);
 
-		// insert keys into btree
-		for (int i = 0; i < keys.size(); i++) {
+        // insert keys into btree
+        for (int i = 0; i < keys.size(); i++) {
 
-		    TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);          
+            TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
             tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
 
-			try {
-			    indexAccessor.insert(tuple);
-			} catch (BTreeException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (BTreeException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		// btree.printTree(leafFrame, interiorFrame, recDescSers);
+        // btree.printTree(leafFrame, interiorFrame, recDescSers);
 
-		int minSearchKey = -100;
-		int maxSearchKey = 100;
+        int minSearchKey = -100;
+        int maxSearchKey = 100;
 
-		// forward searches
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, false, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, false, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, true, true, true, false);
+        // forward searches
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, true, false);
 
-		// backward searches
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, false, true, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, false, false);
-		performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey,
-				maxSearchKey, false, true, true, false);
+        // backward searches
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, false, true, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false, false);
+        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, true, false);
 
-		btree.close();
-	}
+        btree.close();
+    }
 
-	public RangePredicate createRangePredicate(int lk, int hk,
-			boolean isForward, boolean lowKeyInclusive,
-			boolean highKeyInclusive, MultiComparator cmp) throws HyracksDataException {
+    public RangePredicate createRangePredicate(int lk, int hk, boolean isForward, boolean lowKeyInclusive,
+            boolean highKeyInclusive, MultiComparator cmp) throws HyracksDataException {
 
-		// create tuplereferences for search keys
-		ITupleReference lowKey = TupleUtils.createIntegerTuple(lk);
-		ITupleReference highKey = TupleUtils.createIntegerTuple(hk);
+        // create tuplereferences for search keys
+        ITupleReference lowKey = TupleUtils.createIntegerTuple(lk);
+        ITupleReference highKey = TupleUtils.createIntegerTuple(hk);
 
-		IBinaryComparator[] searchCmps = new IBinaryComparator[1];
-		searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
-		MultiComparator searchCmp = new MultiComparator(searchCmps);
+        IBinaryComparator[] searchCmps = new IBinaryComparator[1];
+        searchCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+        MultiComparator searchCmp = new MultiComparator(searchCmps);
 
-		RangePredicate rangePred = new RangePredicate(isForward, lowKey,
-				highKey, lowKeyInclusive, highKeyInclusive, searchCmp,
-				searchCmp);
-		return rangePred;
-	}
+        RangePredicate rangePred = new RangePredicate(isForward, lowKey, highKey, lowKeyInclusive, highKeyInclusive,
+                searchCmp, searchCmp);
+        return rangePred;
+    }
 
-	public void getExpectedResults(ArrayList<Integer> expectedResults,
-			ArrayList<Integer> keys, int lk, int hk, boolean isForward,
-			boolean lowKeyInclusive, boolean highKeyInclusive) {
+    public void getExpectedResults(ArrayList<Integer> expectedResults, ArrayList<Integer> keys, int lk, int hk,
+            boolean isForward, boolean lowKeyInclusive, boolean highKeyInclusive) {
 
-		// special cases
-		if (lk == hk && (!lowKeyInclusive || !highKeyInclusive))
-			return;
-		if (lk > hk)
-			return;
+        // special cases
+        if (lk == hk && (!lowKeyInclusive || !highKeyInclusive))
+            return;
+        if (lk > hk)
+            return;
 
-		if (isForward) {
-			for (int i = 0; i < keys.size(); i++) {
-				if ((lk == keys.get(i) && lowKeyInclusive)
-						|| (hk == keys.get(i) && highKeyInclusive)) {
-					expectedResults.add(keys.get(i));
-					continue;
-				}
+        if (isForward) {
+            for (int i = 0; i < keys.size(); i++) {
+                if ((lk == keys.get(i) && lowKeyInclusive) || (hk == keys.get(i) && highKeyInclusive)) {
+                    expectedResults.add(keys.get(i));
+                    continue;
+                }
 
-				if (lk < keys.get(i) && hk > keys.get(i)) {
-					expectedResults.add(keys.get(i));
-					continue;
-				}
-			}
-		} else {
-			for (int i = keys.size() - 1; i >= 0; i--) {
-				if ((lk == keys.get(i) && lowKeyInclusive)
-						|| (hk == keys.get(i) && highKeyInclusive)) {
-					expectedResults.add(keys.get(i));
-					continue;
-				}
+                if (lk < keys.get(i) && hk > keys.get(i)) {
+                    expectedResults.add(keys.get(i));
+                    continue;
+                }
+            }
+        } else {
+            for (int i = keys.size() - 1; i >= 0; i--) {
+                if ((lk == keys.get(i) && lowKeyInclusive) || (hk == keys.get(i) && highKeyInclusive)) {
+                    expectedResults.add(keys.get(i));
+                    continue;
+                }
 
-				if (lk < keys.get(i) && hk > keys.get(i)) {
-					expectedResults.add(keys.get(i));
-					continue;
-				}
-			}
-		}
-	}
+                if (lk < keys.get(i) && hk > keys.get(i)) {
+                    expectedResults.add(keys.get(i));
+                    continue;
+                }
+            }
+        }
+    }
 
-	public boolean performSearches(ArrayList<Integer> keys, BTree btree,
-			IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
-			int minKey, int maxKey, boolean isForward, boolean lowKeyInclusive,
-			boolean highKeyInclusive, boolean printExpectedResults)
-			throws Exception {
+    public boolean performSearches(ArrayList<Integer> keys, BTree btree, IBTreeLeafFrame leafFrame,
+            IBTreeInteriorFrame interiorFrame, int minKey, int maxKey, boolean isForward, boolean lowKeyInclusive,
+            boolean highKeyInclusive, boolean printExpectedResults) throws Exception {
 
-		ArrayList<Integer> results = new ArrayList<Integer>();
-		ArrayList<Integer> expectedResults = new ArrayList<Integer>();
+        ArrayList<Integer> results = new ArrayList<Integer>();
+        ArrayList<Integer> expectedResults = new ArrayList<Integer>();
 
-		for (int i = minKey; i < maxKey; i++) {
-			for (int j = minKey; j < maxKey; j++) {
+        for (int i = minKey; i < maxKey; i++) {
+            for (int j = minKey; j < maxKey; j++) {
 
-				results.clear();
-				expectedResults.clear();
+                results.clear();
+                expectedResults.clear();
 
-				int lowKey = i;
-				int highKey = j;
+                int lowKey = i;
+                int highKey = j;
 
-				ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
-				RangePredicate rangePred = createRangePredicate(lowKey,
-						highKey, isForward, lowKeyInclusive, highKeyInclusive,
-						btree.getMultiComparator());
-				ITreeIndexAccessor indexAccessor = btree.createAccessor();
-				indexAccessor.search(rangeCursor, rangePred);
+                ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
+                RangePredicate rangePred = createRangePredicate(lowKey, highKey, isForward, lowKeyInclusive,
+                        highKeyInclusive, btree.getMultiComparator());
+                ITreeIndexAccessor indexAccessor = btree.createAccessor();
+                indexAccessor.search(rangeCursor, rangePred);
 
-				try {
-					while (rangeCursor.hasNext()) {
-						rangeCursor.next();
-						ITupleReference frameTuple = rangeCursor.getTuple();
-						ByteArrayInputStream inStream = new ByteArrayInputStream(
-								frameTuple.getFieldData(0), frameTuple
-										.getFieldStart(0), frameTuple
-										.getFieldLength(0));
-						DataInput dataIn = new DataInputStream(inStream);
-						Integer res = IntegerSerializerDeserializer.INSTANCE
-								.deserialize(dataIn);
-						results.add(res);
-					}
-				} catch (Exception e) {
-					e.printStackTrace();
-				} finally {
-					rangeCursor.close();
-				}
+                try {
+                    while (rangeCursor.hasNext()) {
+                        rangeCursor.next();
+                        ITupleReference frameTuple = rangeCursor.getTuple();
+                        ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
+                                frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
+                        DataInput dataIn = new DataInputStream(inStream);
+                        Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
+                        results.add(res);
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                } finally {
+                    rangeCursor.close();
+                }
 
-				getExpectedResults(expectedResults, keys, lowKey, highKey,
-						isForward, lowKeyInclusive, highKeyInclusive);
+                getExpectedResults(expectedResults, keys, lowKey, highKey, isForward, lowKeyInclusive, highKeyInclusive);
 
-				if (printExpectedResults) {
-					if (expectedResults.size() > 0) {
-						char l, u;
+                if (printExpectedResults) {
+                    if (expectedResults.size() > 0) {
+                        char l, u;
 
-						if (lowKeyInclusive)
-							l = '[';
-						else
-							l = '(';
+                        if (lowKeyInclusive)
+                            l = '[';
+                        else
+                            l = '(';
 
-						if (highKeyInclusive)
-							u = ']';
-						else
-							u = ')';
+                        if (highKeyInclusive)
+                            u = ']';
+                        else
+                            u = ')';
 
                         if (LOGGER.isLoggable(Level.INFO)) {
                             LOGGER.info("RANGE: " + l + " " + lowKey + " , " + highKey + " " + u);
                         }
-						StringBuilder strBuilder = new StringBuilder();
-						for (Integer r : expectedResults) {
-							strBuilder.append(r + " ");
-						}
+                        StringBuilder strBuilder = new StringBuilder();
+                        for (Integer r : expectedResults) {
+                            strBuilder.append(r + " ");
+                        }
                         if (LOGGER.isLoggable(Level.INFO)) {
                             LOGGER.info(strBuilder.toString());
                         }
-					}
-				}
+                    }
+                }
 
-				if (results.size() == expectedResults.size()) {
-					for (int k = 0; k < results.size(); k++) {
-						if (!results.get(k).equals(expectedResults.get(k))) {
+                if (results.size() == expectedResults.size()) {
+                    for (int k = 0; k < results.size(); k++) {
+                        if (!results.get(k).equals(expectedResults.get(k))) {
                             if (LOGGER.isLoggable(Level.INFO)) {
                                 LOGGER.info("DIFFERENT RESULTS AT: i=" + i + " j=" + j + " k=" + k);
                                 LOGGER.info(results.get(k) + " " + expectedResults.get(k));
                             }
-							return false;
-						}
-					}
-				} else {
+                            return false;
+                        }
+                    }
+                } else {
                     if (LOGGER.isLoggable(Level.INFO)) {
                         LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i + " j=" + j);
                         LOGGER.info("RESULTS: " + results.size());
                         LOGGER.info("EXPECTED RESULTS: " + expectedResults.size());
                     }
-					return false;
-				}
-			}
-		}
+                    return false;
+                }
+            }
+        }
 
-		return true;
-	}
+        return true;
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
index dc524d7..1ae80d6 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
@@ -12,15 +12,15 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -71,14 +71,14 @@
 
         // declare fields
         int fieldCount = 2;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = new TypeTrait(4);
-        typeTraits[1] = new TypeTrait(4);
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
 
         // declare keys
         int keyFieldCount = 1;
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-        cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
         MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
index 7a61adc..5d43ae1 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
@@ -7,12 +7,12 @@
 
 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.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -35,28 +35,28 @@
 import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 
 public class UpdateSearchTest extends AbstractBTreeTest {
-        
+
     // Update scan test on fixed-length tuples.
     @Test
     public void test01() throws Exception {
         // declare fields
         int fieldCount = 2;
-        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-        typeTraits[0] = new TypeTrait(4);
-        typeTraits[1] = new TypeTrait(4);
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
 
         // declare keys
         int keyFieldCount = 1;
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-        cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;        
-        
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
         MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
-        
+
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
@@ -79,7 +79,7 @@
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference insertTuple = new ArrayTupleReference();
         ITreeIndexAccessor indexAccessor = btree.createAccessor();
-        
+
         int numInserts = 10000;
         for (int i = 0; i < 10000; i++) {
             int f0 = rnd.nextInt() % 10000;
@@ -104,7 +104,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("DURATION: " + duration);
         }
-        
+
         // Update scan.
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("UPDATE SCAN:");
@@ -125,7 +125,7 @@
         } finally {
             updateScanCursor.close();
         }
-        
+
         // Ordered scan to verify the values.
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("ORDERED SCAN:");
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
index ca543a2..b5186b0 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
@@ -15,7 +15,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
@@ -46,7 +46,7 @@
     private static final Logger LOGGER = Logger.getLogger(BTreeTestUtils.class.getName());    
     
     public static BTreeTestContext createBTreeTestContext(IBufferCache bufferCache, int btreeFileId, ISerializerDeserializer[] fieldSerdes, int numKeyFields, BTreeLeafFrameType leafType) throws Exception {        
-        ITypeTrait[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, fieldSerdes.length);
+        ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, fieldSerdes.length);
         IBinaryComparator[] cmps = SerdeUtils.serdesToComparators(fieldSerdes, numKeyFields);
         
         BTree btree = BTreeUtils.createBTree(bufferCache, btreeFileId, typeTraits, cmps, leafType);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index 1f1cc25..7b6bd9f 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -28,17 +28,17 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
@@ -81,8 +81,8 @@
     protected int btreeFileId;
 
     // declare token type traits
-    protected ITypeTrait[] tokenTypeTraits = new ITypeTrait[] { ITypeTrait.VARLEN_TYPE_TRAIT };
-    protected ITypeTrait[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
+    protected ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+    protected ITypeTraits[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
 
     // declare btree keys
     protected int btreeKeyFieldCount = 1;
@@ -109,7 +109,7 @@
     protected int invListsFileId;
 
     protected int invListFields = 1;
-    protected ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
+    protected ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFields];
 
     protected int invListKeys = 1;
     protected IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
@@ -169,11 +169,12 @@
         btreeFileId = fmp.lookupFileId(btreeFile);
         bufferCache.openFile(btreeFileId);
 
-        btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        btreeBinCmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
 
         freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
 
-        btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory,
+                leafFrameFactory);
         btree.create(btreeFileId);
         btree.open(btreeFileId);
 
@@ -183,8 +184,8 @@
         invListsFileId = fmp.lookupFileId(invListsFile);
         bufferCache.openFile(invListsFileId);
 
-        invListTypeTraits[0] = new TypeTrait(4);
-        invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        invListTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        invListBinCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
 
         invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmp);
         invIndex.open(invListsFileId);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index bbbb743..93934ed 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -31,17 +31,17 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -104,13 +104,13 @@
         bufferCache.openFile(invListsFileId);
 
         // Declare token type traits, and compute BTree type traits.
-        ITypeTrait[] tokenTypeTraits = new ITypeTrait[] { ITypeTrait.VARLEN_TYPE_TRAIT }; 
-        ITypeTrait[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
+        ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+        ITypeTraits[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
 
         // declare btree keys
         int keyFieldCount = 1;
         IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-        cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        cmps[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY).createBinaryComparator();
 
         MultiComparator btreeCmp = new MultiComparator(cmps);
 
@@ -123,17 +123,18 @@
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
 
-        BTree btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        BTree btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory,
+                leafFrameFactory);
         btree.create(btreeFileId);
         btree.open(btreeFileId);
 
         int invListFields = 1;
-        ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
-        invListTypeTraits[0] = new TypeTrait(4);
+        ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFields];
+        invListTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
 
         int invListKeys = 1;
         IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
-        invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        invListBinCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
 
         MultiComparator invListCmp = new MultiComparator(invListBinCmps);
 
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
index 0b49ec1..9c7ec09 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
@@ -23,56 +23,53 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeFrameTupleAccessor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeFrameTupleAppender;
 
 public class FixedSizeFrameTupleTest {
 
-	private static int FRAME_SIZE = 4096;
+    private static int FRAME_SIZE = 4096;
 
-	private Random rnd = new Random(50);
+    private Random rnd = new Random(50);
 
-	/**
-	 * This test verifies the correct behavior of the FixedSizeFrameTuple class.
-	 * Frames containing FixedSizeFrameTuple's require neither tuple slots nor
-	 * field slots. The tests inserts generated data into a frame until the
-	 * frame is full, and then verifies the frame's contents.
-	 * 
-	 */
-	@Test
-	public void singleFieldTest() throws Exception {
-		ByteBuffer buffer = ByteBuffer.allocate(FRAME_SIZE);
+    /**
+     * This test verifies the correct behavior of the FixedSizeFrameTuple class.
+     * Frames containing FixedSizeFrameTuple's require neither tuple slots nor
+     * field slots. The tests inserts generated data into a frame until the
+     * frame is full, and then verifies the frame's contents.
+     * 
+     */
+    @Test
+    public void singleFieldTest() throws Exception {
+        ByteBuffer buffer = ByteBuffer.allocate(FRAME_SIZE);
 
-		ITypeTrait[] fields = new TypeTrait[1];
-		fields[0] = new TypeTrait(4);
+        ITypeTraits[] fields = new ITypeTraits[1];
+        fields[0] = IntegerPointable.TYPE_TRAITS;
 
-		FixedSizeFrameTupleAppender ftapp = new FixedSizeFrameTupleAppender(
-				FRAME_SIZE, fields);
-		FixedSizeFrameTupleAccessor ftacc = new FixedSizeFrameTupleAccessor(
-				FRAME_SIZE, fields);
+        FixedSizeFrameTupleAppender ftapp = new FixedSizeFrameTupleAppender(FRAME_SIZE, fields);
+        FixedSizeFrameTupleAccessor ftacc = new FixedSizeFrameTupleAccessor(FRAME_SIZE, fields);
 
-		boolean frameHasSpace = true;
+        boolean frameHasSpace = true;
 
-		ArrayList<Integer> check = new ArrayList<Integer>();
+        ArrayList<Integer> check = new ArrayList<Integer>();
 
-		ftapp.reset(buffer, true);
-		while (frameHasSpace) {
-			int val = rnd.nextInt();
-			frameHasSpace = ftapp.append(val);
-			if (frameHasSpace) {
-				check.add(val);
-				ftapp.incrementTupleCount(1);
-			}
-		}
+        ftapp.reset(buffer, true);
+        while (frameHasSpace) {
+            int val = rnd.nextInt();
+            frameHasSpace = ftapp.append(val);
+            if (frameHasSpace) {
+                check.add(val);
+                ftapp.incrementTupleCount(1);
+            }
+        }
 
-		ftacc.reset(buffer);
-		for (int i = 0; i < ftacc.getTupleCount(); i++) {
-			int val = IntegerSerializerDeserializer.getInt(ftacc.getBuffer()
-					.array(), ftacc.getTupleStartOffset(i));
-			Assert.assertEquals(check.get(i).intValue(), val);
-		}
-	}
+        ftacc.reset(buffer);
+        for (int i = 0; i < ftacc.getTupleCount(); i++) {
+            int val = IntegerSerializerDeserializer.getInt(ftacc.getBuffer().array(), ftacc.getTupleStartOffset(i));
+            Assert.assertEquals(check.get(i).intValue(), val);
+        }
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index faf1e88..9fe2c94 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -27,17 +27,17 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
@@ -66,792 +66,704 @@
 import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class RTreeTest extends AbstractRTreeTest {
-	private static final int PAGE_SIZE = 256;
-	private static final int NUM_PAGES = 10;
-	private static final int MAX_OPEN_FILES = 10;
-	private static final int HYRACKS_FRAME_SIZE = 128;
-	private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    private static final int PAGE_SIZE = 256;
+    private static final int NUM_PAGES = 10;
+    private static final int MAX_OPEN_FILES = 10;
+    private static final int HYRACKS_FRAME_SIZE = 128;
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
-	// create an R-tree of two dimensions
-	// fill the R-tree with random values using insertions
-	// perform ordered scan
-	@Test
-	public void test01() throws Exception {
+    // create an R-tree of two dimensions
+    // fill the R-tree with random values using insertions
+    // perform ordered scan
+    @Test
+    public void test01() throws Exception {
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		// declare keys
-		int keyFieldCount = 4;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
-		cmps[1] = cmps[0];
-		cmps[2] = cmps[0];
-		cmps[3] = cmps[0];
+        // declare keys
+        int keyFieldCount = 4;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+        cmps[1] = cmps[0];
+        cmps[2] = cmps[0];
+        cmps[3] = cmps[0];
 
-		// declare tuple fields
-		int fieldCount = 7;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		typeTraits[0] = new TypeTrait(8);
-		typeTraits[1] = new TypeTrait(8);
-		typeTraits[2] = new TypeTrait(8);
-		typeTraits[3] = new TypeTrait(8);
-		typeTraits[4] = new TypeTrait(8);
-		typeTraits[5] = new TypeTrait(4);
-		typeTraits[6] = new TypeTrait(8);
+        // declare tuple fields
+        int fieldCount = 7;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = DoublePointable.TYPE_TRAITS;
+        typeTraits[1] = DoublePointable.TYPE_TRAITS;
+        typeTraits[2] = DoublePointable.TYPE_TRAITS;
+        typeTraits[3] = DoublePointable.TYPE_TRAITS;
+        typeTraits[4] = DoublePointable.TYPE_TRAITS;
+        typeTraits[5] = DoublePointable.TYPE_TRAITS;
+        typeTraits[6] = DoublePointable.TYPE_TRAITS;
 
-		// create value providers
-		IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps); 
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                cmps.length, DoublePointable.FACTORY);
 
-		MultiComparator cmp = new MultiComparator(cmps);
+        MultiComparator cmp = new MultiComparator(cmps);
 
-		RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-				typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
 
-		ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
-				.createFrame();
-		IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, fileId, 0, metaFrameFactory);
+        IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+        IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 
-		RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
-				interiorFrameFactory, leafFrameFactory);
-		rtree.create(fileId);
-		rtree.open(fileId);
+        RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        rtree.create(fileId);
+        rtree.open(fileId);
 
-		ByteBuffer hyracksFrame = ctx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
-		DataOutput dos = tb.getDataOutput();
+        ByteBuffer hyracksFrame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        DataOutput dos = tb.getDataOutput();
 
-		@SuppressWarnings("rawtypes")
-		ISerializerDeserializer[] recDescSers = {
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE };
-		RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(
-				ctx.getFrameSize(), recDesc);
-		accessor.reset(hyracksFrame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor.reset(hyracksFrame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
+        Random rnd = new Random();
+        rnd.setSeed(50);
 
-		Random rnd2 = new Random();
-		rnd2.setSeed(50);
-		for (int i = 0; i < 5000; i++) {
+        Random rnd2 = new Random();
+        rnd2.setSeed(50);
+        for (int i = 0; i < 5000; i++) {
 
-			double p1x = rnd.nextDouble();
-			double p1y = rnd.nextDouble();
-			double p2x = rnd.nextDouble();
-			double p2y = rnd.nextDouble();
+            double p1x = rnd.nextDouble();
+            double p1y = rnd.nextDouble();
+            double p2x = rnd.nextDouble();
+            double p2y = rnd.nextDouble();
 
-			double pk1 = rnd2.nextDouble();
-			int pk2 = rnd2.nextInt();
-			double pk3 = rnd2.nextDouble();
+            double pk1 = rnd2.nextDouble();
+            int pk2 = rnd2.nextInt();
+            double pk3 = rnd2.nextDouble();
 
-			tb.reset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
+            tuple.reset(accessor, 0);
 
-			if (LOGGER.isLoggable(Level.INFO)) {
-				if (i % 1000 == 0) {
-					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x)
-							+ " " + Math.min(p1y, p2y) + " "
-							+ Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
-				}
-			}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                            + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+                }
+            }
 
-			try {
-				indexAccessor.insert(tuple);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		String rtreeStats = rtree.printStats();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(rtreeStats);
-		}
+        String rtreeStats = rtree.printStats();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(rtreeStats);
+        }
 
-		// disk-order scan
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("DISK-ORDER SCAN:");
-		}
-		TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
-				leafFrame);
-		indexAccessor.diskOrderScan(diskOrderCursor);
-		try {
-			while (diskOrderCursor.hasNext()) {
-				diskOrderCursor.next();
-				ITupleReference frameTuple = diskOrderCursor.getTuple();
-				String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-				if (LOGGER.isLoggable(Level.INFO)) {
-					LOGGER.info(rec);
-				}
-			}
-		} catch (Exception e) {
-			e.printStackTrace();
-		} finally {
-			diskOrderCursor.close();
-		}
+        // disk-order scan
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DISK-ORDER SCAN:");
+        }
+        TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+        indexAccessor.diskOrderScan(diskOrderCursor);
+        try {
+            while (diskOrderCursor.hasNext()) {
+                diskOrderCursor.next();
+                ITupleReference frameTuple = diskOrderCursor.getTuple();
+                String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            diskOrderCursor.close();
+        }
 
-		TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
-				bufferCache, freePageManager, fileId, rtree.getRootPageId());
-		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
-				interiorFrame, metaFrame);
-		String string = stats.toString();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(string);
-		}
+        TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+                rtree.getRootPageId());
+        TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+        String string = stats.toString();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(string);
+        }
 
-		rtree.close();
-		bufferCache.closeFile(fileId);
-		bufferCache.close();
+        rtree.close();
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
 
-	}
+    }
 
-	// create an R-tree of two dimensions
-	// fill the R-tree with random values using insertions
-	// and then delete all the tuples which result of an empty R-tree
-	@Test
-	public void test02() throws Exception {
+    // create an R-tree of two dimensions
+    // fill the R-tree with random values using insertions
+    // and then delete all the tuples which result of an empty R-tree
+    @Test
+    public void test02() throws Exception {
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		// declare keys
-		int keyFieldCount = 4;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
-		cmps[1] = cmps[0];
-		cmps[2] = cmps[0];
-		cmps[3] = cmps[0];
+        // declare keys
+        int keyFieldCount = 4;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+        cmps[1] = cmps[0];
+        cmps[2] = cmps[0];
+        cmps[3] = cmps[0];
 
-		// declare tuple fields
-		int fieldCount = 7;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		typeTraits[0] = new TypeTrait(8);
-		typeTraits[1] = new TypeTrait(8);
-		typeTraits[2] = new TypeTrait(8);
-		typeTraits[3] = new TypeTrait(8);
-		typeTraits[4] = new TypeTrait(8);
-		typeTraits[5] = new TypeTrait(4);
-		typeTraits[6] = new TypeTrait(8);
+        // declare tuple fields
+        int fieldCount = 7;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = DoublePointable.TYPE_TRAITS;
+        typeTraits[1] = DoublePointable.TYPE_TRAITS;
+        typeTraits[2] = DoublePointable.TYPE_TRAITS;
+        typeTraits[3] = DoublePointable.TYPE_TRAITS;
+        typeTraits[4] = DoublePointable.TYPE_TRAITS;
+        typeTraits[5] = DoublePointable.TYPE_TRAITS;
+        typeTraits[6] = DoublePointable.TYPE_TRAITS;
 
-		// create value providers
-		IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps); 
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                cmps.length, DoublePointable.FACTORY);
 
-		MultiComparator cmp = new MultiComparator(cmps);
+        MultiComparator cmp = new MultiComparator(cmps);
 
-		RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-				typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
 
-		ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
-				.createFrame();
-		IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, fileId, 0, metaFrameFactory);
+        IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+        IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 
-		RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
-				interiorFrameFactory, leafFrameFactory);
-		rtree.create(fileId);
-		rtree.open(fileId);
+        RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        rtree.create(fileId);
+        rtree.open(fileId);
 
-		ByteBuffer hyracksFrame = ctx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
-		DataOutput dos = tb.getDataOutput();
+        ByteBuffer hyracksFrame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        DataOutput dos = tb.getDataOutput();
 
-		@SuppressWarnings("rawtypes")
-		ISerializerDeserializer[] recDescSers = {
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE };
-		RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(
-				ctx.getFrameSize(), recDesc);
-		accessor.reset(hyracksFrame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor.reset(hyracksFrame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
+        Random rnd = new Random();
+        rnd.setSeed(50);
 
-		for (int i = 0; i < 5000; i++) {
+        for (int i = 0; i < 5000; i++) {
 
-			double p1x = rnd.nextDouble();
-			double p1y = rnd.nextDouble();
-			double p2x = rnd.nextDouble();
-			double p2y = rnd.nextDouble();
+            double p1x = rnd.nextDouble();
+            double p1y = rnd.nextDouble();
+            double p2x = rnd.nextDouble();
+            double p2y = rnd.nextDouble();
 
-			double pk1 = rnd.nextDouble();
-			int pk2 = rnd.nextInt();
-			double pk3 = rnd.nextDouble();
+            double pk1 = rnd.nextDouble();
+            int pk2 = rnd.nextInt();
+            double pk3 = rnd.nextDouble();
 
-			tb.reset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
+            tuple.reset(accessor, 0);
 
-			if (LOGGER.isLoggable(Level.INFO)) {
-				if (i % 1000 == 0) {
-					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
-							+ " " + Math.max(p1y, p2y));
-				}
-			}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                            + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+                }
+            }
 
-			try {
-				indexAccessor.insert(tuple);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		String rtreeStats = rtree.printStats();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(rtreeStats);
-		}
+        String rtreeStats = rtree.printStats();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(rtreeStats);
+        }
 
-		rnd.setSeed(50);
-		for (int i = 0; i < 5000; i++) {
+        rnd.setSeed(50);
+        for (int i = 0; i < 5000; i++) {
 
-			double p1x = rnd.nextDouble();
-			double p1y = rnd.nextDouble();
-			double p2x = rnd.nextDouble();
-			double p2y = rnd.nextDouble();
+            double p1x = rnd.nextDouble();
+            double p1y = rnd.nextDouble();
+            double p2x = rnd.nextDouble();
+            double p2y = rnd.nextDouble();
 
-			double pk1 = rnd.nextDouble();
-			int pk2 = rnd.nextInt();
-			double pk3 = rnd.nextDouble();
+            double pk1 = rnd.nextDouble();
+            int pk2 = rnd.nextInt();
+            double pk3 = rnd.nextDouble();
 
-			tb.reset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
-			
-			if (LOGGER.isLoggable(Level.INFO)) {
-				if (i % 1000 == 0) {
-					LOGGER.info("DELETING " + i + " " + Math.min(p1x, p2x) + " "
-							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
-							+ " " + Math.max(p1y, p2y));
-				}
-			}
+            tuple.reset(accessor, 0);
 
-			try {
-				indexAccessor.delete(tuple);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("DELETING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                            + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+                }
+            }
 
-		TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
-				bufferCache, freePageManager, fileId, rtree.getRootPageId());
-		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
-				interiorFrame, metaFrame);
-		String string = stats.toString();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(string);
-		}
+            try {
+                indexAccessor.delete(tuple);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+
+        TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+                rtree.getRootPageId());
+        TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+        String string = stats.toString();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(string);
+        }
 
-		rtree.close();
-		bufferCache.closeFile(fileId);
-		bufferCache.close();
+        rtree.close();
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
 
-	}
+    }
 
-	// create an R-tree of three dimensions
-	// fill the R-tree with random values using insertions
-	// perform ordered scan
-	@Test
-	public void test03() throws Exception {
+    // create an R-tree of three dimensions
+    // fill the R-tree with random values using insertions
+    // perform ordered scan
+    @Test
+    public void test03() throws Exception {
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		// declare keys
-		int keyFieldCount = 6;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
-		cmps[1] = cmps[0];
-		cmps[2] = cmps[0];
-		cmps[3] = cmps[0];
-		cmps[4] = cmps[0];
-		cmps[5] = cmps[0];
+        // declare keys
+        int keyFieldCount = 6;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+        cmps[1] = cmps[0];
+        cmps[2] = cmps[0];
+        cmps[3] = cmps[0];
+        cmps[4] = cmps[0];
+        cmps[5] = cmps[0];
 
-		// declare tuple fields
-		int fieldCount = 9;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		typeTraits[0] = new TypeTrait(8);
-		typeTraits[1] = new TypeTrait(8);
-		typeTraits[2] = new TypeTrait(8);
-		typeTraits[3] = new TypeTrait(8);
-		typeTraits[4] = new TypeTrait(8);
-		typeTraits[5] = new TypeTrait(8);
-		typeTraits[6] = new TypeTrait(8);
-		typeTraits[7] = new TypeTrait(4);
-		typeTraits[8] = new TypeTrait(8);
+        // declare tuple fields
+        int fieldCount = 9;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = DoublePointable.TYPE_TRAITS;
+        typeTraits[1] = DoublePointable.TYPE_TRAITS;
+        typeTraits[2] = DoublePointable.TYPE_TRAITS;
+        typeTraits[3] = DoublePointable.TYPE_TRAITS;
+        typeTraits[4] = DoublePointable.TYPE_TRAITS;
+        typeTraits[5] = DoublePointable.TYPE_TRAITS;
+        typeTraits[6] = DoublePointable.TYPE_TRAITS;
+        typeTraits[7] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[8] = DoublePointable.TYPE_TRAITS;
 
-		// create value providers
-		IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps); 
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                cmps.length, DoublePointable.FACTORY);
 
-		MultiComparator cmp = new MultiComparator(cmps);
+        MultiComparator cmp = new MultiComparator(cmps);
 
-		RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-				typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
 
-		ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
-				.createFrame();
-		IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, fileId, 0, metaFrameFactory);
+        IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+        IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 
-		RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
-				interiorFrameFactory, leafFrameFactory);
-		rtree.create(fileId);
-		rtree.open(fileId);
+        RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        rtree.create(fileId);
+        rtree.open(fileId);
 
-		ByteBuffer hyracksFrame = ctx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
-		DataOutput dos = tb.getDataOutput();
+        ByteBuffer hyracksFrame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        DataOutput dos = tb.getDataOutput();
 
-		@SuppressWarnings("rawtypes")
-		ISerializerDeserializer[] recDescSers = {
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE };
-		RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(
-				ctx.getFrameSize(), recDesc);
-		accessor.reset(hyracksFrame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor.reset(hyracksFrame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
+        Random rnd = new Random();
+        rnd.setSeed(50);
 
-		for (int i = 0; i < 5000; i++) {
+        for (int i = 0; i < 5000; i++) {
 
-			double p1x = rnd.nextDouble();
-			double p1y = rnd.nextDouble();
-			double p1z = rnd.nextDouble();
-			double p2x = rnd.nextDouble();
-			double p2y = rnd.nextDouble();
-			double p2z = rnd.nextDouble();
+            double p1x = rnd.nextDouble();
+            double p1y = rnd.nextDouble();
+            double p1z = rnd.nextDouble();
+            double p2x = rnd.nextDouble();
+            double p2y = rnd.nextDouble();
+            double p2z = rnd.nextDouble();
 
-			double pk1 = rnd.nextDouble();
-			int pk2 = rnd.nextInt();
-			double pk3 = rnd.nextDouble();
+            double pk1 = rnd.nextDouble();
+            int pk2 = rnd.nextInt();
+            double pk3 = rnd.nextDouble();
 
-			tb.reset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1z, p2z),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1z, p2z),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1z, p2z), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1z, p2z), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
+            tuple.reset(accessor, 0);
 
-			if (LOGGER.isLoggable(Level.INFO)) {
-				if (i % 1000 == 0) {
-					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-							+ Math.min(p1y, p2y) + " " + Math.min(p1z, p2z)
-							+ " " + " " + Math.max(p1x, p2x) + " "
-							+ Math.max(p1y, p2y) + " " + Math.max(p1z, p2z));
-				}
-			}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                            + Math.min(p1z, p2z) + " " + " " + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y) + " "
+                            + Math.max(p1z, p2z));
+                }
+            }
 
-			try {
-				indexAccessor.insert(tuple);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		String rtreeStats = rtree.printStats();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(rtreeStats);
-		}
+        String rtreeStats = rtree.printStats();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(rtreeStats);
+        }
 
-		// disk-order scan
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("DISK-ORDER SCAN:");
-		}
-		TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
-				leafFrame);
-		indexAccessor.diskOrderScan(diskOrderCursor);
-		try {
-			while (diskOrderCursor.hasNext()) {
-				diskOrderCursor.next();
-				ITupleReference frameTuple = diskOrderCursor.getTuple();
-				String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-				if (LOGGER.isLoggable(Level.INFO)) {
-					LOGGER.info(rec);
-				}
-			}
-		} catch (Exception e) {
-			e.printStackTrace();
-		} finally {
-			diskOrderCursor.close();
-		}
+        // disk-order scan
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DISK-ORDER SCAN:");
+        }
+        TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+        indexAccessor.diskOrderScan(diskOrderCursor);
+        try {
+            while (diskOrderCursor.hasNext()) {
+                diskOrderCursor.next();
+                ITupleReference frameTuple = diskOrderCursor.getTuple();
+                String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            diskOrderCursor.close();
+        }
 
-		TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
-				bufferCache, freePageManager, fileId, rtree.getRootPageId());
-		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
-				interiorFrame, metaFrame);
-		String string = stats.toString();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(string);
-		}
+        TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+                rtree.getRootPageId());
+        TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+        String string = stats.toString();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(string);
+        }
 
-		rtree.close();
-		bufferCache.closeFile(fileId);
-		bufferCache.close();
+        rtree.close();
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
 
-	}
+    }
 
-	// create an R-tree of two dimensions
-	// fill the R-tree with random integer key values using insertions
-	// perform ordered scan
-	@Test
-	public void test04() throws Exception {
+    // create an R-tree of two dimensions
+    // fill the R-tree with random integer key values using insertions
+    // perform ordered scan
+    @Test
+    public void test04() throws Exception {
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		// declare keys
-		int keyFieldCount = 4;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = IntegerBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
-		cmps[1] = cmps[0];
-		cmps[2] = cmps[0];
-		cmps[3] = cmps[0];
+        // declare keys
+        int keyFieldCount = 4;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
+        cmps[1] = cmps[0];
+        cmps[2] = cmps[0];
+        cmps[3] = cmps[0];
 
-		// declare tuple fields
-		int fieldCount = 7;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		typeTraits[0] = new TypeTrait(4);
-		typeTraits[1] = new TypeTrait(4);
-		typeTraits[2] = new TypeTrait(4);
-		typeTraits[3] = new TypeTrait(4);
-		typeTraits[4] = new TypeTrait(8);
-		typeTraits[5] = new TypeTrait(4);
-		typeTraits[6] = new TypeTrait(8);
+        // declare tuple fields
+        int fieldCount = 7;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[3] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[4] = DoublePointable.TYPE_TRAITS;
+        typeTraits[5] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[6] = DoublePointable.TYPE_TRAITS;
 
-		// create value providers
-		IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps); 
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                cmps.length, IntegerPointable.FACTORY);
 
-		MultiComparator cmp = new MultiComparator(cmps);
+        MultiComparator cmp = new MultiComparator(cmps);
 
-		RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-				typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
 
-		ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory
-				.createFrame();
-		IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, fileId, 0, metaFrameFactory);
+        IRTreeFrame interiorFrame = (IRTreeFrame) interiorFrameFactory.createFrame();
+        IRTreeFrame leafFrame = (IRTreeFrame) leafFrameFactory.createFrame();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 
-		RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
-				interiorFrameFactory, leafFrameFactory);
-		rtree.create(fileId);
-		rtree.open(fileId);
+        RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        rtree.create(fileId);
+        rtree.open(fileId);
 
-		ByteBuffer hyracksFrame = ctx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
-		DataOutput dos = tb.getDataOutput();
+        ByteBuffer hyracksFrame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        DataOutput dos = tb.getDataOutput();
 
-		@SuppressWarnings("rawtypes")
-		ISerializerDeserializer[] recDescSers = {
-				IntegerSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE };
-		RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(
-				ctx.getFrameSize(), recDesc);
-		accessor.reset(hyracksFrame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor.reset(hyracksFrame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
+        Random rnd = new Random();
+        rnd.setSeed(50);
 
-		Random rnd2 = new Random();
-		rnd2.setSeed(50);
-		for (int i = 0; i < 5000; i++) {
+        Random rnd2 = new Random();
+        rnd2.setSeed(50);
+        for (int i = 0; i < 5000; i++) {
 
-			int p1x = rnd.nextInt();
-			int p1y = rnd.nextInt();
-			int p2x = rnd.nextInt();
-			int p2y = rnd.nextInt();
+            int p1x = rnd.nextInt();
+            int p1y = rnd.nextInt();
+            int p2x = rnd.nextInt();
+            int p2y = rnd.nextInt();
 
-			double pk1 = rnd2.nextDouble();
-			int pk2 = rnd2.nextInt();
-			double pk3 = rnd2.nextDouble();
+            double pk1 = rnd2.nextDouble();
+            int pk2 = rnd2.nextInt();
+            double pk3 = rnd2.nextDouble();
 
-			tb.reset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(
-					Math.min(p1x, p2x), dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(
-					Math.min(p1y, p2y), dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(
-					Math.max(p1x, p2x), dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(
-					Math.max(p1y, p2y), dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk1, dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk2, dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(pk3, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
+            tuple.reset(accessor, 0);
 
-			if (LOGGER.isLoggable(Level.INFO)) {
-				if (i % 1000 == 0) {
-					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
-							+ " " + Math.max(p1y, p2y));
-				}
-			}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                            + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+                }
+            }
 
-			try {
-				indexAccessor.insert(tuple);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		String rtreeStats = rtree.printStats();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(rtreeStats);
-		}
+        String rtreeStats = rtree.printStats();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(rtreeStats);
+        }
 
-		// disk-order scan
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("DISK-ORDER SCAN:");
-		}
-		TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
-				leafFrame);
-		indexAccessor.diskOrderScan(diskOrderCursor);
-		try {
-			while (diskOrderCursor.hasNext()) {
-				diskOrderCursor.next();
-				ITupleReference frameTuple = diskOrderCursor.getTuple();
-				String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-				if (LOGGER.isLoggable(Level.INFO)) {
-					LOGGER.info(rec);
-				}
-			}
-		} catch (Exception e) {
-			e.printStackTrace();
-		} finally {
-			diskOrderCursor.close();
-		}
+        // disk-order scan
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DISK-ORDER SCAN:");
+        }
+        TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+        indexAccessor.diskOrderScan(diskOrderCursor);
+        try {
+            while (diskOrderCursor.hasNext()) {
+                diskOrderCursor.next();
+                ITupleReference frameTuple = diskOrderCursor.getTuple();
+                String rec = TupleUtils.printTuple(frameTuple, recDescSers);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            diskOrderCursor.close();
+        }
 
-		TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(
-				bufferCache, freePageManager, fileId, rtree.getRootPageId());
-		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
-				interiorFrame, metaFrame);
-		String string = stats.toString();
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info(string);
-		}
+        TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+                rtree.getRootPageId());
+        TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+        String string = stats.toString();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(string);
+        }
 
-		rtree.close();
-		bufferCache.closeFile(fileId);
-		bufferCache.close();
+        rtree.close();
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
 
-	}
+    }
 }
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index 60d0578..a232e37 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -31,16 +31,17 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
@@ -69,212 +70,183 @@
 import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class SearchCursorTest extends AbstractRTreeTest {
-	private static final int PAGE_SIZE = 256;
-	private static final int NUM_PAGES = 10;
-	private static final int MAX_OPEN_FILES = 10;
-	private static final int HYRACKS_FRAME_SIZE = 128;
-	private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    private static final int PAGE_SIZE = 256;
+    private static final int NUM_PAGES = 10;
+    private static final int MAX_OPEN_FILES = 10;
+    private static final int HYRACKS_FRAME_SIZE = 128;
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
-	// create an R-tree of two dimensions
-	// fill the R-tree with random values using insertions
-	// and then perform range search
-	@Test
-	public void searchCursorTest() throws Exception {
+    // create an R-tree of two dimensions
+    // fill the R-tree with random values using insertions
+    // and then perform range search
+    @Test
+    public void searchCursorTest() throws Exception {
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		// declare keys
-		int keyFieldCount = 4;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = DoubleBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
-		cmps[1] = cmps[0];
-		cmps[2] = cmps[0];
-		cmps[3] = cmps[0];
+        // declare keys
+        int keyFieldCount = 4;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY).createBinaryComparator();
+        cmps[1] = cmps[0];
+        cmps[2] = cmps[0];
+        cmps[3] = cmps[0];
 
-		// declare tuple fields
-		int fieldCount = 5;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		typeTraits[0] = new TypeTrait(8);
-		typeTraits[1] = new TypeTrait(8);
-		typeTraits[2] = new TypeTrait(8);
-		typeTraits[3] = new TypeTrait(8);
-		typeTraits[4] = new TypeTrait(4);
+        // declare tuple fields
+        int fieldCount = 5;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = DoublePointable.TYPE_TRAITS;
+        typeTraits[1] = DoublePointable.TYPE_TRAITS;
+        typeTraits[2] = DoublePointable.TYPE_TRAITS;
+        typeTraits[3] = DoublePointable.TYPE_TRAITS;
+        typeTraits[4] = IntegerPointable.TYPE_TRAITS;
 
-		// create value providers
-		IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.comparatorsToPrimitiveValueProviderFactories(cmps); 
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                cmps.length, DoublePointable.FACTORY);
 
-		MultiComparator cmp = new MultiComparator(cmps);
+        MultiComparator cmp = new MultiComparator(cmps);
 
-		RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-				typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
 
-		ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(
-				tupleWriterFactory, valueProviderFactories);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
+                valueProviderFactories);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) interiorFrameFactory
-				.createFrame();
-		IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory
-				.createFrame();
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, fileId, 0, metaFrameFactory);
+        IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) interiorFrameFactory.createFrame();
+        IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory.createFrame();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 
-		RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager,
-				interiorFrameFactory, leafFrameFactory);
-		rtree.create(fileId);
-		rtree.open(fileId);
+        RTree rtree = new RTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+        rtree.create(fileId);
+        rtree.open(fileId);
 
-		ByteBuffer hyracksFrame = ctx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
-		DataOutput dos = tb.getDataOutput();
+        ByteBuffer hyracksFrame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        DataOutput dos = tb.getDataOutput();
 
-		@SuppressWarnings("rawtypes")
-		ISerializerDeserializer[] recDescSers = {
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				DoubleSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE };
-		RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(
-				ctx.getFrameSize(), recDesc);
-		accessor.reset(hyracksFrame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] recDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+        RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor.reset(hyracksFrame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
-		for (int i = 0; i < 5000; i++) {
+        Random rnd = new Random();
+        rnd.setSeed(50);
+        for (int i = 0; i < 5000; i++) {
 
-			double p1x = rnd.nextDouble();
-			double p1y = rnd.nextDouble();
-			double p2x = rnd.nextDouble();
-			double p2y = rnd.nextDouble();
+            double p1x = rnd.nextDouble();
+            double p1y = rnd.nextDouble();
+            double p2x = rnd.nextDouble();
+            double p2y = rnd.nextDouble();
 
-			int pk = rnd.nextInt();
+            int pk = rnd.nextInt();
 
-			tb.reset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
+            tuple.reset(accessor, 0);
 
-			if (LOGGER.isLoggable(Level.INFO)) {
-				if (i % 1000 == 0) {
-					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
-							+ " " + Math.max(p1y, p2y));
-				}
-			}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                            + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+                }
+            }
 
-			try {
-				indexAccessor.insert(tuple);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
+            try {
+                indexAccessor.insert(tuple);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
 
-		for (int i = 0; i < 50; i++) {
-			double p1x = rnd.nextDouble();
-			double p1y = rnd.nextDouble();
-			double p2x = rnd.nextDouble();
-			double p2y = rnd.nextDouble();
+        for (int i = 0; i < 50; i++) {
+            double p1x = rnd.nextDouble();
+            double p1y = rnd.nextDouble();
+            double p2x = rnd.nextDouble();
+            double p2y = rnd.nextDouble();
 
-			int pk = rnd.nextInt();
+            int pk = rnd.nextInt();
 
-			tb.reset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x),
-					dos);
-			tb.addFieldEndOffset();
-			DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y),
-					dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
-			tb.addFieldEndOffset();
+            tb.reset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.min(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1x, p2x), dos);
+            tb.addFieldEndOffset();
+            DoubleSerializerDeserializer.INSTANCE.serialize(Math.max(p1y, p2y), dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(pk, dos);
+            tb.addFieldEndOffset();
 
-			appender.reset(hyracksFrame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
-					tb.getSize());
+            appender.reset(hyracksFrame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			tuple.reset(accessor, 0);
+            tuple.reset(accessor, 0);
 
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info(i + " Searching for: " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-						+ Math.max(p1y, p2y));
-			}
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info(i + " Searching for: " + Math.min(p1x, p2x) + " " + Math.min(p1y, p2y) + " "
+                        + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+            }
 
-			ITreeIndexCursor searchCursor = new RTreeSearchCursor(
-					interiorFrame, leafFrame);
-			SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
+            ITreeIndexCursor searchCursor = new RTreeSearchCursor(interiorFrame, leafFrame);
+            SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
 
-			indexAccessor.search(searchCursor, searchPredicate);
+            indexAccessor.search(searchCursor, searchPredicate);
 
-			ArrayList<Integer> results = new ArrayList<Integer>();
-			try {
-				while (searchCursor.hasNext()) {
-					searchCursor.next();
-					ITupleReference frameTuple = searchCursor.getTuple();
-					ByteArrayInputStream inStream = new ByteArrayInputStream(
-							frameTuple.getFieldData(4),
-							frameTuple.getFieldStart(4),
-							frameTuple.getFieldLength(4));
-					DataInput dataIn = new DataInputStream(inStream);
-					Integer res = IntegerSerializerDeserializer.INSTANCE
-							.deserialize(dataIn);
-					results.add(res);
-				}
-			} catch (Exception e) {
-				e.printStackTrace();
-			} finally {
-				searchCursor.close();
-			}
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("There are " + results.size()
-						+ " objects that satisfy the query");
-			}
-		}
+            ArrayList<Integer> results = new ArrayList<Integer>();
+            try {
+                while (searchCursor.hasNext()) {
+                    searchCursor.next();
+                    ITupleReference frameTuple = searchCursor.getTuple();
+                    ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(4),
+                            frameTuple.getFieldStart(4), frameTuple.getFieldLength(4));
+                    DataInput dataIn = new DataInputStream(inStream);
+                    Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
+                    results.add(res);
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            } finally {
+                searchCursor.close();
+            }
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("There are " + results.size() + " objects that satisfy the query");
+            }
+        }
 
-		rtree.close();
-		bufferCache.closeFile(fileId);
-		bufferCache.close();
-	}
+        rtree.close();
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
+    }
 }
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index d190af3..890f42d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,6 +79,7 @@
   </pluginRepositories>
 
   <modules>
+    <module>hyracks-ipc</module>
     <module>hyracks-api</module>
     <module>hyracks-dataflow-common</module>
     <module>hyracks-dataflow-std</module>
@@ -86,6 +87,7 @@
     <module>hyracks-control-common</module>
     <module>hyracks-control-cc</module>
     <module>hyracks-control-nc</module>
+    <module>hyracks-data</module>
     <module>hyracks-cli</module>
     <module>hyracks-storage-common</module>
     <module>hyracks-storage-am-common</module>
@@ -98,5 +100,6 @@
     <module>hyracks-examples</module>
     <module>hyracks-documentation</module>
     <module>hyracks-hadoop-compat</module>
+    <module>hyracks-algebricks</module>
   </modules>
 </project>