Fixed NC memory leak. Fixed duplicate job cleanup. Cleaned up data treatment in Hyracks

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_dev_next@865 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java
index 98034b6..6ad3079 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/data/ITypeTraitProvider.java
@@ -14,8 +14,8 @@
  */
 package edu.uci.ics.hyracks.algebricks.core.algebra.data;
 
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public interface ITypeTraitProvider {
-    public ITypeTrait getTypeTrait(Object type);
+    public ITypeTraits getTypeTrait(Object type);
 }
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java
index 95bf8a3..4a900f1 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/data/UTF8StringPrinterFactory.java
@@ -19,7 +19,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinter;
 import edu.uci.ics.hyracks.algebricks.core.algebra.data.IPrinterFactory;
 import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class UTF8StringPrinterFactory implements IPrinterFactory {
 
@@ -36,12 +36,12 @@
 
             @Override
             public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
-                int strlen = StringUtils.getUTFLen(b, s);
+                int strlen = UTF8StringPointable.getUTFLen(b, s);
                 int pos = s + 2;
                 int maxPos = pos + strlen;
                 ps.print("\"");
                 while (pos < maxPos) {
-                    char c = StringUtils.charAt(b, pos);
+                    char c = UTF8StringPointable.charAt(b, pos);
                     switch (c) {
                         case '\\':
                         case '"':
@@ -49,7 +49,7 @@
                             break;
                     }
                     ps.print(c);
-                    pos += StringUtils.charSize(b, pos);
+                    pos += UTF8StringPointable.charSize(b, pos);
                 }
                 ps.print("\"");
             }
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java
index 8f03316..6814d74 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/jobgen/impl/JobGenHelper.java
@@ -34,7 +34,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 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.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
@@ -45,11 +45,11 @@
 
     private static final Logger LOGGER = Logger.getLogger(JobGenHelper.class.getName());
 
-    public static ITreeIndexFrameFactory createBTreeNSMInteriorFrameFactory(ITypeTrait[] typeTraits) {
+    public static ITreeIndexFrameFactory createBTreeNSMInteriorFrameFactory(ITypeTraits[] typeTraits) {
         return new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
     }
 
-    public static ITreeIndexFrameFactory createBTreeNSMLeafFrameFactory(ITypeTrait[] typeTraits) {
+    public static ITreeIndexFrameFactory createBTreeNSMLeafFrameFactory(ITypeTraits[] typeTraits) {
         return new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
     }
 
@@ -143,9 +143,9 @@
         return null;
     }
 
-    public static ITypeTrait[] variablesToTypeTraits(Collection<LogicalVariable> varLogical,
+    public static ITypeTraits[] variablesToTypeTraits(Collection<LogicalVariable> varLogical,
             IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
-        ITypeTrait[] typeTraits = new ITypeTrait[varLogical.size()];
+        ITypeTraits[] typeTraits = new ITypeTraits[varLogical.size()];
         ITypeTraitProvider typeTraitProvider = context.getTypeTraitProvider();
         int i = 0;
         for (LogicalVariable v : varLogical) {
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java
index 9780ca6..8ecd16b 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/utils/WriteValueTools.java
@@ -3,7 +3,7 @@
 import java.io.IOException;
 import java.io.OutputStream;
 
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public final class WriteValueTools {
 
@@ -58,20 +58,20 @@
     }
 
     public static void writeUTF8String(byte[] b, int s, int l, OutputStream os) throws IOException {
-        int stringLength = StringUtils.getUTFLen(b, s);
+        int stringLength = UTF8StringPointable.getUTFLen(b, s);
         int position = s + 2;
         int maxPosition = position + stringLength;
         os.write('\"');
         while (position < maxPosition) {
-            char c = StringUtils.charAt(b, position);
+            char c = UTF8StringPointable.charAt(b, position);
             switch (c) {
-                // escape
+            // escape
                 case '\\':
                 case '"':
                     os.write('\\');
                     break;
             }
-            int sz = StringUtils.charSize(b, position);
+            int sz = UTF8StringPointable.charSize(b, position);
             while (sz > 0) {
                 os.write(b[position]);
                 position++;
@@ -82,19 +82,19 @@
     }
 
     public static void writeUTF8StringNoQuotes(byte[] b, int s, int l, OutputStream os) throws IOException {
-        int stringLength = StringUtils.getUTFLen(b, s);
+        int stringLength = UTF8StringPointable.getUTFLen(b, s);
         int position = s + 2;
         int maxPosition = position + stringLength;
         while (position < maxPosition) {
-            char c = StringUtils.charAt(b, position);
+            char c = UTF8StringPointable.charAt(b, position);
             switch (c) {
-                // escape
+            // escape
                 case '\\':
                 case '"':
                     os.write('\\');
                     break;
             }
-            int sz = StringUtils.charSize(b, position);
+            int sz = UTF8StringPointable.charSize(b, position);
             while (sz > 0) {
                 os.write(b[position]);
                 position++;
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/pom.xml b/hyracks-algebricks/hyracks-algebricks-tests/pom.xml
index 2767ae9..d326539 100644
--- a/hyracks-algebricks/hyracks-algebricks-tests/pom.xml
+++ b/hyracks-algebricks/hyracks-algebricks-tests/pom.xml
@@ -113,5 +113,10 @@
   	<artifactId>hyracks-control-nc</artifactId>
   	<version>0.2.0-SNAPSHOT</version>
   </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-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index 6bd055d..c550a3d 100644
--- a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -56,9 +56,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.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.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;
@@ -404,7 +405,8 @@
         // the sort (by nation id)
         RecordDescriptor sortDesc = scannerDesc;
         InMemorySortOperatorDescriptor sort = new InMemorySortOperatorDescriptor(spec, new int[] { 3 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, sortDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                sortDesc);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sort,
                 new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
@@ -423,7 +425,8 @@
         RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor gby = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 3 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, npaaf, gbyDesc);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                npaaf, gbyDesc);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, gby,
                 new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
@@ -487,12 +490,13 @@
         RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(new int[] { 3 },
-                new IBinaryHashFunctionFactory[] { IntegerBinaryHashFunctionFactory.INSTANCE });
+                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) });
         IAggregateFunctionFactory[] aggFuns = new IAggregateFunctionFactory[] { new TupleCountAggregateFunctionFactory() };
         IAccumulatingAggregatorFactory aggFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aggFuns,
                 new int[] { 3 }, new int[] {});
         HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(spec, new int[] { 3 }, tpcf,
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, aggFactory, gbyDesc, 1024);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                aggFactory, gbyDesc, 1024);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, gby,
                 new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
@@ -698,7 +702,8 @@
 
         // the algebricks op.
         InMemorySortRuntimeFactory sort = new InMemorySortRuntimeFactory(new int[] { 1 }, null,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, null);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                null);
         RecordDescriptor sortDesc = scannerDesc;
 
         String fileName = "scanMicroSortWrite.out";
@@ -810,7 +815,7 @@
         // the sort (by nation id)
         RecordDescriptor sortDesc = scannerDesc;
         InMemorySortRuntimeFactory sort = new InMemorySortRuntimeFactory(new int[] { 3 }, null,
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, null);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, null);
 
         // the group-by
         NestedTupleSourceRuntimeFactory nts = new NestedTupleSourceRuntimeFactory();
@@ -826,8 +831,8 @@
         RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         MicroPreClusteredGroupRuntimeFactory gby = new MicroPreClusteredGroupRuntimeFactory(new int[] { 3 },
-                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, npaaf, sortDesc, gbyDesc,
-                null);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                npaaf, sortDesc, gbyDesc, null);
 
         // the algebricks op.
         IEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITypeTrait.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITypeTrait.java
deleted file mode 100644
index 639617d..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITypeTrait.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package edu.uci.ics.hyracks.api.dataflow.value;
-
-import java.io.Serializable;
-
-public interface ITypeTrait extends Serializable {
-	public static final int VARIABLE_LENGTH = -1;
-	
-	public static final ITypeTrait INTEGER_TYPE_TRAIT = new TypeTrait(4);
-    public static final ITypeTrait INTEGER64_TYPE_TRAIT = new TypeTrait(8);
-    public static final ITypeTrait FLOAT_TYPE_TRAIT = new TypeTrait(4);
-    public static final ITypeTrait DOUBLE_TYPE_TRAIT = new TypeTrait(8);
-    public static final ITypeTrait BOOLEAN_TYPE_TRAIT = new TypeTrait(1);
-    public static final ITypeTrait VARLEN_TYPE_TRAIT = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);	
-	int getStaticallyKnownDataLength();
-}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITypeTraits.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITypeTraits.java
new file mode 100644
index 0000000..4a6f826
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITypeTraits.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.Serializable;
+
+public interface ITypeTraits extends Serializable {
+    public boolean isFixedLength();
+
+    public int getFixedLength();
+}
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..3d03c32 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,7 +21,7 @@
     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) {
@@ -30,7 +30,7 @@
     }
     
     // temporarily adding constructor to include type traits
-    public RecordDescriptor(ISerializerDeserializer[] fields, ITypeTrait[] typeTraits) {
+    public RecordDescriptor(ISerializerDeserializer[] fields, ITypeTraits[] typeTraits) {
     	this.fields = fields;
         this.typeTraits = typeTraits;
     }
@@ -39,7 +39,7 @@
         return fields;
     }
     
-    public ITypeTrait[] getTypeTraits() {
+    public ITypeTraits[] getTypeTraits() {
     	return typeTraits;
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/TypeTrait.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/TypeTrait.java
deleted file mode 100644
index 6be9552..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/TypeTrait.java
+++ /dev/null
@@ -1,16 +0,0 @@
-package edu.uci.ics.hyracks.api.dataflow.value;
-
-public class TypeTrait implements ITypeTrait {
-	
-	private static final long serialVersionUID = 1L;
-	private int length;
-	
-	public TypeTrait(int length) {
-		this.length = length;
-	}
-	
-	@Override
-	public int getStaticallyKnownDataLength() {
-		return length;
-	}	
-}
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-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..dce7c13 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,7 +46,7 @@
         }
         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);
         }
diff --git a/hyracks-data/hyracks-data-std/pom.xml b/hyracks-data/hyracks-data-std/pom.xml
new file mode 100644
index 0000000..2bc5d40
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/pom.xml
@@ -0,0 +1,31 @@
+<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>
+  <artifactId>hyracks-data-std</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-data</artifactId>
+    <version>0.2.0-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  <dependency>
+  	<groupId>edu.uci.ics.hyracks</groupId>
+  	<artifactId>hyracks-api</artifactId>
+  	<version>0.2.0-SNAPSHOT</version>
+  </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
new file mode 100644
index 0000000..bab393e
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.accessors;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public class PointableBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final IPointableFactory pf;
+
+    public static PointableBinaryComparatorFactory of(IPointableFactory pf) {
+        return new PointableBinaryComparatorFactory(pf);
+    }
+
+    public PointableBinaryComparatorFactory(IPointableFactory pf) {
+        this.pf = pf;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        final IPointable p = pf.createPointable();
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                p.set(b1, s1, l1);
+                return ((IComparable) p).compareTo(b2, s2, l2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/PointableBinaryHashFunctionFactory.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/PointableBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..fddbe3d
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/PointableBinaryHashFunctionFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.accessors;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public class PointableBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final IPointableFactory pf;
+
+    public static PointableBinaryHashFunctionFactory of(IPointableFactory pf) {
+        return new PointableBinaryHashFunctionFactory(pf);
+    }
+
+    public PointableBinaryHashFunctionFactory(IPointableFactory pf) {
+        this.pf = pf;
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        final IPointable p = pf.createPointable();
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                p.set(bytes, offset, length);
+                return ((IHashable) p).hash();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/AbstractPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/AbstractPointable.java
new file mode 100644
index 0000000..2d0d87c
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/AbstractPointable.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.api;
+
+public abstract class AbstractPointable implements IPointable {
+    protected byte[] bytes;
+
+    protected int start;
+
+    protected int length;
+
+    @Override
+    public void set(byte[] bytes, int start, int length) {
+        this.bytes = bytes;
+        this.start = start;
+        this.length = length;
+    }
+
+    @Override
+    public void set(IPointable pointer) {
+        set(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public byte[] getByteArray() {
+        return bytes;
+    }
+
+    @Override
+    public int getStartOffset() {
+        return start;
+    }
+
+    @Override
+    public int getLength() {
+        return length;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IComparable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IComparable.java
new file mode 100644
index 0000000..d68f356
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IComparable.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.api;
+
+public interface IComparable {
+    public int compareTo(IPointable pointer);
+
+    public int compareTo(byte[] bytes, int start, int length);
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IHashable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IHashable.java
new file mode 100644
index 0000000..3385298
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IHashable.java
@@ -0,0 +1,19 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.api;
+
+public interface IHashable {
+    public int hash();
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/INumeric.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/INumeric.java
new file mode 100644
index 0000000..72fc221
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/INumeric.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.api;
+
+public interface INumeric {
+    public byte byteValue();
+
+    public short shortValue();
+
+    public int intValue();
+
+    public long longValue();
+
+    public float floatValue();
+
+    public double doubleValue();
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IPointable.java
new file mode 100644
index 0000000..17e9b03
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IPointable.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.api;
+
+public interface IPointable {
+    public void set(byte[] bytes, int start, int length);
+
+    public void set(IPointable pointer);
+
+    public byte[] getByteArray();
+
+    public int getStartOffset();
+
+    public int getLength();
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IPointableFactory.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IPointableFactory.java
new file mode 100644
index 0000000..41895a1
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/api/IPointableFactory.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+
+public interface IPointableFactory extends Serializable {
+    public IPointable createPointable();
+    
+    public ITypeTraits getTypeTraits();
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/BooleanPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/BooleanPointable.java
new file mode 100644
index 0000000..7a10a01
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/BooleanPointable.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class BooleanPointable extends AbstractPointable implements IHashable, IComparable {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 1;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new BooleanPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static boolean getBoolean(byte[] bytes, int start) {
+        return bytes[start] == 0 ? false : true;
+    }
+
+    public static void setBoolean(byte[] bytes, int start, boolean value) {
+        bytes[start] = (byte) (value ? 1 : 0);
+    }
+
+    public boolean getBoolean() {
+        return getBoolean(bytes, start);
+    }
+
+    public void setBoolean(boolean value) {
+        setBoolean(bytes, start, value);
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        boolean b = getBoolean();
+        boolean ob = getBoolean(bytes, start);
+        return b == ob ? 0 : (b ? 1 : -1);
+    }
+
+    @Override
+    public int hash() {
+        return getBoolean() ? Boolean.TRUE.hashCode() : Boolean.FALSE.hashCode();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/BytePointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/BytePointable.java
new file mode 100644
index 0000000..a157d7e
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/BytePointable.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class BytePointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 1;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new BytePointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static byte getByte(byte[] bytes, int start) {
+        return bytes[start];
+    }
+
+    public static void setByte(byte[] bytes, int start, byte value) {
+        bytes[start] = value;
+    }
+
+    public byte getByte() {
+        return getByte(bytes, start);
+    }
+
+    public void setByte(byte value) {
+        setByte(bytes, start, value);
+    }
+
+    public byte preIncrement() {
+        byte v = getByte();
+        ++v;
+        setByte(v);
+        return v;
+    }
+
+    public byte postIncrement() {
+        byte v = getByte();
+        byte ov = v++;
+        setByte(v);
+        return ov;
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        byte b = getByte();
+        byte ob = getByte(bytes, start);
+        return b < ob ? -1 : (b > ob ? 1 : 0);
+    }
+
+    @Override
+    public int hash() {
+        return getByte();
+    }
+
+    @Override
+    public byte byteValue() {
+        return getByte();
+    }
+
+    @Override
+    public short shortValue() {
+        return getByte();
+    }
+
+    @Override
+    public int intValue() {
+        return getByte();
+    }
+
+    @Override
+    public long longValue() {
+        return getByte();
+    }
+
+    @Override
+    public float floatValue() {
+        return getByte();
+    }
+
+    @Override
+    public double doubleValue() {
+        return getByte();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java
new file mode 100644
index 0000000..5267086
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java
@@ -0,0 +1,137 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class DoublePointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 8;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new DoublePointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static long getLongBits(byte[] bytes, int start) {
+        return LongPointable.getLong(bytes, start);
+    }
+
+    public static double getDouble(byte[] bytes, int start) {
+        long bits = getLongBits(bytes, start);
+        return Double.longBitsToDouble(bits);
+    }
+
+    public static void setDouble(byte[] bytes, int start, double value) {
+        long bits = Double.doubleToLongBits(value);
+        LongPointable.setLong(bytes, start, bits);
+    }
+
+    public double getDouble() {
+        return getDouble(bytes, start);
+    }
+
+    public void setDouble(double value) {
+        setDouble(bytes, start, value);
+    }
+
+    public double preIncrement() {
+        double v = getDouble();
+        ++v;
+        setDouble(v);
+        return v;
+    }
+
+    public double postIncrement() {
+        double v = getDouble();
+        double ov = v++;
+        setDouble(v);
+        return ov;
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        double v = getDouble();
+        double ov = getDouble(bytes, start);
+        return v < ov ? -1 : (v > ov ? 1 : 0);
+    }
+
+    @Override
+    public int hash() {
+        long bits = getLongBits(bytes, start);
+        return (int) (bits ^ (bits >>> 32));
+    }
+
+    @Override
+    public byte byteValue() {
+        return (byte) getDouble();
+    }
+
+    @Override
+    public short shortValue() {
+        return (short) getDouble();
+    }
+
+    @Override
+    public int intValue() {
+        return (int) getDouble();
+    }
+
+    @Override
+    public long longValue() {
+        return (long) getDouble();
+    }
+
+    @Override
+    public float floatValue() {
+        return (float) getDouble();
+    }
+
+    @Override
+    public double doubleValue() {
+        return getDouble();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/FloatPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/FloatPointable.java
new file mode 100644
index 0000000..c751f8b
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/FloatPointable.java
@@ -0,0 +1,136 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class FloatPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 4;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new FloatPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    private static int getIntBits(byte[] bytes, int start) {
+        return IntegerPointable.getInteger(bytes, start);
+    }
+
+    public static float getFloat(byte[] bytes, int start) {
+        int bits = getIntBits(bytes, start);
+        return Float.intBitsToFloat(bits);
+    }
+
+    public static void setFloat(byte[] bytes, int start, float value) {
+        int bits = Float.floatToIntBits(value);
+        IntegerPointable.setInteger(bytes, start, bits);
+    }
+
+    public float getFloat() {
+        return getFloat(bytes, start);
+    }
+
+    public void setFloat(float value) {
+        setFloat(bytes, start, value);
+    }
+
+    public float preIncrement() {
+        float v = getFloat();
+        ++v;
+        setFloat(v);
+        return v;
+    }
+
+    public float postIncrement() {
+        float v = getFloat();
+        float ov = v++;
+        setFloat(v);
+        return ov;
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        float v = getFloat();
+        float ov = getFloat(bytes, start);
+        return v < ov ? -1 : (v > ov ? 1 : 0);
+    }
+
+    @Override
+    public int hash() {
+        return getIntBits(bytes, start);
+    }
+
+    @Override
+    public byte byteValue() {
+        return (byte) getFloat();
+    }
+
+    @Override
+    public short shortValue() {
+        return (short) getFloat();
+    }
+
+    @Override
+    public int intValue() {
+        return (int) getFloat();
+    }
+
+    @Override
+    public long longValue() {
+        return (long) getFloat();
+    }
+
+    @Override
+    public float floatValue() {
+        return getFloat();
+    }
+
+    @Override
+    public double doubleValue() {
+        return getFloat();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/IntegerPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/IntegerPointable.java
new file mode 100644
index 0000000..f18ae09
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/IntegerPointable.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class IntegerPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 4;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new IntegerPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static int getInteger(byte[] bytes, int start) {
+        return ((bytes[start] & 0xff) << 24) + ((bytes[start + 1] & 0xff) << 16) + ((bytes[start + 2] & 0xff) << 8)
+                + ((bytes[start + 3] & 0xff) << 0);
+    }
+
+    public static void setInteger(byte[] bytes, int start, int value) {
+        bytes[start] = (byte) ((value >>> 24) & 0xFF);
+        bytes[start + 1] = (byte) ((value >>> 16) & 0xFF);
+        bytes[start + 2] = (byte) ((value >>> 8) & 0xFF);
+        bytes[start + 3] = (byte) ((value >>> 0) & 0xFF);
+    }
+
+    public int getInteger() {
+        return getInteger(bytes, start);
+    }
+
+    public void setInteger(int value) {
+        setInteger(bytes, start, value);
+    }
+
+    public int preIncrement() {
+        int v = getInteger();
+        ++v;
+        setInteger(v);
+        return v;
+    }
+
+    public int postIncrement() {
+        int v = getInteger();
+        int ov = v++;
+        setInteger(v);
+        return ov;
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        int v = getInteger();
+        int ov = getInteger(bytes, start);
+        return v < ov ? -1 : (v > ov ? 1 : 0);
+    }
+
+    @Override
+    public int hash() {
+        return getInteger();
+    }
+
+    @Override
+    public byte byteValue() {
+        return (byte) getInteger();
+    }
+
+    @Override
+    public short shortValue() {
+        return (short) getInteger();
+    }
+
+    @Override
+    public int intValue() {
+        return getInteger();
+    }
+
+    @Override
+    public long longValue() {
+        return getInteger();
+    }
+
+    @Override
+    public float floatValue() {
+        return getInteger();
+    }
+
+    @Override
+    public double doubleValue() {
+        return getInteger();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/LongPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/LongPointable.java
new file mode 100644
index 0000000..89f9868
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/LongPointable.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class LongPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 8;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new LongPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static long getLong(byte[] bytes, int start) {
+        return (((long) (bytes[start] & 0xff)) << 56) + (((long) (bytes[start + 1] & 0xff)) << 48)
+                + (((long) (bytes[start + 2] & 0xff)) << 40) + (((long) (bytes[start + 3] & 0xff)) << 32)
+                + (((long) (bytes[start + 4] & 0xff)) << 24) + (((long) (bytes[start + 5] & 0xff)) << 16)
+                + (((long) (bytes[start + 6] & 0xff)) << 8) + (((long) (bytes[start + 7] & 0xff)) << 0);
+    }
+
+    public static void setLong(byte[] bytes, int start, long value) {
+        bytes[start] = (byte) ((value >>> 56) & 0xFF);
+        bytes[start + 1] = (byte) ((value >>> 48) & 0xFF);
+        bytes[start + 2] = (byte) ((value >>> 40) & 0xFF);
+        bytes[start + 3] = (byte) ((value >>> 32) & 0xFF);
+        bytes[start + 4] = (byte) ((value >>> 24) & 0xFF);
+        bytes[start + 5] = (byte) ((value >>> 16) & 0xFF);
+        bytes[start + 6] = (byte) ((value >>> 8) & 0xFF);
+        bytes[start + 7] = (byte) ((value >>> 0) & 0xFF);
+    }
+
+    public long getLong() {
+        return getLong(bytes, start);
+    }
+
+    public void setLong(long value) {
+        setLong(bytes, start, value);
+    }
+
+    public long preIncrement() {
+        long v = getLong();
+        ++v;
+        setLong(v);
+        return v;
+    }
+
+    public long postIncrement() {
+        long v = getLong();
+        long ov = v++;
+        setLong(v);
+        return ov;
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        long v = getLong();
+        long ov = getLong(bytes, start);
+        return v < ov ? -1 : (v > ov ? 1 : 0);
+    }
+
+    @Override
+    public int hash() {
+        long v = getLong();
+        return (int) (v ^ (v >>> 32));
+    }
+
+    @Override
+    public byte byteValue() {
+        return (byte) getLong();
+    }
+
+    @Override
+    public short shortValue() {
+        return (short) getLong();
+    }
+
+    @Override
+    public int intValue() {
+        return (int) getLong();
+    }
+
+    @Override
+    public long longValue() {
+        return getLong();
+    }
+
+    @Override
+    public float floatValue() {
+        return getLong();
+    }
+
+    @Override
+    public double doubleValue() {
+        return getLong();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/ShortPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/ShortPointable.java
new file mode 100644
index 0000000..57b9a1b
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/ShortPointable.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class ShortPointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 2;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new ShortPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static short getShort(byte[] bytes, int start) {
+        return (short) (((bytes[start] & 0xff) << 8) + (bytes[start + 1] & 0xff));
+    }
+
+    public static void setShort(byte[] bytes, int start, short value) {
+        bytes[start] = (byte) ((value >>> 24) & 0xFF);
+        bytes[start + 1] = (byte) ((value >>> 16) & 0xFF);
+    }
+
+    public short getShort() {
+        return getShort(bytes, start);
+    }
+
+    public void setShort(short value) {
+        setShort(bytes, start, value);
+    }
+
+    public short preIncrement() {
+        short v = getShort();
+        ++v;
+        setShort(v);
+        return v;
+    }
+
+    public short postIncrement() {
+        short v = getShort();
+        short ov = v++;
+        setShort(v);
+        return ov;
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        short v = getShort();
+        short ov = getShort(bytes, start);
+        return v < ov ? -1 : (v > ov ? 1 : 0);
+    }
+
+    @Override
+    public int hash() {
+        return getShort();
+    }
+
+    @Override
+    public byte byteValue() {
+        return (byte) getShort();
+    }
+
+    @Override
+    public short shortValue() {
+        return getShort();
+    }
+
+    @Override
+    public int intValue() {
+        return getShort();
+    }
+
+    @Override
+    public long longValue() {
+        return getShort();
+    }
+
+    @Override
+    public float floatValue() {
+        return getShort();
+    }
+
+    @Override
+    public double doubleValue() {
+        return getShort();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
new file mode 100644
index 0000000..21caf77
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -0,0 +1,169 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class UTF8StringPointable extends AbstractPointable implements IHashable, IComparable {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return false;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 0;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new UTF8StringPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    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);
+    }
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        int utflen1 = getUTFLen(this.bytes, this.start);
+        int utflen2 = getUTFLen(bytes, start);
+
+        int c1 = 0;
+        int c2 = 0;
+
+        int s1Start = this.start + 2;
+        int s2Start = start + 2;
+
+        while (c1 < utflen1 && c2 < utflen2) {
+            char ch1 = charAt(this.bytes, s1Start + c1);
+            char ch2 = charAt(bytes, s2Start + c2);
+
+            if (ch1 != ch2) {
+                return ch1 - ch2;
+            }
+            c1 += charSize(this.bytes, s1Start + c1);
+            c2 += charSize(bytes, s2Start + c2);
+        }
+        return utflen1 - utflen2;
+    }
+
+    @Override
+    public int hash() {
+        int h = 0;
+        int utflen = getUTFLen(bytes, start);
+        int sStart = start + 2;
+        int c = 0;
+
+        while (c < utflen) {
+            char ch = charAt(bytes, sStart + c);
+            h = 31 * h + ch;
+            c += charSize(bytes, sStart + c);
+        }
+        return h;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/VoidPointable.java b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/VoidPointable.java
new file mode 100644
index 0000000..4faf606
--- /dev/null
+++ b/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/VoidPointable.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+public final class VoidPointable extends AbstractPointable {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return false;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 0;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new VoidPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+}
\ No newline at end of file
diff --git a/hyracks-data/pom.xml b/hyracks-data/pom.xml
new file mode 100644
index 0000000..ed777e9
--- /dev/null
+++ b/hyracks-data/pom.xml
@@ -0,0 +1,15 @@
+<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>
+  <artifactId>hyracks-data</artifactId>
+  <packaging>pom</packaging>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.0-SNAPSHOT</version>
+  </parent>
+
+  <modules>
+    <module>hyracks-data-std</module>
+  </modules>
+</project>
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/data/comparators/DoubleBinaryComparator.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/DoubleBinaryComparator.java
deleted file mode 100644
index dde4d7b..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/DoubleBinaryComparator.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
-
-public class DoubleBinaryComparator implements IBinaryComparator {
-    @Override
-    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {   
-        return Double.compare(DoubleSerializerDeserializer.getDouble(b1, s1), DoubleSerializerDeserializer
-                .getDouble(b2, s2));
-    }
-}
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/DoubleBinaryComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/DoubleBinaryComparatorFactory.java
deleted file mode 100644
index 0782ab5..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/DoubleBinaryComparatorFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-
-public class DoubleBinaryComparatorFactory implements IBinaryComparatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final DoubleBinaryComparatorFactory INSTANCE = new DoubleBinaryComparatorFactory();
-
-    private DoubleBinaryComparatorFactory() {
-    }
-
-    @Override
-    public IBinaryComparator createBinaryComparator() {
-        return new DoubleBinaryComparator();
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatBinaryComparator.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatBinaryComparator.java
deleted file mode 100644
index d168e29..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatBinaryComparator.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
-
-public class FloatBinaryComparator implements IBinaryComparator {
-    @Override
-    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-        return Float.compare(FloatSerializerDeserializer.getFloat(b1, s1), FloatSerializerDeserializer
-                .getFloat(b2, s2));
-    }
-}
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatBinaryComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatBinaryComparatorFactory.java
deleted file mode 100644
index 97224b2..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatBinaryComparatorFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-
-public class FloatBinaryComparatorFactory implements IBinaryComparatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final FloatBinaryComparatorFactory INSTANCE = new FloatBinaryComparatorFactory();
-
-    private FloatBinaryComparatorFactory() {
-    }
-
-    @Override
-    public IBinaryComparator createBinaryComparator() {
-        return new FloatBinaryComparator();
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatComparatorFactory.java
deleted file mode 100644
index 59f36bf..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/FloatComparatorFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
-
-public class FloatComparatorFactory implements IComparatorFactory<Float> {
-    private static final long serialVersionUID = 1L;
-
-    public static final FloatComparatorFactory INSTANCE = new FloatComparatorFactory();
-
-    private FloatComparatorFactory() {
-    }
-
-    @Override
-    public IComparator<Float> createComparator() {
-        return new IComparator<Float>() {
-            @Override
-            public int compare(Float o1, Float o2) {
-                return o1.compareTo(o2);
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerBinaryComparator.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerBinaryComparator.java
deleted file mode 100644
index b6c4358..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerBinaryComparator.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-public class IntegerBinaryComparator implements IBinaryComparator {
-    @Override
-    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-        int v1 = IntegerSerializerDeserializer.getInt(b1, s1);
-        int v2 = IntegerSerializerDeserializer.getInt(b2, s2);
-        return v1 < v2 ? -1 : (v1 > v2 ? 1 : 0);
-    }
-}
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerBinaryComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerBinaryComparatorFactory.java
deleted file mode 100644
index d67c825..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerBinaryComparatorFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-
-public class IntegerBinaryComparatorFactory implements IBinaryComparatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final IntegerBinaryComparatorFactory INSTANCE = new IntegerBinaryComparatorFactory();
-
-    private IntegerBinaryComparatorFactory() {
-    }
-
-    @Override
-    public IBinaryComparator createBinaryComparator() {
-        return new IntegerBinaryComparator();
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerComparatorFactory.java
deleted file mode 100644
index 9044e47..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/IntegerComparatorFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
-
-public class IntegerComparatorFactory implements IComparatorFactory<Integer> {
-    private static final long serialVersionUID = 1L;
-
-    public static final IntegerComparatorFactory INSTANCE = new IntegerComparatorFactory();
-
-    private IntegerComparatorFactory() {
-    }
-
-    @Override
-    public IComparator<Integer> createComparator() {
-        return new IComparator<Integer>() {
-            @Override
-            public int compare(Integer o1, Integer o2) {
-                return o1.compareTo(o2);
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/StringComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/StringComparatorFactory.java
deleted file mode 100644
index d79def5..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/StringComparatorFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
-
-public class StringComparatorFactory implements IComparatorFactory<String> {
-    private static final long serialVersionUID = 1L;
-
-    public static final StringComparatorFactory INSTANCE = new StringComparatorFactory();
-
-    private StringComparatorFactory() {
-    }
-
-    @Override
-    public IComparator<String> createComparator() {
-        return new IComparator<String>() {
-            @Override
-            public int compare(String o1, String o2) {
-                return o1.compareTo(o2);
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/UTF8StringBinaryComparator.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/UTF8StringBinaryComparator.java
deleted file mode 100644
index 31f8a86..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/UTF8StringBinaryComparator.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
-
-public class UTF8StringBinaryComparator implements IBinaryComparator {
-    @Override
-    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-        int utflen1 = StringUtils.getUTFLen(b1, s1);
-        int utflen2 = StringUtils.getUTFLen(b2, s2);
-
-        int c1 = 0;
-        int c2 = 0;
-
-        int s1Start = s1 + 2;
-        int s2Start = s2 + 2;
-
-        while (c1 < utflen1 && c2 < utflen2) {
-            char ch1 = StringUtils.charAt(b1, s1Start + c1);
-            char ch2 = StringUtils.charAt(b2, s2Start + c2);
-
-            if (ch1 != ch2) {
-                return ch1 - ch2;
-            }
-            c1 += StringUtils.charSize(b1, s1Start + c1);
-            c2 += StringUtils.charSize(b2, s2Start + c2);
-        }
-        return utflen1 - utflen2;
-    }
-}
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/UTF8StringBinaryComparatorFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/UTF8StringBinaryComparatorFactory.java
deleted file mode 100644
index 2d90dcd..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/comparators/UTF8StringBinaryComparatorFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-
-public class UTF8StringBinaryComparatorFactory implements IBinaryComparatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final UTF8StringBinaryComparatorFactory INSTANCE = new UTF8StringBinaryComparatorFactory();
-
-    private UTF8StringBinaryComparatorFactory() {
-    }
-
-    @Override
-    public IBinaryComparator createBinaryComparator() {
-        return new UTF8StringBinaryComparator();
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/FloatBinaryHashFunctionFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/FloatBinaryHashFunctionFactory.java
deleted file mode 100644
index 72294e8..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/FloatBinaryHashFunctionFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
-
-public class FloatBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final FloatBinaryHashFunctionFactory INSTANCE = new FloatBinaryHashFunctionFactory();
-
-    private FloatBinaryHashFunctionFactory() {
-    }
-
-    @Override
-    public IBinaryHashFunction createBinaryHashFunction() {
-        return new IBinaryHashFunction() {
-            @Override
-            public int hash(byte[] bytes, int offset, int length) {
-                return FloatSerializerDeserializer.getIntBits(bytes, offset);
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/FloatHashFunctionFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/FloatHashFunctionFactory.java
deleted file mode 100644
index 481c111..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/FloatHashFunctionFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class FloatHashFunctionFactory implements IHashFunctionFactory<Float> {
-    private static final long serialVersionUID = 1L;
-
-    public static final FloatHashFunctionFactory INSTANCE = new FloatHashFunctionFactory();
-
-    private FloatHashFunctionFactory() {
-    }
-
-    @Override
-    public IHashFunction<Float> createHashFunction() {
-        return new IHashFunction<Float>() {
-            @Override
-            public int hash(Float o) throws HyracksDataException {
-                return o.hashCode();
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/IntegerBinaryHashFunctionFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/IntegerBinaryHashFunctionFactory.java
deleted file mode 100644
index 51cd494..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/IntegerBinaryHashFunctionFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-public class IntegerBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final IntegerBinaryHashFunctionFactory INSTANCE = new IntegerBinaryHashFunctionFactory();
-
-    private IntegerBinaryHashFunctionFactory() {
-    }
-
-    @Override
-    public IBinaryHashFunction createBinaryHashFunction() {
-        return new IBinaryHashFunction() {
-            @Override
-            public int hash(byte[] bytes, int offset, int length) {
-                return IntegerSerializerDeserializer.getInt(bytes, offset);
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/IntegerHashFunctionFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/IntegerHashFunctionFactory.java
deleted file mode 100644
index c3880bf..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/IntegerHashFunctionFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class IntegerHashFunctionFactory implements IHashFunctionFactory<Integer> {
-    private static final long serialVersionUID = 1L;
-
-    public static final IntegerHashFunctionFactory INSTANCE = new IntegerHashFunctionFactory();
-
-    private IntegerHashFunctionFactory() {
-    }
-
-    @Override
-    public IHashFunction<Integer> createHashFunction() {
-        return new IHashFunction<Integer>() {
-            @Override
-            public int hash(Integer o) throws HyracksDataException {
-                return o.hashCode();
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/StringHashFunctionFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/StringHashFunctionFactory.java
deleted file mode 100644
index 3a6d9a9..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/StringHashFunctionFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class StringHashFunctionFactory implements IHashFunctionFactory<String> {
-    private static final long serialVersionUID = 1L;
-
-    public static final StringHashFunctionFactory INSTANCE = new StringHashFunctionFactory();
-
-    private StringHashFunctionFactory() {
-    }
-
-    @Override
-    public IHashFunction<String> createHashFunction() {
-        return new IHashFunction<String>() {
-            @Override
-            public int hash(String o) throws HyracksDataException {
-                return o.hashCode();
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/UTF8StringBinaryHashFunctionFactory.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/UTF8StringBinaryHashFunctionFactory.java
deleted file mode 100644
index 98634b6..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/hash/UTF8StringBinaryHashFunctionFactory.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.data.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
-
-public class UTF8StringBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
-    public static final UTF8StringBinaryHashFunctionFactory INSTANCE = new UTF8StringBinaryHashFunctionFactory();
-
-    private static final long serialVersionUID = 1L;
-
-    private UTF8StringBinaryHashFunctionFactory() {
-    }
-
-    @Override
-    public IBinaryHashFunction createBinaryHashFunction() {
-        return new IBinaryHashFunction() {
-            @Override
-            public int hash(byte[] bytes, int offset, int length) {
-                int h = 0;
-                int utflen = StringUtils.getUTFLen(bytes, offset);
-                int sStart = offset + 2;
-                int c = 0;
-
-                while (c < utflen) {
-                    char ch = StringUtils.charAt(bytes, sStart + c);
-                    h = 31 * h + ch;
-                    c += StringUtils.charSize(bytes, sStart + c);
-                }
-                return h;
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-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/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-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
index af5d16b..32f94fe 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.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;
@@ -71,13 +71,13 @@
         if (keyFields.length >= outRecordDescriptor.getFields().length) {
             // for the case of zero-aggregations
             ISerializerDeserializer<?>[] fields = outRecordDescriptor.getFields();
-            ITypeTrait[] types = outRecordDescriptor.getTypeTraits();
+            ITypeTraits[] 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;
+            ITypeTraits[] newTypes = null;
             if (types != null) {
-                newTypes = new ITypeTrait[types.length + 1];
+                newTypes = new ITypeTraits[types.length + 1];
                 for (int i = 0; i < types.length; i++)
                     newTypes[i] = types[i];
             }
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..8255c58 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
@@ -25,14 +25,14 @@
 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;
@@ -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..ae988cd 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
@@ -24,13 +24,14 @@
 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;
@@ -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..dbc0bca 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
@@ -23,14 +23,15 @@
 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;
@@ -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..bf2c2e1 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
@@ -21,13 +21,13 @@
 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;
@@ -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..29f8257 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
@@ -23,15 +23,15 @@
 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;
@@ -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/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/CountOfCountsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index c0a1d10..c43238d 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,20 +79,23 @@
         PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
                 spec,
                 new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 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 MultiAggregatorFactory(
-                        new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), desc3);
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -100,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, csvScanner, 0, sorter, 0);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
@@ -108,7 +114,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 +145,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,20 +154,23 @@
         PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
                 spec,
                 new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 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 MultiAggregatorFactory(
-                        new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), desc3);
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -169,7 +180,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 +189,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 +220,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,20 +229,23 @@
         PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
                 spec,
                 new int[] { 0 },
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 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 MultiAggregatorFactory(
-                        new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), desc3);
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(
+                spec,
+                new int[] { 1 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -238,7 +255,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 +264,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..5571a4f 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,
@@ -364,10 +383,13 @@
             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,
@@ -443,11 +465,17 @@
             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,
@@ -523,11 +551,17 @@
             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/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
index a134a39..be547e7 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.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;
@@ -64,61 +65,39 @@
  */
 public class ExternalAggregateTest 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"))) });
 
     static final boolean isOutputFile = true;
 
-    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, boolean isFile, String prefix)
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, boolean isFile, String prefix)
             throws IOException {
         AbstractSingleActivityOperatorDescriptor printer;
 
         if (!isOutputFile)
             printer = new PrinterOperatorDescriptor(spec);
         else
-            printer = new PlainFileWriterOperatorDescriptor(spec,
-                    new ConstantFileSplitProvider(new FileSplit[] {
-                            new FileSplit(NC1_ID, createTempFile()
-                                    .getAbsolutePath()),
-                            new FileSplit(NC2_ID, createTempFile()
-                                    .getAbsolutePath()) }), "\t");
+            printer = new PlainFileWriterOperatorDescriptor(spec, new ConstantFileSplitProvider(new FileSplit[] {
+                    new FileSplit(NC1_ID, createTempFile().getAbsolutePath()),
+                    new FileSplit(NC2_ID, createTempFile().getAbsolutePath()) }), "\t");
 
         return printer;
     }
@@ -127,51 +106,38 @@
     public void hashSingleKeyScalarGroupTest() 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 });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 3;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new CountAggregatorDescriptorFactory(),
-                new IntSumAggregatorDescriptorFactory(keyFields.length),
-                outputRec,
-                new HashSpillableGroupingTableFactory(
-                        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 CountAggregatorDescriptorFactory(),
+                new IntSumAggregatorDescriptorFactory(keyFields.length), outputRec,
+                new HashSpillableGroupingTableFactory(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,
-                isOutputFile, "hashSingleKeyScalarGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
+                "hashSingleKeyScalarGroupTest");
 
-        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);
@@ -184,56 +150,40 @@
     public void hashMultipleKeyScalarGroupTest() 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, });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, });
 
         int[] keyFields = new int[] { 0, 9 };
         int frameLimits = 3;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new IntSumAggregatorDescriptorFactory(1),
-                new IntSumAggregatorDescriptorFactory(keyFields.length),
-                outputRec,
-                new HashSpillableGroupingTableFactory(
-                        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 IntSumAggregatorDescriptorFactory(1),
+                new IntSumAggregatorDescriptorFactory(keyFields.length), outputRec,
+                new HashSpillableGroupingTableFactory(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,
-                isOutputFile, "hashMultipleKeyScalarGroupTest");
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
+                "hashMultipleKeyScalarGroupTest");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -246,63 +196,43 @@
     public void hashMultipleKeyMultipleScalarGroupTest() 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[] { 0, 9 };
         int frameLimits = 3;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] {
-                                new IntSumAggregatorDescriptorFactory(1, 2),
-                                new IntSumAggregatorDescriptorFactory(2, 3) }),
-                new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] {
-                                new IntSumAggregatorDescriptorFactory(2, 2),
-                                new IntSumAggregatorDescriptorFactory(3, 3) }),
-                outputRec,
-                new HashSpillableGroupingTableFactory(
-                        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 MultiAggregatorDescriptorFactory(
+                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(1, 2),
+                                new IntSumAggregatorDescriptorFactory(2, 3) }), new MultiAggregatorDescriptorFactory(
+                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(2, 2),
+                                new IntSumAggregatorDescriptorFactory(3, 3) }), outputRec,
+                new HashSpillableGroupingTableFactory(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,
-                isOutputFile, "hashMultipleKeyMultipleScalarGroupTest");
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
+                "hashMultipleKeyMultipleScalarGroupTest");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -315,50 +245,37 @@
     public void hashMultipleKeyNonScalarGroupTest() 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 });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 3;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new ConcatAggregatorDescriptorFactory(9),
-                new ConcatAggregatorDescriptorFactory(keyFields.length),
-                outputRec,
-                new HashSpillableGroupingTableFactory(
-                        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 ConcatAggregatorDescriptorFactory(9),
+                new ConcatAggregatorDescriptorFactory(keyFields.length), outputRec,
+                new HashSpillableGroupingTableFactory(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,
-                isOutputFile, "hashMultipleKeyNonScalarGroupTest");
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
+                "hashMultipleKeyNonScalarGroupTest");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -371,66 +288,46 @@
     public void hashMultipleKeyMultipleFieldsGroupTest() 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,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0, 9 };
         int frameLimits = 3;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        UTF8StringBinaryComparatorFactory.INSTANCE,
-                        UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] {
-                                new IntSumAggregatorDescriptorFactory(1, 2),
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiAggregatorDescriptorFactory(
+                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(1, 2),
                                 new IntSumAggregatorDescriptorFactory(2, 3),
-                                new ConcatAggregatorDescriptorFactory(9, 4) }),
-                new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] {
-                                new IntSumAggregatorDescriptorFactory(2, 2),
+                                new ConcatAggregatorDescriptorFactory(9, 4) }), new MultiAggregatorDescriptorFactory(
+                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(2, 2),
                                 new IntSumAggregatorDescriptorFactory(3, 3),
-                                new ConcatAggregatorDescriptorFactory(4, 4) }),
-                outputRec,
-                new HashSpillableGroupingTableFactory(
-                        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 ConcatAggregatorDescriptorFactory(4, 4) }), outputRec,
+                new HashSpillableGroupingTableFactory(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,
-                isOutputFile, "hashMultipleKeyMultipleFieldsGroupTest");
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
+                "hashMultipleKeyMultipleFieldsGroupTest");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -443,51 +340,38 @@
     public void hashSingleKeyScalarAvgGroupTest() 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 });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 3;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new AvgAggregatorDescriptorFactory(1),
-                new AvgAggregatorDescriptorFactory(keyFields.length),
-                outputRec,
-                new HashSpillableGroupingTableFactory(
-                        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 AvgAggregatorDescriptorFactory(1),
+                new AvgAggregatorDescriptorFactory(keyFields.length), outputRec, new HashSpillableGroupingTableFactory(
+                        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,
-                isOutputFile, "hashSingleKeyScalarGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
+                "hashSingleKeyScalarGroupTest");
 
-        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/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 68e33cb..a1a5179 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
@@ -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;
@@ -205,45 +206,46 @@
                         spec,
                         keys,
                         framesLimit,
-                        new IBinaryComparatorFactory[] {
-                        // IntegerBinaryComparatorFactory.INSTANCE,
-                        IntegerBinaryComparatorFactory.INSTANCE },
+                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
                         new IntegerNormalizedKeyComputerFactory(),
                         new MultiAggregatorDescriptorFactory(
                                 new IAggregatorDescriptorFactory[] { new CountAggregatorDescriptorFactory() }),
                         new MultiAggregatorDescriptorFactory(
                                 new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(keys.length) }),
                         outDesc, new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                IntegerBinaryHashFunctionFactory.INSTANCE }), htSize), false);
+                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                        .of(IntegerPointable.FACTORY) }), htSize), false);
 
                 createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
                 IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                IntegerBinaryHashFunctionFactory.INSTANCE }));
+                        new FieldHashPartitionComputerFactory(keys,
+                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                        .of(IntegerPointable.FACTORY) }));
                 spec.connect(scanGroupConn, fileScanner, 0, grouper, 0);
                 break;
             case 1: // External sort + pre-cluster
-                ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, framesLimit, keys,
-                        new IBinaryComparatorFactory[] {
-                        // IntegerBinaryComparatorFactory.INSTANCE,
-                        IntegerBinaryComparatorFactory.INSTANCE }, inDesc);
+                ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
+                        spec,
+                        framesLimit,
+                        keys,
+                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        inDesc);
                 createPartitionConstraint(spec, sorter, inSplits);
 
                 // Connect scan operator with the sorter
                 IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                IntegerBinaryHashFunctionFactory.INSTANCE }));
+                        new FieldHashPartitionComputerFactory(keys,
+                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                        .of(IntegerPointable.FACTORY) }));
                 spec.connect(scanSortConn, fileScanner, 0, sorter, 0);
 
-                grouper = new PreclusteredGroupOperatorDescriptor(spec, keys, new IBinaryComparatorFactory[] {
-                // IntegerBinaryComparatorFactory.INSTANCE,
-                        IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                grouper = new PreclusteredGroupOperatorDescriptor(
+                        spec,
+                        keys,
+                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        new MultiAggregatorFactory(
                                 new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc);
 
                 createPartitionConstraint(spec, grouper, outSplits);
@@ -253,12 +255,14 @@
                 spec.connect(sortGroupConn, sorter, 0, grouper, 0);
                 break;
             case 2: // In-memory hash group
-                grouper = new HashGroupOperatorDescriptor(spec, keys, new FieldHashPartitionComputerFactory(keys,
-                        new IBinaryHashFunctionFactory[] {
-                        // IntegerBinaryHashFunctionFactory.INSTANCE,
-                        IntegerBinaryHashFunctionFactory.INSTANCE }), new IBinaryComparatorFactory[] {
-                // IntegerBinaryComparatorFactory.INSTANCE,
-                        IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                grouper = new HashGroupOperatorDescriptor(
+                        spec,
+                        keys,
+                        new FieldHashPartitionComputerFactory(keys,
+                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                        .of(IntegerPointable.FACTORY) }),
+                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                        new MultiAggregatorFactory(
                                 new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
                         htSize);
 
@@ -266,9 +270,9 @@
 
                 // Connect scanner with the grouper
                 IConnectorDescriptor scanConn = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                IntegerBinaryHashFunctionFactory.INSTANCE }));
+                        new FieldHashPartitionComputerFactory(keys,
+                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                        .of(IntegerPointable.FACTORY) }));
                 spec.connect(scanConn, fileScanner, 0, grouper, 0);
                 break;
             default:
@@ -276,26 +280,23 @@
                         spec,
                         keys,
                         framesLimit,
-                        new IBinaryComparatorFactory[] {
-                        // IntegerBinaryComparatorFactory.INSTANCE,
-                        IntegerBinaryComparatorFactory.INSTANCE },
+                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
                         new IntegerNormalizedKeyComputerFactory(),
                         new MultiAggregatorDescriptorFactory(
                                 new IAggregatorDescriptorFactory[] { new CountAggregatorDescriptorFactory() }),
                         new MultiAggregatorDescriptorFactory(
                                 new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(keys.length) }),
                         outDesc, new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                IntegerBinaryHashFunctionFactory.INSTANCE }), htSize), false);
+                                new IBinaryHashFunctionFactory[] { 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[] {
-                        // IntegerBinaryHashFunctionFactory.INSTANCE,
-                                IntegerBinaryHashFunctionFactory.INSTANCE }));
+                        new FieldHashPartitionComputerFactory(keys,
+                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                        .of(IntegerPointable.FACTORY) }));
                 spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
         }
 
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 0fa6424..17acc105 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
@@ -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;
@@ -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 MultiAggregatorFactory(
                             new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                     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 MultiAggregatorFactory(
                             new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                     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 a43aafb..cffa0f4 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
@@ -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;
@@ -117,24 +118,16 @@
         CmdLineParser parser = new CmdLineParser(options);
         parser.parseArgument(args);
 
-        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host,
-                options.port);
+        IHyracksClientConnection hcc = new HyracksRMIConnection(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 MultiAggregatorFactory(
                             new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                     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-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/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/data/PointablePrimitiveValueProviderFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/data/PointablePrimitiveValueProviderFactory.java
new file mode 100644
index 0000000..c159ba5
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/data/PointablePrimitiveValueProviderFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.storage.am.common.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.INumeric;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+
+public class PointablePrimitiveValueProviderFactory implements IPrimitiveValueProviderFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final IPointableFactory pf;
+
+    public PointablePrimitiveValueProviderFactory(IPointableFactory pf) {
+        this.pf = pf;
+    }
+
+    @Override
+    public IPrimitiveValueProvider createPrimitiveValueProvider() {
+        final IPointable p = pf.createPointable();
+        ITypeTraits traits = pf.getTypeTraits();
+        assert traits.isFixedLength();
+        final int length = traits.getFixedLength();
+        return new IPrimitiveValueProvider() {
+            @Override
+            public double getValue(byte[] bytes, int offset) {
+                p.set(bytes, offset, length);
+                return ((INumeric) p).doubleValue();
+            }
+        };
+    }
+}
\ No newline at end of file
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/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..47e3246 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;
@@ -79,7 +79,7 @@
 
     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 7bdec3c..18c5cb0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -86,6 +86,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>