You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org> on 2015/11/26 19:29:59 UTC

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Yingyi Bu has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/522

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
38 files changed, 456 insertions(+), 773 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/1

diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
index 677e913..29ce728 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
@@ -20,6 +20,7 @@
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.List;
 
@@ -41,7 +42,6 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
 import org.apache.asterix.external.library.TypeInfo;
 import org.apache.asterix.external.library.java.JObjects.JBoolean;
@@ -86,6 +86,7 @@
 import org.apache.asterix.om.util.container.IObjectPool;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 public class JObjectAccessors {
 
@@ -226,7 +227,7 @@
     }
 
     public static class JStringAccessor implements IJObjectAccessor {
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final UTF8StringReader reader = new UTF8StringReader();
 
         @Override
         public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
@@ -236,8 +237,11 @@
             int l = pointable.getLength();
 
             String v = null;
-            v = aStringSerDer.deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)))
-                    .getStringValue();
+            try {
+                v = reader.readUTF(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
             JObjectUtil.getNormalizedString(v);
 
             IJObject jObject = objectPool.allocate(BuiltinType.ASTRING);
@@ -444,7 +448,7 @@
         private final JRecord jRecord;
         private final IJObject[] jObjects;
         private final LinkedHashMap<String, IJObject> openFields;
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final UTF8StringReader reader = new UTF8StringReader();
 
         public JRecordAccessor(ARecordType recordType, IObjectPool<IJObject, IAType> objectPool) {
             this.typeInfo = new TypeInfo(objectPool, null, null);
@@ -502,9 +506,7 @@
                         byte[] b = fieldName.getByteArray();
                         int s = fieldName.getStartOffset();
                         int l = fieldName.getLength();
-                        String v = aStringSerDer
-                                .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)))
-                                .getStringValue();
+                        String v = reader.readUTF(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
                         openFields.put(v, fieldObject);
                     }
                     index++;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
index b5458e2..04a1e80 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
@@ -392,7 +392,7 @@
                         dis.readInt();
                     }
                     for (int i = 0; i < numberOfOpenFields; i++) {
-                        fieldNames[i] = new AStringSerializerDeserializer().deserialize(dis).getStringValue();
+                        fieldNames[i] = AStringSerializerDeserializer.INSTANCE.deserialize(dis).getStringValue();
                         ATypeTag openFieldTypeTag = SerializerDeserializerUtil.deserializeTag(dis);
                         openFields[i] = getJType(openFieldTypeTag, null, dis, objectPool);
                         fieldTypes[i] = openFields[i].getIAObject().getType();
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
index 406d242..e9a63b2 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
@@ -366,7 +366,7 @@
 
     public static final class JString extends JObject {
 
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final AStringSerializerDeserializer aStringSerDer = AStringSerializerDeserializer.INSTANCE;
 
         public JString(String v) {
             super(new AMutableString(v));
@@ -978,7 +978,7 @@
         private ARecordType recordType;
         private IJObject[] fields;
         private Map<String, IJObject> openFields;
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final AStringSerializerDeserializer aStringSerDer = AStringSerializerDeserializer.INSTANCE;
 
         public JRecord(ARecordType recordType, IJObject[] fields) {
             this.recordType = recordType;
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
index e81da3f..ef3ac49 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
@@ -24,19 +24,18 @@
 import java.io.DataInputStream;
 
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IValueExtractor;
-import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 /**
  * Extracts the value of field 'DatasetName' from an ITupleReference that
  * contains a serialized representation of a Dataset metadata entity.
  */
 public class DatasetNameValueExtractor implements IValueExtractor<String> {
-    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+    private final UTF8StringReader reader = new UTF8StringReader();
 
     @Override
     public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
@@ -45,6 +44,10 @@
         int recordLength = tuple.getFieldLength(2);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        return (((AString) aObjSerDer.deserialize(in)).getStringValue());
+        try {
+            return reader.readUTF(in);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
     }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
index 9a50a31..edb3d2e 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
@@ -22,16 +22,15 @@
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
-import java.rmi.RemoteException;
+import java.io.IOException;
 
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.api.IValueExtractor;
-import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 /**
  * Extracts the value of field 'DataypeName' from an ITupleReference that
@@ -40,7 +39,7 @@
 public class DatatypeNameValueExtractor implements IValueExtractor<String> {
     private final String dataverseName;
     private final MetadataNode metadataNode;
-    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+    private final UTF8StringReader reader = new UTF8StringReader();
 
     public DatatypeNameValueExtractor(String dataverseName, MetadataNode metadataNode) {
         this.dataverseName = dataverseName;
@@ -54,16 +53,16 @@
         int recordLength = tuple.getFieldLength(2);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        String typeName = ((AString) aObjSerDer.deserialize(in)).getStringValue();
         try {
+            String typeName = reader.readUTF(in);
             if (metadataNode.getDatatype(jobId, dataverseName, typeName).getIsAnonymous()) {
                 // Get index 0 because it is anonymous type, and it is used in
                 // only one non-anonymous type.
                 typeName = metadataNode.getDatatypeNamesUsingThisDatatype(jobId, dataverseName, typeName).get(0);
             }
-        } catch (RemoteException e) {
+            return typeName;
+        } catch (IOException e) {
             throw new MetadataException(e);
         }
-        return typeName;
     }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 41d92c9..bdc36e8 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -22,14 +22,14 @@
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
+import java.io.IOException;
 
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IValueExtractor;
-import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 /**
  * Extracts the value of field 'DataypeName' of the first nested type from an
@@ -43,7 +43,8 @@
     public NestedDatatypeNameValueExtractor(String datatypeName) {
         this.datatypeName = datatypeName;
     }
-    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+
+    private final UTF8StringReader reader = new UTF8StringReader();
 
     @Override
     public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
@@ -52,14 +53,18 @@
         int recordLength = tuple.getFieldLength(2);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        String nestedType = ((AString) aObjSerDer.deserialize(in)).getStringValue();
-        if (nestedType.equals(datatypeName)) {
-            recordStartOffset = tuple.getFieldStart(1);
-            recordLength = tuple.getFieldLength(1);
-            stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
-            in = new DataInputStream(stream);
-            return ((AString) aObjSerDer.deserialize(in)).getStringValue();
+        try {
+            String nestedType = reader.readUTF(in);
+            if (nestedType.equals(datatypeName)) {
+                recordStartOffset = tuple.getFieldStart(1);
+                recordLength = tuple.getFieldLength(1);
+                stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+                in = new DataInputStream(stream);
+                return reader.readUTF(in);
+            }
+            return null;
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
         }
-        return null;
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
index 3bde3b3..ee180bf 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
@@ -30,19 +30,20 @@
 public class ABinarySerializerDeserializer implements ISerializerDeserializer<ABinary> {
 
     private static final long serialVersionUID = 1L;
+    public static final ABinarySerializerDeserializer INSTANCE = new ABinarySerializerDeserializer();
 
-    public ABinarySerializerDeserializer() {}
-
-    private ByteArraySerializerDeserializer byteArrayPtrSerDer = new ByteArraySerializerDeserializer();
+    private ABinarySerializerDeserializer() {
+    }
 
     @Override
     public ABinary deserialize(DataInput in) throws HyracksDataException {
-        return new ABinary(byteArrayPtrSerDer.deserialize(in));
+        return new ABinary(ByteArraySerializerDeserializer.INSTANCE.deserialize(in));
     }
 
     @Override
     public void serialize(ABinary binary, DataOutput out) throws HyracksDataException {
-        byteArrayPtrSerDer.serialize(binary.getBytes(), binary.getStart(), binary.getLength(), out);
+        ByteArraySerializerDeserializer.INSTANCE.serialize(binary.getBytes(), binary.getStart(), binary.getLength(),
+                out);
     }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
index a80047d..4460fd9 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
@@ -23,12 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ACircle;
-import org.apache.asterix.om.base.AMutableCircle;
-import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.APoint;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,12 +33,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ACircleSerializerDeserializer INSTANCE = new ACircleSerializerDeserializer();
-
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<ACircle> circleSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ACIRCLE);
-    private final static AMutableCircle aCircle = new AMutableCircle(null, 0);
-    private final static AMutablePoint aPoint = new AMutablePoint(0, 0);
 
     private ACircleSerializerDeserializer() {
     }
@@ -82,16 +72,5 @@
 
     public final static int getRadiusOffset() throws HyracksDataException {
         return 17;
-    }
-
-    public static void parse(String circle, DataOutput out) throws HyracksDataException {
-        try {
-            String[] parts = circle.split(" ");
-            aPoint.setValue(Double.parseDouble(parts[0].split(",")[0]), Double.parseDouble(parts[0].split(",")[1]));
-            aCircle.setValue(aPoint, Double.parseDouble(parts[1].substring(0, parts[1].length())));
-            circleSerde.serialize(aCircle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(circle + " can not be an instance of circle");
-        }
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
index 081a08b..9509b0b 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,9 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ADateSerializerDeserializer INSTANCE = new ADateSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADate> dateSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADATE);
 
     private ADateSerializerDeserializer() {
     }
@@ -59,25 +51,6 @@
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-    }
-
-    public static void parse(String date, DataOutput out) throws HyracksDataException {
-        AMutableDate aDate = new AMutableDate(0);
-
-        long chrononTimeInMs = 0;
-        try {
-            chrononTimeInMs = ADateParserFactory.parseDatePart(date, 0, date.length());
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        short temp = 0;
-        if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
-            temp = 1;
-        }
-        aDate.setValue((int) (chrononTimeInMs / GregorianCalendarSystem.CHRONON_OF_DAY) - temp);
-
-        dateSerde.serialize(aDate, out);
     }
 
     public static int getChronon(byte[] byteArray, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
index 657b60e..6eb3600 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
@@ -22,13 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,11 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ADateTimeSerializerDeserializer INSTANCE = new ADateTimeSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADateTime> datetimeSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADATETIME);
-    private static final AMutableDateTime aDateTime = new AMutableDateTime(0L);
-    private static final String errorMessage = "This can not be an instance of datetime";
 
     private ADateTimeSerializerDeserializer() {
     }
@@ -62,35 +51,6 @@
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-    }
-
-    public static void parse(String datetime, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMs = 0;
-        try {
-
-            // +1 if it is negative (-)
-            short timeOffset = (short) ((datetime.charAt(0) == '-') ? 1 : 0);
-
-            timeOffset += 8;
-
-            if (datetime.charAt(timeOffset) != 'T') {
-                timeOffset += 2;
-                if (datetime.charAt(timeOffset) != 'T') {
-                    throw new AlgebricksException(errorMessage + ": missing T");
-                }
-            }
-
-            chrononTimeInMs = ADateParserFactory.parseDatePart(datetime, 0, timeOffset);
-
-            chrononTimeInMs += ATimeParserFactory.parseTimePart(datetime, timeOffset + 1, datetime.length()
-                    - timeOffset - 1);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-        aDateTime.setValue(chrononTimeInMs);
-
-        datetimeSerde.serialize(aDateTime, out);
     }
 
     public static long getChronon(byte[] data, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
index 79f3239..c2e7729 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADayTimeDuration;
-import org.apache.asterix.om.base.AMutableDayTimeDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,10 +32,8 @@
 
     public static final ADayTimeDurationSerializerDeserializer INSTANCE = new ADayTimeDurationSerializerDeserializer();
 
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
-    private static final AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
+    private ADayTimeDurationSerializerDeserializer() {
+    }
 
     @Override
     public ADayTimeDuration deserialize(DataInput in) throws HyracksDataException {
@@ -57,16 +50,6 @@
             out.writeLong(instance.getMilliseconds());
         } catch (IOException e) {
             throw new HyracksDataException();
-        }
-    }
-
-    public void parse(String durationString, DataOutput out) throws HyracksDataException {
-        try {
-            ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
-                    ADurationParseOption.All);
-            dayTimeDurationSerde.serialize(aDayTimeDuration, out);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
         }
     }
 
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
index 3108ac9..5eb8369 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADuration;
-import org.apache.asterix.om.base.AMutableDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,10 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ADurationSerializerDeserializer INSTANCE = new ADurationSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADURATION);
-    private static final AMutableDuration aDuration = new AMutableDuration(0, 0);
 
     private ADurationSerializerDeserializer() {
     }
@@ -63,18 +54,9 @@
         }
     }
 
-    public static void parse(String duration, DataOutput out) throws HyracksDataException {
-        try {
-            ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration, ADurationParseOption.All);
-            durationSerde.serialize(aDuration, out);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
     /**
      * Get the year-month field of the duration as an integer number of days.
-     * 
+     *
      * @param data
      * @param offset
      * @return
@@ -85,7 +67,7 @@
 
     /**
      * Get the day-time field of the duration as an long integer number of milliseconds.
-     * 
+     *
      * @param data
      * @param offset
      * @return
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
index a0469d7..a0c2657 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
@@ -32,6 +32,9 @@
 
     public static final AInt16SerializerDeserializer INSTANCE = new AInt16SerializerDeserializer();
 
+    private AInt16SerializerDeserializer() {
+    }
+
     @Override
     public AInt16 deserialize(DataInput in) throws HyracksDataException {
         try {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
index 80ba3ab..16c5f46 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
@@ -32,6 +32,9 @@
 
     public static final AInt8SerializerDeserializer INSTANCE = new AInt8SerializerDeserializer();
 
+    private AInt8SerializerDeserializer() {
+    }
+
     @Override
     public AInt8 deserialize(DataInput in) throws HyracksDataException {
         try {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
index 5300b28..5d08d52 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
@@ -22,15 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AInterval;
-import org.apache.asterix.om.base.AMutableInterval;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -39,11 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final AIntervalSerializerDeserializer INSTANCE = new AIntervalSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<AInterval> intervalSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINTERVAL);
-    private static final AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
-    private static final String errorMessage = "This can not be an instance of interval";
 
     private AIntervalSerializerDeserializer() {
     }
@@ -87,181 +74,6 @@
 
     public static byte getIntervalTimeType(byte[] data, int offset) {
         return data[offset + 8 * 2];
-    }
-
-    /**
-     * create an interval value from two given datetime instance.
-     * 
-     * @param interval
-     * @param out
-     * @throws HyracksDataException
-     */
-    public static void parseDatetime(String interval, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMsStart = 0;
-        long chrononTimeInMsEnd = 0;
-        try {
-
-            // the starting point for parsing (so for the accessor)
-            int startOffset = 0;
-            int endOffset, timeSeperatorOffsetInDatetimeString;
-
-            // Get the index for the comma
-            int commaIndex = interval.indexOf(',');
-            if (commaIndex < 1) {
-                throw new AlgebricksException("comma is missing for a string of interval");
-            }
-
-            endOffset = commaIndex - 1;
-
-            timeSeperatorOffsetInDatetimeString = interval.indexOf('T');
-
-            if (timeSeperatorOffsetInDatetimeString < 0) {
-                throw new AlgebricksException(errorMessage + ": missing T for a datetime value.");
-            }
-
-            chrononTimeInMsStart = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
-
-            chrononTimeInMsStart += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
-
-            // Interval End
-            startOffset = commaIndex + 1;
-            endOffset = interval.length() - 1;
-
-            timeSeperatorOffsetInDatetimeString = interval.indexOf('T', startOffset);
-
-            if (timeSeperatorOffsetInDatetimeString < 0) {
-                throw new AlgebricksException(errorMessage + ": missing T for a datetime value.");
-            }
-
-            chrononTimeInMsEnd = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
-
-            chrononTimeInMsEnd += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        try {
-            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.DATETIME.serialize());
-        } catch (AlgebricksException e) {
-            throw new HyracksDataException(e);
-        }
-
-        intervalSerde.serialize(aInterval, out);
-    }
-
-    public static void parseTime(String interval, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMsStart = 0;
-        long chrononTimeInMsEnd = 0;
-        try {
-
-            int startOffset = 0;
-            int endOffset;
-
-            // Get the index for the comma
-            int commaIndex = interval.indexOf(',');
-            if (commaIndex < 0) {
-                throw new AlgebricksException("comma is missing for a string of interval");
-            }
-
-            endOffset = commaIndex - 1;
-
-            // Interval Start
-            chrononTimeInMsStart = parseTimePart(interval, startOffset, endOffset);
-
-            if (chrononTimeInMsStart < 0) {
-                chrononTimeInMsStart += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-
-            // Interval End
-            startOffset = commaIndex + 1;
-            endOffset = interval.length() - 1;
-
-            chrononTimeInMsEnd = parseTimePart(interval, startOffset, endOffset);
-
-            if (chrononTimeInMsEnd < 0) {
-                chrononTimeInMsEnd += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        try {
-            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.TIME.serialize());
-        } catch (AlgebricksException e) {
-            throw new HyracksDataException(e);
-        }
-        intervalSerde.serialize(aInterval, out);
-    }
-
-    public static void parseDate(String interval, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMsStart = 0;
-        long chrononTimeInMsEnd = 0;
-        try {
-
-            // the starting point for parsing (so for the accessor)
-            int startOffset = 0;
-            int endOffset;
-
-            // Get the index for the comma
-            int commaIndex = interval.indexOf(',');
-            if (commaIndex < 1) {
-                throw new AlgebricksException("comma is missing for a string of interval");
-            }
-
-            endOffset = commaIndex - 1;
-
-            chrononTimeInMsStart = parseDatePart(interval, startOffset, endOffset);
-
-            // Interval End
-            startOffset = commaIndex + 1;
-            endOffset = interval.length() - 1;
-
-            chrononTimeInMsEnd = parseDatePart(interval, startOffset, endOffset);
-
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        try {
-            aInterval.setValue((chrononTimeInMsStart / GregorianCalendarSystem.CHRONON_OF_DAY),
-                    (chrononTimeInMsEnd / GregorianCalendarSystem.CHRONON_OF_DAY), ATypeTag.DATE.serialize());
-        } catch (AlgebricksException e) {
-            throw new HyracksDataException(e);
-        }
-
-        intervalSerde.serialize(aInterval, out);
-    }
-
-    private static long parseDatePart(String interval, int startOffset, int endOffset) throws AlgebricksException,
-            HyracksDataException {
-
-        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
-            endOffset--;
-        }
-
-        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
-            startOffset++;
-        }
-
-        return ADateParserFactory.parseDatePart(interval, startOffset, endOffset - startOffset + 1);
-    }
-
-    private static int parseTimePart(String interval, int startOffset, int endOffset) throws AlgebricksException,
-            HyracksDataException {
-
-        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
-            endOffset--;
-        }
-
-        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
-            startOffset++;
-        }
-
-        return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
     }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
index 67408cd..aef8de0 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
@@ -23,12 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ALine;
-import org.apache.asterix.om.base.AMutableLine;
-import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.APoint;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -38,12 +34,8 @@
 
     public static final ALineSerializerDeserializer INSTANCE = new ALineSerializerDeserializer();
 
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<ALine> lineSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ALINE);
-    private final static AMutableLine aLine = new AMutableLine(null, null);
-    private final static AMutablePoint aLinePoint1 = new AMutablePoint(0, 0);
-    private final static AMutablePoint aLinePoint2 = new AMutablePoint(0, 0);
+    private ALineSerializerDeserializer() {
+    }
 
     @Override
     public ALine deserialize(DataInput in) throws HyracksDataException {
@@ -92,19 +84,4 @@
         }
     }
 
-    public static void parse(String line, DataOutput out) throws HyracksDataException {
-        try {
-            String[] points = line.split(" ");
-            if (points.length != 2)
-                throw new HyracksDataException("line consists of only 2 points.");
-            aLinePoint1.setValue(Double.parseDouble(points[0].split(",")[0]),
-                    Double.parseDouble(points[0].split(",")[1]));
-            aLinePoint2.setValue(Double.parseDouble(points[1].split(",")[0]),
-                    Double.parseDouble(points[1].split(",")[1]));
-            aLine.setValue(aLinePoint1, aLinePoint2);
-            lineSerde.serialize(aLine, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(line + " can not be an instance of line");
-        }
-    }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index c2c9fd6..9aff59e 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -57,105 +57,69 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
-    private final ABinarySerializerDeserializer aBinarySerDer = new ABinarySerializerDeserializer();
-    private ARecordSerializerDeserializer aRecordSerDer = null;
+    public static final AObjectSerializerDeserializer INSTANCE = new AObjectSerializerDeserializer();
 
-    private ARecordSerializerDeserializer getRecordSerDer() {
-        if (aRecordSerDer == null) {
-            aRecordSerDer = ARecordSerializerDeserializer.CREATE_SCHEMALESS_INSTANCE();
-        }
-        return aRecordSerDer;
-    }
-
-    public AObjectSerializerDeserializer() {
+    private AObjectSerializerDeserializer() {
     }
 
     @Override
     public IAObject deserialize(DataInput in) throws HyracksDataException {
         ATypeTag typeTag = SerializerDeserializerUtil.deserializeTag(in);
         switch (typeTag) {
-            case NULL: {
+            case NULL:
                 return ANullSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case BOOLEAN: {
+            case BOOLEAN:
                 return ABooleanSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT8: {
+            case INT8:
                 return AInt8SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT16: {
+            case INT16:
                 return AInt16SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT32: {
+            case INT32:
                 return AInt32SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT64: {
+            case INT64:
                 return AInt64SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case FLOAT: {
+            case FLOAT:
                 return AFloatSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DOUBLE: {
+            case DOUBLE:
                 return ADoubleSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case STRING: {
-                return aStringSerDer.deserialize(in);
-            }
-            case BINARY: {
-                return aBinarySerDer.deserialize(in);
-            }
-            case DATE: {
+            case STRING:
+                return AStringSerializerDeserializer.INSTANCE.deserialize(in);
+            case BINARY:
+                return ABinarySerializerDeserializer.INSTANCE.deserialize(in);
+            case DATE:
                 return ADateSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case TIME: {
+            case TIME:
                 return ATimeSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DATETIME: {
+            case DATETIME:
                 return ADateTimeSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DURATION: {
+            case DURATION:
                 return ADurationSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case YEARMONTHDURATION: {
+            case YEARMONTHDURATION:
                 return AYearMonthDurationSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DAYTIMEDURATION: {
+            case DAYTIMEDURATION:
                 return ADayTimeDurationSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INTERVAL: {
+            case INTERVAL:
                 return AIntervalSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case POINT: {
+            case POINT:
                 return APointSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case POINT3D: {
+            case POINT3D:
                 return APoint3DSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case LINE: {
+            case LINE:
                 return ALineSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case RECTANGLE: {
+            case RECTANGLE:
                 return ARectangleSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case POLYGON: {
+            case POLYGON:
                 return APolygonSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case CIRCLE: {
+            case CIRCLE:
                 return ACircleSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case RECORD: {
-                return getRecordSerDer().deserialize(in);
-            }
-            case ORDEREDLIST: {
+            case RECORD:
+                return ARecordSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
+            case ORDEREDLIST:
                 return AOrderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
-            }
-            case UNORDEREDLIST: {
+            case UNORDEREDLIST:
                 return AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
-            }
-            default: {
+            default:
                 throw new NotImplementedException("No serializer/deserializer implemented for type " + typeTag + " .");
-            }
         }
     }
 
@@ -202,11 +166,11 @@
                 break;
             }
             case STRING: {
-                aStringSerDer.serialize((AString) instance, out);
+                AStringSerializerDeserializer.INSTANCE.serialize((AString) instance, out);
                 break;
             }
             case BINARY: {
-                aBinarySerDer.serialize((ABinary) instance, out);
+                ABinarySerializerDeserializer.INSTANCE.serialize((ABinary) instance, out);
                 break;
             }
             case DATE: {
@@ -254,7 +218,7 @@
                 break;
             }
             case RECORD: {
-                getRecordSerDer().serialize((ARecord) instance, out);
+                ARecordSerializerDeserializer.SCHEMALESS_INSTANCE.serialize((ARecord) instance, out);
                 break;
             }
             case ORDEREDLIST: {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
index e8317eb..5a7ff0b 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
@@ -20,8 +20,8 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -43,30 +43,24 @@
     private static final long serialVersionUID = 1L;
     public static final AOrderedListSerializerDeserializer SCHEMALESS_INSTANCE = new AOrderedListSerializerDeserializer();
 
-    private IAType itemType;
+    private final IAType itemType;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer serializer;
+    private final ISerializerDeserializer serializer;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer deserializer;
-    private AOrderedListType orderedlistType;
+    private final ISerializerDeserializer deserializer;
+    private final AOrderedListType orderedlistType;
 
     private AOrderedListSerializerDeserializer() {
-        this.itemType = null;
-        this.orderedlistType = null;
-        initSerializerDeserializer(BuiltinType.ANY);
+        this(new AOrderedListType(BuiltinType.ANY, "orderedlist"));
     }
 
     public AOrderedListSerializerDeserializer(AOrderedListType orderedlistType) {
         this.orderedlistType = orderedlistType;
-        initSerializerDeserializer(orderedlistType.getItemType());
-    }
-
-    private void initSerializerDeserializer(IAType itemType) {
-        this.itemType = itemType;
+        this.itemType = orderedlistType.getItemType();
         serializer = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-        deserializer = itemType.getTypeTag() == ATypeTag.ANY ? AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(itemType) : AqlSerializerDeserializerProvider.INSTANCE
-                .getNonTaggedSerializerDeserializer(itemType);
+        deserializer = itemType.getTypeTag() == ATypeTag.ANY
+                ? AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType)
+                : AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(itemType);
     }
 
     @Override
@@ -87,33 +81,33 @@
                     fixedSize = true;
                     break;
             }
-            
-            if (itemType.getTypeTag() == ATypeTag.ANY && typeTag != ATypeTag.ANY)   
-            try {   
-                initSerializerDeserializer(BuiltinType.builtinTypeFromString(typeTag.name().toLowerCase()));    
-            } catch (AsterixException e) {  
-                throw new HyracksDataException(e);  
-            }
-            
 
+            IAType currentItemType = itemType;
+            @SuppressWarnings("rawtypes")
+            ISerializerDeserializer currentDeserializer = deserializer;
+            if (itemType.getTypeTag() == ATypeTag.ANY && typeTag != ATypeTag.ANY) {
+                currentItemType = BuiltinType.builtinTypeFromString(typeTag.name().toLowerCase());
+                currentDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+                        .getNonTaggedSerializerDeserializer(currentItemType);
+            }
+
+            List<IAObject> items = new ArrayList<IAObject>();
             in.readInt(); // list size
             int numberOfitems;
             numberOfitems = in.readInt();
-            ArrayList<IAObject> items = new ArrayList<IAObject>();
             if (numberOfitems > 0) {
                 if (!fixedSize) {
                     for (int i = 0; i < numberOfitems; i++)
                         in.readInt();
                 }
                 for (int i = 0; i < numberOfitems; i++) {
-                    IAObject v = (IAObject) deserializer.deserialize(in);
+                    IAObject v = (IAObject) currentDeserializer.deserialize(in);
                     items.add(v);
                 }
             }
-            AOrderedListType type = new AOrderedListType(itemType, "orderedlist");
+            AOrderedListType type = new AOrderedListType(currentItemType, "orderedlist");
             return new AOrderedList(type, items);
-
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
index 69c7a71..57e54a7 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
@@ -23,11 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutablePoint3D;
 import org.apache.asterix.om.base.APoint3D;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,10 +33,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final APoint3DSerializerDeserializer INSTANCE = new APoint3DSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<APoint3D> point3DSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.APOINT3D);
-    private final static AMutablePoint3D aPoint3D = new AMutablePoint3D(0, 0, 0);
 
     private APoint3DSerializerDeserializer() {
     }
@@ -90,16 +83,4 @@
         }
     }
 
-    public static void parse(String point3d, DataOutput out) throws HyracksDataException {
-        try {
-            int firstCommaIndex = point3d.indexOf(',');
-            int secondCommaIndex = point3d.indexOf(',', firstCommaIndex + 1);
-            aPoint3D.setValue(Double.parseDouble(point3d.substring(0, firstCommaIndex)),
-                    Double.parseDouble(point3d.substring(firstCommaIndex + 1, secondCommaIndex)),
-                    Double.parseDouble(point3d.substring(secondCommaIndex + 1, point3d.length())));
-            point3DSerde.serialize(aPoint3D, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point3d + " can not be an instance of point3d");
-        }
-    }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
index 4fa06ce..a927f95 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
@@ -23,10 +23,7 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.APoint;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -35,10 +32,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final APointSerializerDeserializer INSTANCE = new APointSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<APoint> pointSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.APOINT);
-    private final static AMutablePoint aPoint = new AMutablePoint(0, 0);
 
     private APointSerializerDeserializer() {
     }
@@ -82,16 +75,6 @@
                 throw new HyracksDataException("Wrong coordinate");
         }
 
-    }
-
-    public static void parse(String point, DataOutput out) throws HyracksDataException {
-        try {
-            aPoint.setValue(Double.parseDouble(point.substring(0, point.indexOf(','))),
-                    Double.parseDouble(point.substring(point.indexOf(',') + 1, point.length())));
-            pointSerde.serialize(aPoint, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point + " can not be an instance of point");
-        }
     }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 0d1f63f..22f169d 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -48,34 +48,23 @@
 public class ARecordSerializerDeserializer implements ISerializerDeserializer<ARecord> {
     private static final long serialVersionUID = 1L;
 
-    public static final ARecordSerializerDeserializer CREATE_SCHEMALESS_INSTANCE() {
-        return new ARecordSerializerDeserializer();
-    }
+    public static final ARecordSerializerDeserializer SCHEMALESS_INSTANCE = new ARecordSerializerDeserializer();
 
-    private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
-    private AObjectSerializerDeserializer aObjSerDer = null;
-
-    private AObjectSerializerDeserializer getObjSerDer() {
-        if (aObjSerDer == null) {
-            aObjSerDer = new AObjectSerializerDeserializer();
-        }
-        return aObjSerDer;
-    }
-
-    private ARecordType recordType;
-    private int numberOfSchemaFields = 0;
+    private final ARecordType recordType;
+    private final int numberOfSchemaFields;
 
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer serializers[];
+    private final ISerializerDeserializer serializers[];
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer deserializers[];
+    private final ISerializerDeserializer deserializers[];
 
     private ARecordSerializerDeserializer() {
+        this(null);
     }
 
     public ARecordSerializerDeserializer(ARecordType recordType) {
-        this.recordType = recordType;
         if (recordType != null) {
+            this.recordType = recordType;
             this.numberOfSchemaFields = recordType.getFieldNames().length;
             serializers = new ISerializerDeserializer[numberOfSchemaFields];
             deserializers = new ISerializerDeserializer[numberOfSchemaFields];
@@ -97,6 +86,11 @@
                 serializers[i] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(t2);
                 deserializers[i] = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(t2);
             }
+        } else {
+            this.recordType = null;
+            this.numberOfSchemaFields = 0;
+            this.serializers = null;
+            this.deserializers = null;
         }
     }
 
@@ -150,8 +144,8 @@
                     in.readInt();
                 }
                 for (int i = 0; i < numberOfOpenFields; i++) {
-                    fieldNames[i] = aStringSerDer.deserialize(in).getStringValue();
-                    openFields[i] = getObjSerDer().deserialize(in);
+                    fieldNames[i] = AStringSerializerDeserializer.INSTANCE.deserialize(in).getStringValue();
+                    openFields[i] = AObjectSerializerDeserializer.INSTANCE.deserialize(in);
                     fieldTypes[i] = openFields[i].getType();
                 }
                 ARecordType openPartRecType = new ARecordType(null, fieldNames, fieldTypes, true);
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
index 4306c76..7e0bf0f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
@@ -23,12 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutablePoint;
-import org.apache.asterix.om.base.AMutableRectangle;
 import org.apache.asterix.om.base.APoint;
 import org.apache.asterix.om.base.ARectangle;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,13 +33,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ARectangleSerializerDeserializer INSTANCE = new ARectangleSerializerDeserializer();
-
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<ARectangle> rectangleSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ARECTANGLE);
-    private final static AMutableRectangle aRectangle = new AMutableRectangle(null, null);
-    private final static AMutablePoint aRectanglePoint1 = new AMutablePoint(0, 0);
-    private final static AMutablePoint aRectanglePoint2 = new AMutablePoint(0, 0);
 
     private ARectangleSerializerDeserializer() {
     }
@@ -92,30 +81,6 @@
                 return 25;
             default:
                 throw new HyracksDataException("Wrong coordinate");
-        }
-    }
-
-    public static void parse(String rectangle, DataOutput out) throws HyracksDataException {
-        try {
-            String[] points = rectangle.split(" ");
-            if (points.length != 2)
-                throw new HyracksDataException("rectangle consists of only 2 points.");
-            aRectanglePoint1.setValue(Double.parseDouble(points[0].split(",")[0]),
-                    Double.parseDouble(points[0].split(",")[1]));
-            aRectanglePoint2.setValue(Double.parseDouble(points[1].split(",")[0]),
-                    Double.parseDouble(points[1].split(",")[1]));
-            if (aRectanglePoint1.getX() > aRectanglePoint2.getX() && aRectanglePoint1.getY() > aRectanglePoint2.getY()) {
-                aRectangle.setValue(aRectanglePoint2, aRectanglePoint1);
-            } else if (aRectanglePoint1.getX() < aRectanglePoint2.getX()
-                    && aRectanglePoint1.getY() < aRectanglePoint2.getY()) {
-                aRectangle.setValue(aRectanglePoint1, aRectanglePoint2);
-            } else {
-                throw new IllegalArgumentException(
-                        "Rectangle arugment must be either (bottom left point, top right point) or (top right point, bottom left point)");
-            }
-            rectangleSerde.serialize(aRectangle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(rectangle + " can not be an instance of rectangle");
         }
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
index 272b36f..85b128f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
@@ -25,20 +25,21 @@
 import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.util.string.UTF8StringUtil;
 
 public class AStringSerializerDeserializer implements ISerializerDeserializer<AString> {
 
     private static final long serialVersionUID = 1L;
 
-    private final UTF8StringSerializerDeserializer utf8SerDer = new UTF8StringSerializerDeserializer();
+    public static final AStringSerializerDeserializer INSTANCE = new AStringSerializerDeserializer();
 
-    public AStringSerializerDeserializer() {}
+    private AStringSerializerDeserializer() {
+    }
 
     @Override
     public AString deserialize(DataInput in) throws HyracksDataException {
         try {
-            return new AString(utf8SerDer.deserialize(in));
+            return new AString(UTF8StringUtil.readUTF8(in));
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
@@ -47,7 +48,7 @@
     @Override
     public void serialize(AString instance, DataOutput out) throws HyracksDataException {
         try {
-            utf8SerDer.serialize(instance.getStringValue(), out);
+            UTF8StringUtil.writeUTF8(instance.getStringValue(), out);
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
index a059cde..1138164 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
@@ -22,11 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
 import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -35,11 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ATimeSerializerDeserializer INSTANCE = new ATimeSerializerDeserializer();
-
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ATime> timeSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ATIME);
-    private static final AMutableTime aTime = new AMutableTime(0);
 
     private ATimeSerializerDeserializer() {
     }
@@ -62,20 +53,6 @@
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-    }
-
-    public static void parse(String time, DataOutput out) throws HyracksDataException {
-        int chrononTimeInMs;
-
-        try {
-            chrononTimeInMs = ATimeParserFactory.parseTimePart(time, 0, time.length());
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        aTime.setValue(chrononTimeInMs);
-
-        timeSerde.serialize(aTime, out);
     }
 
     public static int getChronon(byte[] byteArray, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
index caf1eff..8df0b0d 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
@@ -34,6 +34,9 @@
 
     public static final AUUIDSerializerDeserializer INSTANCE = new AUUIDSerializerDeserializer();
 
+    private AUUIDSerializerDeserializer() {
+    }
+
     @Override
     public AUUID deserialize(DataInput in) throws HyracksDataException {
         long msb = Integer64SerializerDeserializer.INSTANCE.deserialize(in);
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
index e109a32..a50af01 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
@@ -20,7 +20,6 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.asterix.builders.UnorderedListBuilder;
@@ -31,6 +30,7 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
@@ -47,22 +47,21 @@
     private IAType itemType;
     private AUnorderedListType unorderedlistType;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer nontaggedSerDes;
+    private ISerializerDeserializer deserializer;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer taggedSerDes;
+    private ISerializerDeserializer serializer;
 
     private AUnorderedListSerializerDeserializer() {
-        this.itemType = null;
-        this.unorderedlistType = null;
+        this(new AUnorderedListType(BuiltinType.ANY, "unorderedlist"));
     }
 
     public AUnorderedListSerializerDeserializer(AUnorderedListType unorderedlistType) {
-        this.itemType = unorderedlistType.getItemType();
         this.unorderedlistType = unorderedlistType;
-        nontaggedSerDes = itemType.getTypeTag() == ATypeTag.ANY
+        this.itemType = unorderedlistType.getItemType();
+        serializer = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
+        deserializer = itemType.getTypeTag() == ATypeTag.ANY
                 ? AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType)
                 : AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(itemType);
-        taggedSerDes = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
     }
 
     @Override
@@ -84,6 +83,16 @@
                     fixedSize = true;
                     break;
             }
+
+            IAType currentItemType = itemType;
+            @SuppressWarnings("rawtypes")
+            ISerializerDeserializer currentDeserializer = deserializer;
+            if (itemType.getTypeTag() == ATypeTag.ANY && typeTag != ATypeTag.ANY) {
+                currentItemType = BuiltinType.builtinTypeFromString(typeTag.name().toLowerCase());
+                currentDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+                        .getNonTaggedSerializerDeserializer(currentItemType);
+            }
+
             in.readInt(); // list size
             int numberOfitems;
             numberOfitems = in.readInt();
@@ -94,13 +103,12 @@
                         in.readInt();
                 }
                 for (int i = 0; i < numberOfitems; i++) {
-                    items.add((IAObject) nontaggedSerDes.deserialize(in));
+                    items.add((IAObject) currentDeserializer.deserialize(in));
                 }
             }
-            AUnorderedListType type = new AUnorderedListType(itemType, "orderedlist");
+            AUnorderedListType type = new AUnorderedListType(currentItemType, "unorderedlist");
             return new AUnorderedList(type, items);
-
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
@@ -115,7 +123,7 @@
         IACursor cursor = instance.getCursor();
         while (cursor.next()) {
             itemValue.reset();
-            taggedSerDes.serialize(cursor.get(), itemValue.getDataOutput());
+            serializer.serialize(cursor.get(), itemValue.getDataOutput());
             listBuilder.addItem(itemValue);
         }
         listBuilder.write(out, false);
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
index b5a6408..eb6942f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableYearMonthDuration;
 import org.apache.asterix.om.base.AYearMonthDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,10 +32,8 @@
 
     public static final AYearMonthDurationSerializerDeserializer INSTANCE = new AYearMonthDurationSerializerDeserializer();
 
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
-    private static final AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
+    private AYearMonthDurationSerializerDeserializer() {
+    }
 
     @Override
     public AYearMonthDuration deserialize(DataInput in) throws HyracksDataException {
@@ -57,16 +50,6 @@
             out.writeInt(instance.getMonths());
         } catch (IOException e) {
             throw new HyracksDataException();
-        }
-    }
-
-    public void parse(String durationString, DataOutput out) throws HyracksDataException {
-        try {
-            ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
-                    ADurationParseOption.All);
-            yearMonthDurationSerde.serialize(aYearMonthDuration, out);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
         }
     }
 
diff --git a/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java b/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
index 910f633..9ca94f1 100644
--- a/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
+++ b/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
@@ -84,7 +84,7 @@
         switch (aqlType.getTypeTag()) {
             case ANY:
             case UNION: { // we could do smth better for nullable fields
-                return new AObjectSerializerDeserializer();
+                return AObjectSerializerDeserializer.INSTANCE;
             }
             default: {
                 return addTag(getNonTaggedSerializerDeserializer(aqlType), aqlType.getTypeTag());
@@ -132,10 +132,10 @@
                 return ANullSerializerDeserializer.INSTANCE;
             }
             case STRING: {
-                return new AStringSerializerDeserializer();
+                return AStringSerializerDeserializer.INSTANCE;
             }
             case BINARY: {
-                return new ABinarySerializerDeserializer();
+                return ABinarySerializerDeserializer.INSTANCE;
             }
             case TIME: {
                 return ATimeSerializerDeserializer.INSTANCE;
@@ -183,8 +183,8 @@
                 return ShortSerializerDeserializer.INSTANCE;
             }
             default: {
-                throw new NotImplementedException("No serializer/deserializer implemented for type "
-                        + aqlType.getTypeTag() + " .");
+                throw new NotImplementedException(
+                        "No serializer/deserializer implemented for type " + aqlType.getTypeTag() + " .");
             }
         }
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java b/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
index 35be4ab..bb9a7bf 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
@@ -18,11 +18,11 @@
  */
 package org.apache.asterix.om.base;
 
-import java.util.ArrayList;
+import java.util.List;
 
 public class ACollectionCursor implements IACursor {
 
-    private ArrayList<IAObject> values;
+    private List<IAObject> values;
     private int pos;
     private int size;
     private IAObject object = null;
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java b/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
index 9130c64..94416b3 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
@@ -21,19 +21,18 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.visitors.IOMVisitor;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 public class AOrderedList implements IACollection {
 
-    protected ArrayList<IAObject> values;
+    protected List<IAObject> values;
     protected AOrderedListType type;
 
     public AOrderedList(AOrderedListType type) {
@@ -41,7 +40,7 @@
         this.type = type;
     }
 
-    public AOrderedList(AOrderedListType type, ArrayList<IAObject> sequence) {
+    public AOrderedList(AOrderedListType type, List<IAObject> sequence) {
         values = sequence;
         this.type = type;
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java b/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
index f8c02b0..8f19b7e 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
@@ -19,19 +19,19 @@
 package org.apache.asterix.om.base;
 
 import java.util.ArrayList;
-
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.visitors.IOMVisitor;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 public class AUnorderedList implements IACollection {
 
-    protected ArrayList<IAObject> values;
+    protected List<IAObject> values;
     protected AUnorderedListType type;
 
     public AUnorderedList(AUnorderedListType type) {
@@ -39,7 +39,7 @@
         this.type = type;
     }
 
-    public AUnorderedList(AUnorderedListType type, ArrayList<IAObject> sequence) {
+    public AUnorderedList(AUnorderedListType type, List<IAObject> sequence) {
         values = sequence;
         this.type = type;
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
index de4b5d3..0efc35f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
@@ -892,17 +892,4 @@
         throw new AsterixException("No string translation for type: " + str + " .");
     }
 
-    public static ATypeTag builtinTypeTagFromString(String str) throws AsterixException {
-        if (str.equals("int32")) {
-            return ATypeTag.INT32;
-        } else if (str.equals("string")) {
-            return ATypeTag.STRING;
-        } else if (str.equals("double")) {
-            return ATypeTag.DOUBLE;
-        } else if (str.equals("float")) {
-            return ATypeTag.FLOAT;
-        }
-        throw new AsterixException("No string translation for type: " + str + " .");
-    }
-
 }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
index e7e90ad..e3c793a 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
@@ -42,6 +42,7 @@
 
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new IsNullDescriptor();
         }
@@ -60,7 +61,7 @@
                     private DataOutput out = output.getDataOutput();
                     private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
                     private ICopyEvaluator eval = args[0].createEvaluator(argOut);
-                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+                    private final AObjectSerializerDeserializer aObjSerDer = AObjectSerializerDeserializer.INSTANCE;
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
index 47ae0d6..40fac14 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
@@ -42,6 +42,7 @@
 
     private final static byte SER_SYSTEM_NULL_TYPE_TAG = ATypeTag.SYSTEM_NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new IsSystemNullDescriptor();
         }
@@ -60,13 +61,14 @@
                     private DataOutput out = output.getDataOutput();
                     private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
                     private ICopyEvaluator eval = args[0].createEvaluator(argOut);
-                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+                    private final AObjectSerializerDeserializer aObjSerDer = AObjectSerializerDeserializer.INSTANCE;
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
                         argOut.reset();
                         eval.evaluate(tuple);
-                        boolean isSystemNull = argOut.getByteArray()[argOut.getStartOffset()] == SER_SYSTEM_NULL_TYPE_TAG;
+                        boolean isSystemNull = argOut.getByteArray()[argOut
+                                .getStartOffset()] == SER_SYSTEM_NULL_TYPE_TAG;
                         ABoolean res = isSystemNull ? ABoolean.TRUE : ABoolean.FALSE;
                         try {
                             aObjSerDer.serialize(res, out);
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index aebf186..bec1f79 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import org.apache.asterix.fuzzyjoin.IntArray;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -39,7 +40,6 @@
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.common.DoubleArray;
 import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
-import org.apache.asterix.fuzzyjoin.IntArray;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -56,6 +56,7 @@
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SpatialIntersectDescriptor();
         }
@@ -84,12 +85,12 @@
                     private final DoubleArray trianglesY0 = new DoubleArray();
                     private final DoubleArray trianglesX1 = new DoubleArray();
                     private final DoubleArray trianglesY1 = new DoubleArray();
-                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+                    private final AObjectSerializerDeserializer aBooleanSerDer = AObjectSerializerDeserializer.INSTANCE;
 
                     private boolean pointOnLine(double pX, double pY, double startX, double startY, double endX,
                             double endY) throws HyracksDataException {
-                        double crossProduct = SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY, endX
-                                - startX);
+                        double crossProduct = SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY,
+                                endX - startX);
                         if (Math.abs(crossProduct) > SpatialUtils.doubleEpsilon()) { // crossProduct != 0
                             return false;
                         }
@@ -257,7 +258,8 @@
                         return false;
                     }
 
-                    private boolean lineRectangleIntersection(byte[] bytes0, byte[] bytes1) throws HyracksDataException {
+                    private boolean lineRectangleIntersection(byte[] bytes0, byte[] bytes1)
+                            throws HyracksDataException {
                         double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
                                 ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
                         double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
@@ -434,7 +436,7 @@
 
                     private boolean triangleTriangleIntersection(DoubleArray trianglesX0, DoubleArray trianglesY0,
                             int triangleId0, DoubleArray trianglesX1, DoubleArray trianglesY1, int triangleId1)
-                            throws HyracksDataException { // separating axis theorem
+                                    throws HyracksDataException { // separating axis theorem
 
                         for (int side = 0; side < 3; side++) {
                             spatialUtils.findNormals(trianglesX0, trianglesY0, triangleId0, side);
@@ -763,15 +765,21 @@
                                 case POINT:
                                     switch (tag1) {
                                         case POINT:
-                                            if (ascDoubleComp.compare(outInput0.getByteArray(),
-                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.X), 8,
-                                                    outInput1.getByteArray(),
-                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.X), 8) == 0) {
-                                                if (ascDoubleComp.compare(outInput0.getByteArray(),
-                                                        APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y),
-                                                        8, outInput1.getByteArray(),
-                                                        APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y),
-                                                        8) == 0) {
+                                            if (ascDoubleComp
+                                                    .compare(outInput0.getByteArray(),
+                                                            APointSerializerDeserializer.getCoordinateOffset(
+                                                                    Coordinate.X),
+                                                            8, outInput1.getByteArray(), APointSerializerDeserializer
+                                                                    .getCoordinateOffset(Coordinate.X),
+                                                            8) == 0) {
+                                                if (ascDoubleComp
+                                                        .compare(outInput0.getByteArray(),
+                                                                APointSerializerDeserializer.getCoordinateOffset(
+                                                                        Coordinate.Y),
+                                                                8, outInput1.getByteArray(),
+                                                                APointSerializerDeserializer
+                                                                        .getCoordinateOffset(Coordinate.Y),
+                                                                8) == 0) {
                                                     res = true;
                                                 }
                                             }
@@ -784,18 +792,18 @@
                                                     outInput0.getByteArray(),
                                                     APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
 
-                                            double startX = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double startX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
 
                                             res = pointOnLine(pX, pY, startX, startY, endX, endY);
                                             break;
@@ -812,11 +820,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case LINE:
@@ -829,49 +835,49 @@
                                                     outInput1.getByteArray(),
                                                     APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
 
-                                            double startX = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double startX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
 
                                             res = pointOnLine(pX, pY, startX, startY, endX, endY);
                                             break;
                                         case LINE:
-                                            double startX1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double startX1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
 
-                                            double startX2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
-                                            res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2,
-                                                    startY2, endX2, endY2);
+                                            double startX2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
+                                            res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
+                                                    endX2, endY2);
                                             break;
                                         case POLYGON:
                                             res = linePolygonIntersection(outInput0.getByteArray(),
@@ -889,11 +895,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case POLYGON:
@@ -991,11 +995,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case CIRCLE:
@@ -1023,11 +1025,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case RECTANGLE:
@@ -1078,25 +1078,22 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case NULL:
                                     res = false;
                                     break;
                                 default:
-                                    throw new NotImplementedException(
-                                            AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                    + ": does not support the type: " + tag1
-                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                    throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                            .getName() + ": does not support the type: " + tag1
+                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                             }
 
                             ABoolean aResult = res ? (ABoolean.TRUE) : (ABoolean.FALSE);
-                            aObjSerDer.serialize(aResult, out);
+                            aBooleanSerDer.serialize(aResult, out);
                         } catch (HyracksDataException hde) {
                             throw new AlgebricksException(hde);
                         }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index cdecce5..f6d5ec5 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
@@ -50,6 +49,7 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 //The record merge evaluator is used to combine two records with no matching fieldnames
 //If both records have the same fieldname for a non-record field anywhere in the schema, the merge will fail
@@ -88,11 +88,6 @@
 
             private static final long serialVersionUID = 1L;
 
-            @SuppressWarnings("unchecked")
-            private final ISerializerDeserializer<ANull> nullSerDe = AqlSerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(BuiltinType.ANULL);
-            private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
-
             @Override
             public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
                 final ARecordType recType;
@@ -114,6 +109,11 @@
                 final ICopyEvaluator eval1 = args[1].createEvaluator(abvs1);
 
                 return new ICopyEvaluator() {
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ANull> nullSerDe = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+                    final UTF8StringReader reader = new UTF8StringReader();
+
                     private final Stack<RecordBuilder> rbStack = new Stack<RecordBuilder>();
                     private final ArrayBackedValueStorage tabvs = new ArrayBackedValueStorage();
                     private final ByteArrayAccessibleOutputStream nameOutputStream = new ByteArrayAccessibleOutputStream();
@@ -220,7 +220,7 @@
                         nameOutputStream.write(fieldNamePointable.getByteArray(),
                                 fieldNamePointable.getStartOffset() + 1, fieldNamePointable.getLength());
                         namedis.reset();
-                        String fieldName = aStringSerDer.deserialize(namedis).getStringValue();
+                        String fieldName = reader.readUTF(namedis);
 
                         //Add the merged field
                         if (combinedType.isClosedField(fieldName)) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
index 6e51311..6e4c175 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
@@ -32,7 +32,6 @@
 import org.apache.asterix.builders.RecordBuilderFactory;
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
@@ -302,8 +301,7 @@
                 if (checkType(ATypeTag.INTERVAL, objectType)) {
                     if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                         if (admLexer.next() == AdmLexer.TOKEN_STRING_LITERAL) {
-                            AIntervalSerializerDeserializer.parseDate(admLexer.getLastTokenImage(), out);
-
+                            parseDateInterval(admLexer.getLastTokenImage(), out);
                             if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_CLOSE) {
                                 break;
                             }
@@ -316,8 +314,7 @@
                 if (checkType(ATypeTag.INTERVAL, objectType)) {
                     if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                         if (admLexer.next() == AdmLexer.TOKEN_STRING_LITERAL) {
-                            AIntervalSerializerDeserializer.parseTime(admLexer.getLastTokenImage(), out);
-
+                            parseTimeInterval(admLexer.getLastTokenImage(), out);
                             if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_CLOSE) {
                                 break;
                             }
@@ -330,8 +327,7 @@
                 if (checkType(ATypeTag.INTERVAL, objectType)) {
                     if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                         if (admLexer.next() == AdmLexer.TOKEN_STRING_LITERAL) {
-                            AIntervalSerializerDeserializer.parseDatetime(admLexer.getLastTokenImage(), out);
-
+                            parseDateTimeInterval(admLexer.getLastTokenImage(), out);
                             if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_CLOSE) {
                                 break;
                             }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
index 4ce8ee8..2dd1e40 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AInterval;
 import org.apache.asterix.om.base.ALine;
 import org.apache.asterix.om.base.AMutableBinary;
 import org.apache.asterix.om.base.AMutableCircle;
@@ -47,6 +48,7 @@
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.om.base.AMutableInterval;
 import org.apache.asterix.om.base.AMutableLine;
 import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.AMutablePoint3D;
@@ -68,6 +70,7 @@
 import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import org.apache.asterix.om.base.temporal.ATimeParserFactory;
 import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -104,6 +107,7 @@
     protected AMutablePoint aPoint2 = new AMutablePoint(0, 0);
     protected AMutableLine aLine = new AMutableLine(null, null);
     protected AMutableDate aDate = new AMutableDate(0);
+    protected final AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
 
     // Serializers
     @SuppressWarnings("unchecked")
@@ -182,6 +186,9 @@
     @SuppressWarnings("unchecked")
     protected final static ISerializerDeserializer<ALine> lineSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ALINE);
+    @SuppressWarnings("unchecked")
+    private static final ISerializerDeserializer<AInterval> intervalSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINTERVAL);
 
     protected String filename;
 
@@ -352,4 +359,163 @@
         aBinary.setValue(base64Parser.getByteArray(), 0, base64Parser.getLength());
         binarySerde.serialize(aBinary, out);
     }
+
+    protected void parseDateTimeInterval(String interval, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMsStart = 0;
+        long chrononTimeInMsEnd = 0;
+        try {
+            // the starting point for parsing (so for the accessor)
+            int startOffset = 0;
+            int endOffset, timeSeperatorOffsetInDatetimeString;
+
+            // Get the index for the comma
+            int commaIndex = interval.indexOf(',');
+            if (commaIndex < 1) {
+                throw new AlgebricksException("comma is missing for a string of interval");
+            }
+
+            endOffset = commaIndex - 1;
+            timeSeperatorOffsetInDatetimeString = interval.indexOf('T');
+
+            if (timeSeperatorOffsetInDatetimeString < 0) {
+                throw new AlgebricksException(
+                        "This can not be an instance of interval: missing T for a datetime value.");
+            }
+
+            chrononTimeInMsStart = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
+
+            chrononTimeInMsStart += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
+
+            // Interval End
+            startOffset = commaIndex + 1;
+            endOffset = interval.length() - 1;
+
+            timeSeperatorOffsetInDatetimeString = interval.indexOf('T', startOffset);
+
+            if (timeSeperatorOffsetInDatetimeString < 0) {
+                throw new AlgebricksException(
+                        "This can not be an instance of interval: missing T for a datetime value.");
+            }
+
+            chrononTimeInMsEnd = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
+
+            chrononTimeInMsEnd += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+
+        try {
+            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.DATETIME.serialize());
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e);
+        }
+
+        intervalSerde.serialize(aInterval, out);
+    }
+
+    protected void parseTimeInterval(String interval, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMsStart = 0;
+        long chrononTimeInMsEnd = 0;
+        try {
+            int startOffset = 0;
+            int endOffset;
+
+            // Get the index for the comma
+            int commaIndex = interval.indexOf(',');
+            if (commaIndex < 0) {
+                throw new AlgebricksException("comma is missing for a string of interval");
+            }
+
+            endOffset = commaIndex - 1;
+            // Interval Start
+            chrononTimeInMsStart = parseTimePart(interval, startOffset, endOffset);
+
+            if (chrononTimeInMsStart < 0) {
+                chrononTimeInMsStart += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+
+            // Interval End
+            startOffset = commaIndex + 1;
+            endOffset = interval.length() - 1;
+
+            chrononTimeInMsEnd = parseTimePart(interval, startOffset, endOffset);
+            if (chrononTimeInMsEnd < 0) {
+                chrononTimeInMsEnd += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+
+        try {
+            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.TIME.serialize());
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e);
+        }
+        intervalSerde.serialize(aInterval, out);
+    }
+
+    public void parseDateInterval(String interval, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMsStart = 0;
+        long chrononTimeInMsEnd = 0;
+        try {
+            // the starting point for parsing (so for the accessor)
+            int startOffset = 0;
+            int endOffset;
+
+            // Get the index for the comma
+            int commaIndex = interval.indexOf(',');
+            if (commaIndex < 1) {
+                throw new AlgebricksException("comma is missing for a string of interval");
+            }
+
+            endOffset = commaIndex - 1;
+            chrononTimeInMsStart = parseDatePart(interval, startOffset, endOffset);
+
+            // Interval End
+            startOffset = commaIndex + 1;
+            endOffset = interval.length() - 1;
+
+            chrononTimeInMsEnd = parseDatePart(interval, startOffset, endOffset);
+
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+
+        try {
+            aInterval.setValue((chrononTimeInMsStart / GregorianCalendarSystem.CHRONON_OF_DAY),
+                    (chrononTimeInMsEnd / GregorianCalendarSystem.CHRONON_OF_DAY), ATypeTag.DATE.serialize());
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e);
+        }
+        intervalSerde.serialize(aInterval, out);
+    }
+
+    private long parseDatePart(String interval, int startOffset, int endOffset)
+            throws AlgebricksException, HyracksDataException {
+
+        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
+            endOffset--;
+        }
+
+        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
+            startOffset++;
+        }
+
+        return ADateParserFactory.parseDatePart(interval, startOffset, endOffset - startOffset + 1);
+    }
+
+    private int parseTimePart(String interval, int startOffset, int endOffset)
+            throws AlgebricksException, HyracksDataException {
+
+        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
+            endOffset--;
+        }
+
+        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
+            startOffset++;
+        }
+
+        return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
+    }
 }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#2).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
63 files changed, 979 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 8: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/471/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/470/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/464/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#4).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/pom.xml
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
A asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
65 files changed, 1,111 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/557/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/463/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has submitted this change and it was merged.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Reviewed-on: https://asterix-gerrit.ics.uci.edu/522
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Jianfeng Jia <ji...@gmail.com>
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/pom.xml
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
A asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
65 files changed, 1,116 insertions(+), 1,170 deletions(-)

Approvals:
  Jianfeng Jia: Looks good to me, approved
  Jenkins: Verified



diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index f5ef424..bb5f659 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -50,7 +50,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.commons.lang3.ArrayUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -505,9 +504,6 @@
                     }
                 }
 
-            } catch (AsterixException e) {
-                throw new AlgebricksException(
-                        "Cannot enforce typed fields " + StringUtils.join(index.getKeyFieldNames()), e);
             } catch (IOException e) {
                 throw new AsterixException(e);
             }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
index bff41ab..2d405f7 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
@@ -30,10 +30,10 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.expression.OrderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.RecordTypeDefinition;
+import org.apache.asterix.lang.common.expression.RecordTypeDefinition.RecordKind;
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
-import org.apache.asterix.lang.common.expression.RecordTypeDefinition.RecordKind;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -49,7 +49,6 @@
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeSignature;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class TypeTranslator {
 
@@ -323,14 +322,8 @@
             fldNames[i++] = s;
         }
         boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
-        ARecordType recType;
-        try {
-            recType = new ARecordType(typeSignature == null ? null : typeSignature.getName(), fldNames, fldTypes,
-                    isOpen);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
-
+        ARecordType recType = new ARecordType(typeSignature == null ? null : typeSignature.getName(), fldNames,
+                fldTypes, isOpen);
         List<IRecordFieldDataGen> fieldDataGen = rtd.getFieldDataGen();
         if (fieldDataGen.size() == n) {
             IRecordFieldDataGen[] rfdg = new IRecordFieldDataGen[n];
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
index 677e913..29ce728 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
@@ -20,6 +20,7 @@
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
+import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.List;
 
@@ -41,7 +42,6 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
 import org.apache.asterix.external.library.TypeInfo;
 import org.apache.asterix.external.library.java.JObjects.JBoolean;
@@ -86,6 +86,7 @@
 import org.apache.asterix.om.util.container.IObjectPool;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 public class JObjectAccessors {
 
@@ -226,7 +227,7 @@
     }
 
     public static class JStringAccessor implements IJObjectAccessor {
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final UTF8StringReader reader = new UTF8StringReader();
 
         @Override
         public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
@@ -236,8 +237,11 @@
             int l = pointable.getLength();
 
             String v = null;
-            v = aStringSerDer.deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)))
-                    .getStringValue();
+            try {
+                v = reader.readUTF(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
             JObjectUtil.getNormalizedString(v);
 
             IJObject jObject = objectPool.allocate(BuiltinType.ASTRING);
@@ -444,7 +448,7 @@
         private final JRecord jRecord;
         private final IJObject[] jObjects;
         private final LinkedHashMap<String, IJObject> openFields;
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final UTF8StringReader reader = new UTF8StringReader();
 
         public JRecordAccessor(ARecordType recordType, IObjectPool<IJObject, IAType> objectPool) {
             this.typeInfo = new TypeInfo(objectPool, null, null);
@@ -502,9 +506,7 @@
                         byte[] b = fieldName.getByteArray();
                         int s = fieldName.getStartOffset();
                         int l = fieldName.getLength();
-                        String v = aStringSerDer
-                                .deserialize(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)))
-                                .getStringValue();
+                        String v = reader.readUTF(new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1)));
                         openFields.put(v, fieldObject);
                     }
                     index++;
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
index b5458e2..146a247 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
@@ -58,7 +58,6 @@
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.util.container.IObjectPool;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class JObjectUtil {
 
@@ -392,7 +391,7 @@
                         dis.readInt();
                     }
                     for (int i = 0; i < numberOfOpenFields; i++) {
-                        fieldNames[i] = new AStringSerializerDeserializer().deserialize(dis).getStringValue();
+                        fieldNames[i] = AStringSerializerDeserializer.INSTANCE.deserialize(dis).getStringValue();
                         ATypeTag openFieldTypeTag = SerializerDeserializerUtil.deserializeTag(dis);
                         openFields[i] = getJType(openFieldTypeTag, null, dis, objectPool);
                         fieldTypes[i] = openFields[i].getIAObject().getType();
@@ -443,10 +442,6 @@
             fieldNames[i] = recType2.getFieldNames()[j];
             fieldTypes[i] = recType2.getFieldTypes()[j];
         }
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 }
diff --git a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
index 406d242..e9a63b2 100644
--- a/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
+++ b/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
@@ -366,7 +366,7 @@
 
     public static final class JString extends JObject {
 
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final AStringSerializerDeserializer aStringSerDer = AStringSerializerDeserializer.INSTANCE;
 
         public JString(String v) {
             super(new AMutableString(v));
@@ -978,7 +978,7 @@
         private ARecordType recordType;
         private IJObject[] fields;
         private Map<String, IJObject> openFields;
-        private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
+        private final AStringSerializerDeserializer aStringSerDer = AStringSerializerDeserializer.INSTANCE;
 
         public JRecord(ARecordType recordType, IJObject[] fields) {
             this.recordType = recordType;
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
index 70f05bd..39f8271 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
@@ -28,7 +28,6 @@
 import java.util.concurrent.Executors;
 
 import org.apache.asterix.common.feeds.api.IFeedAdapter;
-import org.apache.asterix.common.feeds.api.IFeedAdapter.DataExchangeMode;
 import org.apache.asterix.external.dataset.adapter.StreamBasedAdapter;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.hyracks.api.comm.IFrameWriter;
diff --git a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
index 17e14c5..0100ea1 100644
--- a/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
+++ b/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
@@ -20,28 +20,19 @@
 
 import java.util.Map;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.feeds.api.IDatasourceAdapter;
 import org.apache.asterix.common.feeds.api.IIntakeProgressTracker;
-import org.apache.asterix.metadata.entities.DatasourceAdapter.AdapterType;
-import org.apache.asterix.metadata.external.IAdapterFactory.SupportedOperation;
 import org.apache.asterix.metadata.feeds.IFeedAdapterFactory;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory;
 import org.apache.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
 
 public class TestTypedAdapterFactory implements IFeedAdapterFactory {
 
-    /**
-     * 
-     */
     private static final long serialVersionUID = 1L;
 
     public static final String NAME = "test_typed_adapter";
@@ -55,18 +46,6 @@
     @Override
     public SupportedOperation getSupportedOperations() {
         return SupportedOperation.READ;
-    }
-
-    private static ARecordType initOutputType() {
-        String[] fieldNames = new String[] { "id", "message-text" };
-        IAType[] fieldTypes = new IAType[] { BuiltinType.AINT64, BuiltinType.ASTRING };
-        ARecordType outputType = null;
-        try {
-            outputType = new ARecordType("TestTypedAdapterOutputType", fieldNames, fieldTypes, false);
-        } catch (AsterixException | HyracksDataException exception) {
-            throw new IllegalStateException("Unable to create output type for adapter " + NAME);
-        }
-        return outputType;
     }
 
     @Override
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index cb409d5..602cb4c 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -27,7 +27,6 @@
 
 import org.apache.asterix.common.config.AsterixMetadataProperties;
 import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
@@ -48,7 +47,6 @@
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * Provides access to Asterix metadata via remote methods to the metadata node.
@@ -428,18 +426,14 @@
         datatype = cache.getDatatype(dataverseName, datatypeName);
         if (datatype != null) {
             // Datatype is already in the cache, don't add it again.
-            try {
-                //create a new Datatype object with a new ARecordType object in order to avoid
-                //concurrent access to UTF8StringPointable comparator in ARecordType object.
-                //see issue 510
-                ARecordType aRecType = (ARecordType) datatype.getDatatype();
-                return new Datatype(
-                        datatype.getDataverseName(), datatype.getDatatypeName(), new ARecordType(aRecType.getTypeName(),
-                                aRecType.getFieldNames(), aRecType.getFieldTypes(), aRecType.isOpen()),
-                        datatype.getIsAnonymous());
-            } catch (AsterixException | HyracksDataException e) {
-                throw new MetadataException(e);
-            }
+            //create a new Datatype object with a new ARecordType object in order to avoid
+            //concurrent access to UTF8StringPointable comparator in ARecordType object.
+            //see issue 510
+            ARecordType aRecType = (ARecordType) datatype.getDatatype();
+            return new Datatype(
+                    datatype.getDataverseName(), datatype.getDatatypeName(), new ARecordType(aRecType.getTypeName(),
+                            aRecType.getFieldNames(), aRecType.getFieldTypes(), aRecType.isOpen()),
+                    datatype.getIsAnonymous());
         }
         try {
             datatype = metadataNode.getDatatype(ctx.getJobId(), dataverseName, datatypeName);
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index df48269..fe42a2c 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -134,11 +134,7 @@
         AUnorderedListType listPropertiesType = new AUnorderedListType(POLICY_PARAMS_RECORDTYPE, null);
         String[] fieldNames = { "DataverseName", "PolicyName", "Description", "Properties" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, listPropertiesType };
-        try {
-            return new ARecordType("FeedPolicyRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("FeedPolicyRecordType", fieldNames, fieldTypes, true);
     }
 
     // Helper constants for accessing fields in an ARecord of type
@@ -149,14 +145,10 @@
     public static final int DATAVERSE_ARECORD_PENDINGOP_FIELD_INDEX = 3;
 
     private static final ARecordType createDataverseRecordType() throws AsterixException {
-        try {
-            return new ARecordType("DataverseRecordType",
-                    new String[] { "DataverseName", "DataFormat", "Timestamp", "PendingOp" },
-                    new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 },
-                    true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("DataverseRecordType",
+                new String[] { "DataverseName", "DataFormat", "Timestamp", "PendingOp" },
+                new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 },
+                true);
     }
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -168,11 +160,7 @@
     private static final ARecordType createPropertiesRecordType() throws AsterixException {
         String[] fieldNames = { "Name", "Value" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING };
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -189,11 +177,7 @@
         String[] fieldNames = { "FileStructure", "PartitioningStrategy", "PartitioningKey", "PrimaryKey",
                 "Autogenerated" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, ololType, ololType, BuiltinType.ABOOLEAN };
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -208,11 +192,7 @@
         String[] fieldNames = { "DatasourceAdapter", "Properties", "LastRefreshTime", "TransactionState", };
         IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType, BuiltinType.ADATETIME,
                 BuiltinType.AINT32 };
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
     public static final int COMPACTION_POLICY_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
@@ -222,11 +202,7 @@
     private static ARecordType createCompactionPolicyRecordType() throws AsterixException {
         String[] fieldNames = { "DataverseName", "CompactionPolicy", "Classname" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
-        try {
-            return new ARecordType("CompactionPolicyRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("CompactionPolicyRecordType", fieldNames, fieldTypes, true);
     }
 
     public static final int FEED_DETAILS_ARECORD_FILESTRUCTURE_FIELD_INDEX = 0;
@@ -256,12 +232,7 @@
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, orderedListType, orderedListType,
                 BuiltinType.ASTRING, BuiltinType.ASTRING, orderedListOfPropertiesType, feedFunctionUnion,
                 BuiltinType.ASTRING, BuiltinType.ASTRING, compactionPolicyPropertyListType };
-
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
     public static final int DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
@@ -294,11 +265,7 @@
                 BuiltinType.ASTRING, BuiltinType.ASTRING, compactionPolicyPropertyListType, internalRecordUnion,
                 externalRecordUnion, unorderedListOfHintsType, BuiltinType.ASTRING, BuiltinType.AINT32,
                 BuiltinType.AINT32 };
-        try {
-            return new ARecordType("DatasetRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("DatasetRecordType", fieldNames, fieldTypes, true);
     }
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -310,11 +277,7 @@
     private static final ARecordType createFieldRecordType() throws AsterixException {
         String[] fieldNames = { "FieldName", "FieldType", "IsNullable" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ABOOLEAN };
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -326,11 +289,7 @@
         AOrderedListType olType = new AOrderedListType(FIELD_RECORDTYPE, null);
         String[] fieldNames = { "IsOpen", "Fields" };
         IAType[] fieldTypes = { BuiltinType.ABOOLEAN, olType };
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -348,11 +307,7 @@
 
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ABOOLEAN, recordUnion, collectionUnion,
                 collectionUnion };
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of type
@@ -366,11 +321,7 @@
         String[] fieldNames = { "DataverseName", "DatatypeName", "Derived", "Timestamp" };
         AUnionType recordUnion = AUnionType.createNullableType(DERIVEDTYPE_RECORDTYPE);
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, recordUnion, BuiltinType.ASTRING };
-        try {
-            return new ARecordType("DatatypeRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("DatatypeRecordType", fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of type
@@ -391,11 +342,7 @@
                 "Timestamp", "PendingOp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
                 ololType, BuiltinType.ABOOLEAN, BuiltinType.ASTRING, BuiltinType.AINT32 };
-        try {
-            return new ARecordType("IndexRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("IndexRecordType", fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of type
@@ -407,11 +354,7 @@
     private static final ARecordType createNodeRecordType() throws AsterixException {
         String[] fieldNames = { "NodeName", "NumberOfCores", "WorkingMemorySize" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.AINT64, BuiltinType.AINT64 };
-        try {
-            return new ARecordType("NodeRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("NodeRecordType", fieldNames, fieldTypes, true);
     };
 
     // Helper constants for accessing fields in an ARecord of type
@@ -424,11 +367,7 @@
         AUnorderedListType ulType = new AUnorderedListType(BuiltinType.ASTRING, null);
         String[] fieldNames = { "GroupName", "NodeNames", "Timestamp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, ulType, BuiltinType.ASTRING };
-        try {
-            return new ARecordType("NodeGroupRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("NodeGroupRecordType", fieldNames, fieldTypes, true);
     };
 
     private static IAType createFunctionParamsRecordType() {
@@ -447,17 +386,12 @@
     public static final int FUNCTION_ARECORD_FUNCTION_KIND_FIELD_INDEX = 7;
 
     private static final ARecordType createFunctionRecordType() throws AsterixException {
-
         String[] fieldNames = { "DataverseName", "Name", "Arity", "Params", "ReturnType", "Definition", "Language",
                 "Kind" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
                 createFunctionParamsRecordType(), BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
                 BuiltinType.ASTRING };
-        try {
-            return new ARecordType("FunctionRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("FunctionRecordType", fieldNames, fieldTypes, true);
     }
 
     public static final int DATASOURCE_ADAPTER_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
@@ -470,11 +404,7 @@
         String[] fieldNames = { "DataverseName", "Name", "Classname", "Type", "Timestamp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
                 BuiltinType.ASTRING };
-        try {
-            return new ARecordType("DatasourceAdapterRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("DatasourceAdapterRecordType", fieldNames, fieldTypes, true);
     }
 
     // Helper constants for accessing fields in an ARecord of type
@@ -543,11 +473,7 @@
     private static ARecordType createLibraryRecordType() throws AsterixException {
         String[] fieldNames = { "DataverseName", "Name", "Timestamp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
-        try {
-            return new ARecordType("LibraryRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("LibraryRecordType", fieldNames, fieldTypes, true);
     }
 
     public static final int EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
@@ -563,10 +489,6 @@
                 "PendingOp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.ASTRING,
                 BuiltinType.AINT64, BuiltinType.ADATETIME, BuiltinType.AINT32 };
-        try {
-            return new ARecordType("ExternalFileRecordType", fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
+        return new ARecordType("ExternalFileRecordType", fieldNames, fieldTypes, true);
     }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index 9a33f93..8948f43 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -95,7 +95,7 @@
         int recordLength = frameTuple.getFieldLength(DATATYPE_PAYLOAD_TUPLE_FIELD_INDEX);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        ARecord datatypeRecord = (ARecord) recordSerDes.deserialize(in);
+        ARecord datatypeRecord = recordSerDes.deserialize(in);
         return createDataTypeFromARecord(datatypeRecord);
     }
 
@@ -109,8 +109,9 @@
             // Derived Type
             ARecord derivedTypeRecord = (ARecord) datatypeRecord
                     .getValueByPos(MetadataRecordTypes.DATATYPE_ARECORD_DERIVED_FIELD_INDEX);
-            DerivedTypeTag tag = DerivedTypeTag.valueOf(((AString) derivedTypeRecord
-                    .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_TAG_FIELD_INDEX)).getStringValue());
+            DerivedTypeTag tag = DerivedTypeTag.valueOf(
+                    ((AString) derivedTypeRecord.getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_TAG_FIELD_INDEX))
+                            .getStringValue());
             boolean isAnonymous = ((ABoolean) derivedTypeRecord
                     .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_ISANONYMOUS_FIELD_INDEX)).getBoolean();
             switch (tag) {
@@ -119,7 +120,7 @@
                             .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_RECORD_FIELD_INDEX);
                     boolean isOpen = ((ABoolean) recordType
                             .getValueByPos(MetadataRecordTypes.RECORDTYPE_ARECORD_ISOPEN_FIELD_INDEX)).getBoolean()
-                            .booleanValue();
+                                    .booleanValue();
                     int numberOfFields = ((AOrderedList) recordType
                             .getValueByPos(MetadataRecordTypes.RECORDTYPE_ARECORD_FIELDS_FIELD_INDEX)).size();
                     IACursor cursor = ((AOrderedList) recordType
@@ -132,39 +133,37 @@
                         ARecord field = (ARecord) cursor.get();
                         fieldNames[fieldId] = ((AString) field
                                 .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDNAME_FIELD_INDEX))
-                                .getStringValue();
+                                        .getStringValue();
                         fieldTypeName = ((AString) field
                                 .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDTYPE_FIELD_INDEX))
-                                .getStringValue();
+                                        .getStringValue();
                         boolean isNullable = ((ABoolean) field
                                 .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_ISNULLABLE_FIELD_INDEX)).getBoolean()
-                                .booleanValue();
+                                        .booleanValue();
                         fieldTypes[fieldId] = AsterixBuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId,
                                 dataverseName, fieldTypeName, isNullable);
                         fieldId++;
                     }
-                    try {
-                        return new Datatype(dataverseName, datatypeName, new ARecordType(datatypeName, fieldNames,
-                                fieldTypes, isOpen), isAnonymous);
-                    } catch (AsterixException | HyracksDataException e) {
-                        throw new MetadataException(e);
-                    }
+                    return new Datatype(dataverseName, datatypeName,
+                            new ARecordType(datatypeName, fieldNames, fieldTypes, isOpen), isAnonymous);
                 }
                 case UNORDEREDLIST: {
                     String unorderedlistTypeName = ((AString) derivedTypeRecord
                             .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_UNORDEREDLIST_FIELD_INDEX))
-                            .getStringValue();
-                    return new Datatype(dataverseName, datatypeName, new AUnorderedListType(
-                            AsterixBuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId, dataverseName,
-                                    unorderedlistTypeName, false), datatypeName), isAnonymous);
+                                    .getStringValue();
+                    return new Datatype(dataverseName, datatypeName,
+                            new AUnorderedListType(AsterixBuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId,
+                                    dataverseName, unorderedlistTypeName, false), datatypeName),
+                            isAnonymous);
                 }
                 case ORDEREDLIST: {
                     String orderedlistTypeName = ((AString) derivedTypeRecord
                             .getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_ORDEREDLIST_FIELD_INDEX))
-                            .getStringValue();
-                    return new Datatype(dataverseName, datatypeName, new AOrderedListType(
-                            AsterixBuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId, dataverseName,
-                                    orderedlistTypeName, false), datatypeName), isAnonymous);
+                                    .getStringValue();
+                    return new Datatype(dataverseName, datatypeName,
+                            new AOrderedListType(AsterixBuiltinTypeMap.getTypeFromTypeName(metadataNode, jobId,
+                                    dataverseName, orderedlistTypeName, false), datatypeName),
+                            isAnonymous);
                 }
                 default:
                     throw new UnsupportedOperationException("Unsupported derived type: " + tag);
@@ -250,8 +249,8 @@
                 tag = DerivedTypeTag.RECORD;
                 break;
             default:
-                throw new UnsupportedOperationException("No metadata record Type for "
-                        + derivedDatatype.getDisplayName());
+                throw new UnsupportedOperationException(
+                        "No metadata record Type for " + derivedDatatype.getDisplayName());
         }
 
         derivedRecordBuilder.reset(MetadataRecordTypes.DERIVEDTYPE_RECORDTYPE);
@@ -300,8 +299,8 @@
         stringSerde.serialize(aString, out);
     }
 
-    private void writeRecordType(Datatype instance, AbstractComplexType type, DataOutput out) throws IOException,
-            AsterixException {
+    private void writeRecordType(Datatype instance, AbstractComplexType type, DataOutput out)
+            throws IOException, AsterixException {
 
         ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
         ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
index e81da3f..ef3ac49 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
@@ -24,19 +24,18 @@
 import java.io.DataInputStream;
 
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IValueExtractor;
-import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 /**
  * Extracts the value of field 'DatasetName' from an ITupleReference that
  * contains a serialized representation of a Dataset metadata entity.
  */
 public class DatasetNameValueExtractor implements IValueExtractor<String> {
-    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+    private final UTF8StringReader reader = new UTF8StringReader();
 
     @Override
     public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
@@ -45,6 +44,10 @@
         int recordLength = tuple.getFieldLength(2);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        return (((AString) aObjSerDer.deserialize(in)).getStringValue());
+        try {
+            return reader.readUTF(in);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
     }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
index 9a50a31..edb3d2e 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
@@ -22,16 +22,15 @@
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
-import java.rmi.RemoteException;
+import java.io.IOException;
 
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataNode;
 import org.apache.asterix.metadata.api.IValueExtractor;
-import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 /**
  * Extracts the value of field 'DataypeName' from an ITupleReference that
@@ -40,7 +39,7 @@
 public class DatatypeNameValueExtractor implements IValueExtractor<String> {
     private final String dataverseName;
     private final MetadataNode metadataNode;
-    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+    private final UTF8StringReader reader = new UTF8StringReader();
 
     public DatatypeNameValueExtractor(String dataverseName, MetadataNode metadataNode) {
         this.dataverseName = dataverseName;
@@ -54,16 +53,16 @@
         int recordLength = tuple.getFieldLength(2);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        String typeName = ((AString) aObjSerDer.deserialize(in)).getStringValue();
         try {
+            String typeName = reader.readUTF(in);
             if (metadataNode.getDatatype(jobId, dataverseName, typeName).getIsAnonymous()) {
                 // Get index 0 because it is anonymous type, and it is used in
                 // only one non-anonymous type.
                 typeName = metadataNode.getDatatypeNamesUsingThisDatatype(jobId, dataverseName, typeName).get(0);
             }
-        } catch (RemoteException e) {
+            return typeName;
+        } catch (IOException e) {
             throw new MetadataException(e);
         }
-        return typeName;
     }
 }
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 41d92c9..bdc36e8 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -22,14 +22,14 @@
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
+import java.io.IOException;
 
 import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.api.IValueExtractor;
-import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 /**
  * Extracts the value of field 'DataypeName' of the first nested type from an
@@ -43,7 +43,8 @@
     public NestedDatatypeNameValueExtractor(String datatypeName) {
         this.datatypeName = datatypeName;
     }
-    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+
+    private final UTF8StringReader reader = new UTF8StringReader();
 
     @Override
     public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
@@ -52,14 +53,18 @@
         int recordLength = tuple.getFieldLength(2);
         ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
         DataInput in = new DataInputStream(stream);
-        String nestedType = ((AString) aObjSerDer.deserialize(in)).getStringValue();
-        if (nestedType.equals(datatypeName)) {
-            recordStartOffset = tuple.getFieldStart(1);
-            recordLength = tuple.getFieldLength(1);
-            stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
-            in = new DataInputStream(stream);
-            return ((AString) aObjSerDer.deserialize(in)).getStringValue();
+        try {
+            String nestedType = reader.readUTF(in);
+            if (nestedType.equals(datatypeName)) {
+                recordStartOffset = tuple.getFieldStart(1);
+                recordLength = tuple.getFieldLength(1);
+                stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+                in = new DataInputStream(stream);
+                return reader.readUTF(in);
+            }
+            return null;
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
         }
-        return null;
     }
 }
diff --git a/asterix-om/pom.xml b/asterix-om/pom.xml
index 7e42853..e16a7da 100644
--- a/asterix-om/pom.xml
+++ b/asterix-om/pom.xml
@@ -59,5 +59,11 @@
 			<groupId>org.apache.hyracks</groupId>
 			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
 		</dependency>
+		<dependency>
+        <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>0.9.10</version>
+            <scope>test</scope>
+        </dependency>
 	</dependencies>
 </project>
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
index 3bde3b3..ee180bf 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
@@ -30,19 +30,20 @@
 public class ABinarySerializerDeserializer implements ISerializerDeserializer<ABinary> {
 
     private static final long serialVersionUID = 1L;
+    public static final ABinarySerializerDeserializer INSTANCE = new ABinarySerializerDeserializer();
 
-    public ABinarySerializerDeserializer() {}
-
-    private ByteArraySerializerDeserializer byteArrayPtrSerDer = new ByteArraySerializerDeserializer();
+    private ABinarySerializerDeserializer() {
+    }
 
     @Override
     public ABinary deserialize(DataInput in) throws HyracksDataException {
-        return new ABinary(byteArrayPtrSerDer.deserialize(in));
+        return new ABinary(ByteArraySerializerDeserializer.INSTANCE.deserialize(in));
     }
 
     @Override
     public void serialize(ABinary binary, DataOutput out) throws HyracksDataException {
-        byteArrayPtrSerDer.serialize(binary.getBytes(), binary.getStart(), binary.getLength(), out);
+        ByteArraySerializerDeserializer.INSTANCE.serialize(binary.getBytes(), binary.getStart(), binary.getLength(),
+                out);
     }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
index a80047d..4460fd9 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
@@ -23,12 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ACircle;
-import org.apache.asterix.om.base.AMutableCircle;
-import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.APoint;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,12 +33,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ACircleSerializerDeserializer INSTANCE = new ACircleSerializerDeserializer();
-
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<ACircle> circleSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ACIRCLE);
-    private final static AMutableCircle aCircle = new AMutableCircle(null, 0);
-    private final static AMutablePoint aPoint = new AMutablePoint(0, 0);
 
     private ACircleSerializerDeserializer() {
     }
@@ -82,16 +72,5 @@
 
     public final static int getRadiusOffset() throws HyracksDataException {
         return 17;
-    }
-
-    public static void parse(String circle, DataOutput out) throws HyracksDataException {
-        try {
-            String[] parts = circle.split(" ");
-            aPoint.setValue(Double.parseDouble(parts[0].split(",")[0]), Double.parseDouble(parts[0].split(",")[1]));
-            aCircle.setValue(aPoint, Double.parseDouble(parts[1].substring(0, parts[1].length())));
-            circleSerde.serialize(aCircle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(circle + " can not be an instance of circle");
-        }
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
index 081a08b..9509b0b 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,9 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ADateSerializerDeserializer INSTANCE = new ADateSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADate> dateSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADATE);
 
     private ADateSerializerDeserializer() {
     }
@@ -59,25 +51,6 @@
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-    }
-
-    public static void parse(String date, DataOutput out) throws HyracksDataException {
-        AMutableDate aDate = new AMutableDate(0);
-
-        long chrononTimeInMs = 0;
-        try {
-            chrononTimeInMs = ADateParserFactory.parseDatePart(date, 0, date.length());
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        short temp = 0;
-        if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
-            temp = 1;
-        }
-        aDate.setValue((int) (chrononTimeInMs / GregorianCalendarSystem.CHRONON_OF_DAY) - temp);
-
-        dateSerde.serialize(aDate, out);
     }
 
     public static int getChronon(byte[] byteArray, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
index 657b60e..6eb3600 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
@@ -22,13 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,11 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ADateTimeSerializerDeserializer INSTANCE = new ADateTimeSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADateTime> datetimeSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADATETIME);
-    private static final AMutableDateTime aDateTime = new AMutableDateTime(0L);
-    private static final String errorMessage = "This can not be an instance of datetime";
 
     private ADateTimeSerializerDeserializer() {
     }
@@ -62,35 +51,6 @@
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-    }
-
-    public static void parse(String datetime, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMs = 0;
-        try {
-
-            // +1 if it is negative (-)
-            short timeOffset = (short) ((datetime.charAt(0) == '-') ? 1 : 0);
-
-            timeOffset += 8;
-
-            if (datetime.charAt(timeOffset) != 'T') {
-                timeOffset += 2;
-                if (datetime.charAt(timeOffset) != 'T') {
-                    throw new AlgebricksException(errorMessage + ": missing T");
-                }
-            }
-
-            chrononTimeInMs = ADateParserFactory.parseDatePart(datetime, 0, timeOffset);
-
-            chrononTimeInMs += ATimeParserFactory.parseTimePart(datetime, timeOffset + 1, datetime.length()
-                    - timeOffset - 1);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-        aDateTime.setValue(chrononTimeInMs);
-
-        datetimeSerde.serialize(aDateTime, out);
     }
 
     public static long getChronon(byte[] data, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
index 79f3239..c2e7729 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADayTimeDuration;
-import org.apache.asterix.om.base.AMutableDayTimeDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,10 +32,8 @@
 
     public static final ADayTimeDurationSerializerDeserializer INSTANCE = new ADayTimeDurationSerializerDeserializer();
 
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
-    private static final AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
+    private ADayTimeDurationSerializerDeserializer() {
+    }
 
     @Override
     public ADayTimeDuration deserialize(DataInput in) throws HyracksDataException {
@@ -57,16 +50,6 @@
             out.writeLong(instance.getMilliseconds());
         } catch (IOException e) {
             throw new HyracksDataException();
-        }
-    }
-
-    public void parse(String durationString, DataOutput out) throws HyracksDataException {
-        try {
-            ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
-                    ADurationParseOption.All);
-            dayTimeDurationSerde.serialize(aDayTimeDuration, out);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
         }
     }
 
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
index 3108ac9..5eb8369 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ADuration;
-import org.apache.asterix.om.base.AMutableDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,10 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ADurationSerializerDeserializer INSTANCE = new ADurationSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ADuration> durationSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ADURATION);
-    private static final AMutableDuration aDuration = new AMutableDuration(0, 0);
 
     private ADurationSerializerDeserializer() {
     }
@@ -63,18 +54,9 @@
         }
     }
 
-    public static void parse(String duration, DataOutput out) throws HyracksDataException {
-        try {
-            ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration, ADurationParseOption.All);
-            durationSerde.serialize(aDuration, out);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
     /**
      * Get the year-month field of the duration as an integer number of days.
-     * 
+     *
      * @param data
      * @param offset
      * @return
@@ -85,7 +67,7 @@
 
     /**
      * Get the day-time field of the duration as an long integer number of milliseconds.
-     * 
+     *
      * @param data
      * @param offset
      * @return
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
index a0469d7..a0c2657 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
@@ -32,6 +32,9 @@
 
     public static final AInt16SerializerDeserializer INSTANCE = new AInt16SerializerDeserializer();
 
+    private AInt16SerializerDeserializer() {
+    }
+
     @Override
     public AInt16 deserialize(DataInput in) throws HyracksDataException {
         try {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
index 80ba3ab..16c5f46 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
@@ -32,6 +32,9 @@
 
     public static final AInt8SerializerDeserializer INSTANCE = new AInt8SerializerDeserializer();
 
+    private AInt8SerializerDeserializer() {
+    }
+
     @Override
     public AInt8 deserialize(DataInput in) throws HyracksDataException {
         try {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
index 5300b28..5d08d52 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
@@ -22,15 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AInterval;
-import org.apache.asterix.om.base.AMutableInterval;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -39,11 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final AIntervalSerializerDeserializer INSTANCE = new AIntervalSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<AInterval> intervalSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AINTERVAL);
-    private static final AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
-    private static final String errorMessage = "This can not be an instance of interval";
 
     private AIntervalSerializerDeserializer() {
     }
@@ -87,181 +74,6 @@
 
     public static byte getIntervalTimeType(byte[] data, int offset) {
         return data[offset + 8 * 2];
-    }
-
-    /**
-     * create an interval value from two given datetime instance.
-     * 
-     * @param interval
-     * @param out
-     * @throws HyracksDataException
-     */
-    public static void parseDatetime(String interval, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMsStart = 0;
-        long chrononTimeInMsEnd = 0;
-        try {
-
-            // the starting point for parsing (so for the accessor)
-            int startOffset = 0;
-            int endOffset, timeSeperatorOffsetInDatetimeString;
-
-            // Get the index for the comma
-            int commaIndex = interval.indexOf(',');
-            if (commaIndex < 1) {
-                throw new AlgebricksException("comma is missing for a string of interval");
-            }
-
-            endOffset = commaIndex - 1;
-
-            timeSeperatorOffsetInDatetimeString = interval.indexOf('T');
-
-            if (timeSeperatorOffsetInDatetimeString < 0) {
-                throw new AlgebricksException(errorMessage + ": missing T for a datetime value.");
-            }
-
-            chrononTimeInMsStart = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
-
-            chrononTimeInMsStart += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
-
-            // Interval End
-            startOffset = commaIndex + 1;
-            endOffset = interval.length() - 1;
-
-            timeSeperatorOffsetInDatetimeString = interval.indexOf('T', startOffset);
-
-            if (timeSeperatorOffsetInDatetimeString < 0) {
-                throw new AlgebricksException(errorMessage + ": missing T for a datetime value.");
-            }
-
-            chrononTimeInMsEnd = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
-
-            chrononTimeInMsEnd += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        try {
-            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.DATETIME.serialize());
-        } catch (AlgebricksException e) {
-            throw new HyracksDataException(e);
-        }
-
-        intervalSerde.serialize(aInterval, out);
-    }
-
-    public static void parseTime(String interval, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMsStart = 0;
-        long chrononTimeInMsEnd = 0;
-        try {
-
-            int startOffset = 0;
-            int endOffset;
-
-            // Get the index for the comma
-            int commaIndex = interval.indexOf(',');
-            if (commaIndex < 0) {
-                throw new AlgebricksException("comma is missing for a string of interval");
-            }
-
-            endOffset = commaIndex - 1;
-
-            // Interval Start
-            chrononTimeInMsStart = parseTimePart(interval, startOffset, endOffset);
-
-            if (chrononTimeInMsStart < 0) {
-                chrononTimeInMsStart += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-
-            // Interval End
-            startOffset = commaIndex + 1;
-            endOffset = interval.length() - 1;
-
-            chrononTimeInMsEnd = parseTimePart(interval, startOffset, endOffset);
-
-            if (chrononTimeInMsEnd < 0) {
-                chrononTimeInMsEnd += GregorianCalendarSystem.CHRONON_OF_DAY;
-            }
-
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        try {
-            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.TIME.serialize());
-        } catch (AlgebricksException e) {
-            throw new HyracksDataException(e);
-        }
-        intervalSerde.serialize(aInterval, out);
-    }
-
-    public static void parseDate(String interval, DataOutput out) throws HyracksDataException {
-
-        long chrononTimeInMsStart = 0;
-        long chrononTimeInMsEnd = 0;
-        try {
-
-            // the starting point for parsing (so for the accessor)
-            int startOffset = 0;
-            int endOffset;
-
-            // Get the index for the comma
-            int commaIndex = interval.indexOf(',');
-            if (commaIndex < 1) {
-                throw new AlgebricksException("comma is missing for a string of interval");
-            }
-
-            endOffset = commaIndex - 1;
-
-            chrononTimeInMsStart = parseDatePart(interval, startOffset, endOffset);
-
-            // Interval End
-            startOffset = commaIndex + 1;
-            endOffset = interval.length() - 1;
-
-            chrononTimeInMsEnd = parseDatePart(interval, startOffset, endOffset);
-
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        try {
-            aInterval.setValue((chrononTimeInMsStart / GregorianCalendarSystem.CHRONON_OF_DAY),
-                    (chrononTimeInMsEnd / GregorianCalendarSystem.CHRONON_OF_DAY), ATypeTag.DATE.serialize());
-        } catch (AlgebricksException e) {
-            throw new HyracksDataException(e);
-        }
-
-        intervalSerde.serialize(aInterval, out);
-    }
-
-    private static long parseDatePart(String interval, int startOffset, int endOffset) throws AlgebricksException,
-            HyracksDataException {
-
-        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
-            endOffset--;
-        }
-
-        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
-            startOffset++;
-        }
-
-        return ADateParserFactory.parseDatePart(interval, startOffset, endOffset - startOffset + 1);
-    }
-
-    private static int parseTimePart(String interval, int startOffset, int endOffset) throws AlgebricksException,
-            HyracksDataException {
-
-        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
-            endOffset--;
-        }
-
-        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
-            startOffset++;
-        }
-
-        return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
     }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
index 67408cd..aef8de0 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
@@ -23,12 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ALine;
-import org.apache.asterix.om.base.AMutableLine;
-import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.APoint;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -38,12 +34,8 @@
 
     public static final ALineSerializerDeserializer INSTANCE = new ALineSerializerDeserializer();
 
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<ALine> lineSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ALINE);
-    private final static AMutableLine aLine = new AMutableLine(null, null);
-    private final static AMutablePoint aLinePoint1 = new AMutablePoint(0, 0);
-    private final static AMutablePoint aLinePoint2 = new AMutablePoint(0, 0);
+    private ALineSerializerDeserializer() {
+    }
 
     @Override
     public ALine deserialize(DataInput in) throws HyracksDataException {
@@ -92,19 +84,4 @@
         }
     }
 
-    public static void parse(String line, DataOutput out) throws HyracksDataException {
-        try {
-            String[] points = line.split(" ");
-            if (points.length != 2)
-                throw new HyracksDataException("line consists of only 2 points.");
-            aLinePoint1.setValue(Double.parseDouble(points[0].split(",")[0]),
-                    Double.parseDouble(points[0].split(",")[1]));
-            aLinePoint2.setValue(Double.parseDouble(points[1].split(",")[0]),
-                    Double.parseDouble(points[1].split(",")[1]));
-            aLine.setValue(aLinePoint1, aLinePoint2);
-            lineSerde.serialize(aLine, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(line + " can not be an instance of line");
-        }
-    }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index c2c9fd6..6c754f1 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -57,105 +57,69 @@
 
     private static final long serialVersionUID = 1L;
 
-    private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
-    private final ABinarySerializerDeserializer aBinarySerDer = new ABinarySerializerDeserializer();
-    private ARecordSerializerDeserializer aRecordSerDer = null;
+    public static final AObjectSerializerDeserializer INSTANCE = new AObjectSerializerDeserializer();
 
-    private ARecordSerializerDeserializer getRecordSerDer() {
-        if (aRecordSerDer == null) {
-            aRecordSerDer = ARecordSerializerDeserializer.CREATE_SCHEMALESS_INSTANCE();
-        }
-        return aRecordSerDer;
-    }
-
-    public AObjectSerializerDeserializer() {
+    private AObjectSerializerDeserializer() {
     }
 
     @Override
     public IAObject deserialize(DataInput in) throws HyracksDataException {
         ATypeTag typeTag = SerializerDeserializerUtil.deserializeTag(in);
         switch (typeTag) {
-            case NULL: {
+            case NULL:
                 return ANullSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case BOOLEAN: {
+            case BOOLEAN:
                 return ABooleanSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT8: {
+            case INT8:
                 return AInt8SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT16: {
+            case INT16:
                 return AInt16SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT32: {
+            case INT32:
                 return AInt32SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INT64: {
+            case INT64:
                 return AInt64SerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case FLOAT: {
+            case FLOAT:
                 return AFloatSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DOUBLE: {
+            case DOUBLE:
                 return ADoubleSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case STRING: {
-                return aStringSerDer.deserialize(in);
-            }
-            case BINARY: {
-                return aBinarySerDer.deserialize(in);
-            }
-            case DATE: {
+            case STRING:
+                return AStringSerializerDeserializer.INSTANCE.deserialize(in);
+            case BINARY:
+                return ABinarySerializerDeserializer.INSTANCE.deserialize(in);
+            case DATE:
                 return ADateSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case TIME: {
+            case TIME:
                 return ATimeSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DATETIME: {
+            case DATETIME:
                 return ADateTimeSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DURATION: {
+            case DURATION:
                 return ADurationSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case YEARMONTHDURATION: {
+            case YEARMONTHDURATION:
                 return AYearMonthDurationSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case DAYTIMEDURATION: {
+            case DAYTIMEDURATION:
                 return ADayTimeDurationSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case INTERVAL: {
+            case INTERVAL:
                 return AIntervalSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case POINT: {
+            case POINT:
                 return APointSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case POINT3D: {
+            case POINT3D:
                 return APoint3DSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case LINE: {
+            case LINE:
                 return ALineSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case RECTANGLE: {
+            case RECTANGLE:
                 return ARectangleSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case POLYGON: {
+            case POLYGON:
                 return APolygonSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case CIRCLE: {
+            case CIRCLE:
                 return ACircleSerializerDeserializer.INSTANCE.deserialize(in);
-            }
-            case RECORD: {
-                return getRecordSerDer().deserialize(in);
-            }
-            case ORDEREDLIST: {
+            case RECORD:
+                return ARecordSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
+            case ORDEREDLIST:
                 return AOrderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
-            }
-            case UNORDEREDLIST: {
+            case UNORDEREDLIST:
                 return AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
-            }
-            default: {
+            default:
                 throw new NotImplementedException("No serializer/deserializer implemented for type " + typeTag + " .");
-            }
         }
     }
 
@@ -169,110 +133,84 @@
             throw new HyracksDataException(e);
         }
         switch (tag) {
-            case NULL: {
+            case NULL:
                 ANullSerializerDeserializer.INSTANCE.serialize((ANull) instance, out);
                 break;
-            }
-            case BOOLEAN: {
+            case BOOLEAN:
                 ABooleanSerializerDeserializer.INSTANCE.serialize((ABoolean) instance, out);
                 break;
-            }
-            case INT8: {
+            case INT8:
                 AInt8SerializerDeserializer.INSTANCE.serialize((AInt8) instance, out);
                 break;
-            }
-            case INT16: {
+            case INT16:
                 AInt16SerializerDeserializer.INSTANCE.serialize((AInt16) instance, out);
                 break;
-            }
-            case INT32: {
+            case INT32:
                 AInt32SerializerDeserializer.INSTANCE.serialize((AInt32) instance, out);
                 break;
-            }
-            case INT64: {
+            case INT64:
                 AInt64SerializerDeserializer.INSTANCE.serialize((AInt64) instance, out);
                 break;
-            }
-            case FLOAT: {
+            case FLOAT:
                 AFloatSerializerDeserializer.INSTANCE.serialize((AFloat) instance, out);
                 break;
-            }
-            case DOUBLE: {
+            case DOUBLE:
                 ADoubleSerializerDeserializer.INSTANCE.serialize((ADouble) instance, out);
                 break;
-            }
-            case STRING: {
-                aStringSerDer.serialize((AString) instance, out);
+            case STRING:
+                AStringSerializerDeserializer.INSTANCE.serialize((AString) instance, out);
                 break;
-            }
-            case BINARY: {
-                aBinarySerDer.serialize((ABinary) instance, out);
+            case BINARY:
+                ABinarySerializerDeserializer.INSTANCE.serialize((ABinary) instance, out);
                 break;
-            }
-            case DATE: {
+            case DATE:
                 ADateSerializerDeserializer.INSTANCE.serialize((ADate) instance, out);
                 break;
-            }
-            case TIME: {
+            case TIME:
                 ATimeSerializerDeserializer.INSTANCE.serialize((ATime) instance, out);
                 break;
-            }
-            case DATETIME: {
+            case DATETIME:
                 ADateTimeSerializerDeserializer.INSTANCE.serialize((ADateTime) instance, out);
                 break;
-            }
-            case DURATION: {
+            case DURATION:
                 ADurationSerializerDeserializer.INSTANCE.serialize((ADuration) instance, out);
                 break;
-            }
-            case INTERVAL: {
+            case INTERVAL:
                 AIntervalSerializerDeserializer.INSTANCE.serialize((AInterval) instance, out);
                 break;
-            }
-            case POINT: {
+            case POINT:
                 APointSerializerDeserializer.INSTANCE.serialize((APoint) instance, out);
                 break;
-            }
-            case POINT3D: {
+            case POINT3D:
                 APoint3DSerializerDeserializer.INSTANCE.serialize((APoint3D) instance, out);
                 break;
-            }
-            case LINE: {
+            case LINE:
                 ALineSerializerDeserializer.INSTANCE.serialize((ALine) instance, out);
                 break;
-            }
-            case RECTANGLE: {
+            case RECTANGLE:
                 ARectangleSerializerDeserializer.INSTANCE.serialize((ARectangle) instance, out);
                 break;
-            }
-            case POLYGON: {
+            case POLYGON:
                 APolygonSerializerDeserializer.INSTANCE.serialize((APolygon) instance, out);
                 break;
-            }
-            case CIRCLE: {
+            case CIRCLE:
                 ACircleSerializerDeserializer.INSTANCE.serialize((ACircle) instance, out);
                 break;
-            }
-            case RECORD: {
-                getRecordSerDer().serialize((ARecord) instance, out);
+            case RECORD:
+                ARecordSerializerDeserializer.SCHEMALESS_INSTANCE.serialize((ARecord) instance, out);
                 break;
-            }
-            case ORDEREDLIST: {
+            case ORDEREDLIST:
                 AOrderedListSerializerDeserializer.SCHEMALESS_INSTANCE.serialize((AOrderedList) instance, out);
                 break;
-            }
-            case UNORDEREDLIST: {
+            case UNORDEREDLIST:
                 AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.serialize((AUnorderedList) instance, out);
                 break;
-            }
-            case TYPE: {
+            case TYPE:
                 ATypeSerializerDeserializer.INSTANCE.serialize((IAType) instance, out);
                 break;
-            }
-            default: {
+            default:
                 throw new NotImplementedException(
                         "No serializer/deserializer implemented for type " + t.getTypeTag() + " .");
-            }
         }
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
index e8317eb..0004977 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
@@ -20,8 +20,8 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.asterix.builders.OrderedListBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -33,6 +33,7 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -43,30 +44,24 @@
     private static final long serialVersionUID = 1L;
     public static final AOrderedListSerializerDeserializer SCHEMALESS_INSTANCE = new AOrderedListSerializerDeserializer();
 
-    private IAType itemType;
+    private final IAType itemType;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer serializer;
+    private final ISerializerDeserializer serializer;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer deserializer;
-    private AOrderedListType orderedlistType;
+    private final ISerializerDeserializer deserializer;
+    private final AOrderedListType orderedlistType;
 
     private AOrderedListSerializerDeserializer() {
-        this.itemType = null;
-        this.orderedlistType = null;
-        initSerializerDeserializer(BuiltinType.ANY);
+        this(new AOrderedListType(BuiltinType.ANY, "orderedlist"));
     }
 
     public AOrderedListSerializerDeserializer(AOrderedListType orderedlistType) {
         this.orderedlistType = orderedlistType;
-        initSerializerDeserializer(orderedlistType.getItemType());
-    }
-
-    private void initSerializerDeserializer(IAType itemType) {
-        this.itemType = itemType;
+        this.itemType = orderedlistType.getItemType();
         serializer = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-        deserializer = itemType.getTypeTag() == ATypeTag.ANY ? AqlSerializerDeserializerProvider.INSTANCE
-                .getSerializerDeserializer(itemType) : AqlSerializerDeserializerProvider.INSTANCE
-                .getNonTaggedSerializerDeserializer(itemType);
+        deserializer = itemType.getTypeTag() == ATypeTag.ANY
+                ? AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType)
+                : AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(itemType);
     }
 
     @Override
@@ -87,33 +82,33 @@
                     fixedSize = true;
                     break;
             }
-            
-            if (itemType.getTypeTag() == ATypeTag.ANY && typeTag != ATypeTag.ANY)   
-            try {   
-                initSerializerDeserializer(BuiltinType.builtinTypeFromString(typeTag.name().toLowerCase()));    
-            } catch (AsterixException e) {  
-                throw new HyracksDataException(e);  
-            }
-            
 
+            IAType currentItemType = itemType;
+            @SuppressWarnings("rawtypes")
+            ISerializerDeserializer currentDeserializer = deserializer;
+            if (itemType.getTypeTag() == ATypeTag.ANY && typeTag != ATypeTag.ANY) {
+                currentItemType = TypeTagUtil.getBuiltinTypeByTag(typeTag);
+                currentDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+                        .getNonTaggedSerializerDeserializer(currentItemType);
+            }
+
+            List<IAObject> items = new ArrayList<IAObject>();
             in.readInt(); // list size
             int numberOfitems;
             numberOfitems = in.readInt();
-            ArrayList<IAObject> items = new ArrayList<IAObject>();
             if (numberOfitems > 0) {
                 if (!fixedSize) {
                     for (int i = 0; i < numberOfitems; i++)
                         in.readInt();
                 }
                 for (int i = 0; i < numberOfitems; i++) {
-                    IAObject v = (IAObject) deserializer.deserialize(in);
+                    IAObject v = (IAObject) currentDeserializer.deserialize(in);
                     items.add(v);
                 }
             }
-            AOrderedListType type = new AOrderedListType(itemType, "orderedlist");
+            AOrderedListType type = new AOrderedListType(currentItemType, "orderedlist");
             return new AOrderedList(type, items);
-
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
index 69c7a71..57e54a7 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
@@ -23,11 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutablePoint3D;
 import org.apache.asterix.om.base.APoint3D;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,10 +33,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final APoint3DSerializerDeserializer INSTANCE = new APoint3DSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<APoint3D> point3DSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.APOINT3D);
-    private final static AMutablePoint3D aPoint3D = new AMutablePoint3D(0, 0, 0);
 
     private APoint3DSerializerDeserializer() {
     }
@@ -90,16 +83,4 @@
         }
     }
 
-    public static void parse(String point3d, DataOutput out) throws HyracksDataException {
-        try {
-            int firstCommaIndex = point3d.indexOf(',');
-            int secondCommaIndex = point3d.indexOf(',', firstCommaIndex + 1);
-            aPoint3D.setValue(Double.parseDouble(point3d.substring(0, firstCommaIndex)),
-                    Double.parseDouble(point3d.substring(firstCommaIndex + 1, secondCommaIndex)),
-                    Double.parseDouble(point3d.substring(secondCommaIndex + 1, point3d.length())));
-            point3DSerde.serialize(aPoint3D, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point3d + " can not be an instance of point3d");
-        }
-    }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
index 4fa06ce..a927f95 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
@@ -23,10 +23,7 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.APoint;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -35,10 +32,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final APointSerializerDeserializer INSTANCE = new APointSerializerDeserializer();
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<APoint> pointSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.APOINT);
-    private final static AMutablePoint aPoint = new AMutablePoint(0, 0);
 
     private APointSerializerDeserializer() {
     }
@@ -82,16 +75,6 @@
                 throw new HyracksDataException("Wrong coordinate");
         }
 
-    }
-
-    public static void parse(String point, DataOutput out) throws HyracksDataException {
-        try {
-            aPoint.setValue(Double.parseDouble(point.substring(0, point.indexOf(','))),
-                    Double.parseDouble(point.substring(point.indexOf(',') + 1, point.length())));
-            pointSerde.serialize(aPoint, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(point + " can not be an instance of point");
-        }
     }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 0d1f63f..543b20e 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -48,34 +48,23 @@
 public class ARecordSerializerDeserializer implements ISerializerDeserializer<ARecord> {
     private static final long serialVersionUID = 1L;
 
-    public static final ARecordSerializerDeserializer CREATE_SCHEMALESS_INSTANCE() {
-        return new ARecordSerializerDeserializer();
-    }
+    public static final ARecordSerializerDeserializer SCHEMALESS_INSTANCE = new ARecordSerializerDeserializer();
 
-    private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
-    private AObjectSerializerDeserializer aObjSerDer = null;
-
-    private AObjectSerializerDeserializer getObjSerDer() {
-        if (aObjSerDer == null) {
-            aObjSerDer = new AObjectSerializerDeserializer();
-        }
-        return aObjSerDer;
-    }
-
-    private ARecordType recordType;
-    private int numberOfSchemaFields = 0;
+    private final ARecordType recordType;
+    private final int numberOfSchemaFields;
 
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer serializers[];
+    private final ISerializerDeserializer serializers[];
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer deserializers[];
+    private final ISerializerDeserializer deserializers[];
 
     private ARecordSerializerDeserializer() {
+        this(null);
     }
 
     public ARecordSerializerDeserializer(ARecordType recordType) {
-        this.recordType = recordType;
         if (recordType != null) {
+            this.recordType = recordType;
             this.numberOfSchemaFields = recordType.getFieldNames().length;
             serializers = new ISerializerDeserializer[numberOfSchemaFields];
             deserializers = new ISerializerDeserializer[numberOfSchemaFields];
@@ -97,6 +86,11 @@
                 serializers[i] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(t2);
                 deserializers[i] = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(t2);
             }
+        } else {
+            this.recordType = null;
+            this.numberOfSchemaFields = 0;
+            this.serializers = null;
+            this.deserializers = null;
         }
     }
 
@@ -137,7 +131,6 @@
                     }
                     closedFields[fieldId] = (IAObject) deserializers[fieldId].deserialize(in);
                 }
-
             }
 
             if (isExpanded) {
@@ -150,8 +143,8 @@
                     in.readInt();
                 }
                 for (int i = 0; i < numberOfOpenFields; i++) {
-                    fieldNames[i] = aStringSerDer.deserialize(in).getStringValue();
-                    openFields[i] = getObjSerDer().deserialize(in);
+                    fieldNames[i] = AStringSerializerDeserializer.INSTANCE.deserialize(in).getStringValue();
+                    openFields[i] = AObjectSerializerDeserializer.INSTANCE.deserialize(in);
                     fieldTypes[i] = openFields[i].getType();
                 }
                 ARecordType openPartRecType = new ARecordType(null, fieldNames, fieldTypes, true);
@@ -167,6 +160,35 @@
             }
         } catch (IOException | AsterixException e) {
             throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(ARecord instance, DataOutput out) throws HyracksDataException {
+        this.serialize(instance, out, false);
+    }
+
+    // This serialize method will NOT work if <code>recordType</code> is not equal to the type of the instance.
+    @SuppressWarnings("unchecked")
+    public void serialize(ARecord instance, DataOutput out, boolean writeTypeTag) throws HyracksDataException {
+        IARecordBuilder recordBuilder = new RecordBuilder();
+        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+        recordBuilder.reset(recordType);
+        recordBuilder.init();
+        if (recordType != null) {
+            int fieldIndex = 0;
+            for (; fieldIndex < recordType.getFieldNames().length; ++fieldIndex) {
+                fieldValue.reset();
+                serializers[fieldIndex].serialize(instance.getValueByPos(fieldIndex), fieldValue.getDataOutput());
+                recordBuilder.addField(fieldIndex, fieldValue);
+            }
+            try {
+                recordBuilder.write(out, writeTypeTag);
+            } catch (IOException | AsterixException e) {
+                throw new HyracksDataException(e);
+            }
+        } else {
+            throw new NotImplementedException("Serializer for schemaless records is not implemented.");
         }
     }
 
@@ -197,39 +219,7 @@
             fieldNames[i] = recType2.getFieldNames()[j];
             fieldTypes[i] = recType2.getFieldTypes()[j];
         }
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (HyracksDataException e) {
-            throw new AsterixException(e);
-        }
-    }
-
-    @Override
-    public void serialize(ARecord instance, DataOutput out) throws HyracksDataException {
-        this.serialize(instance, out, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    public void serialize(ARecord instance, DataOutput out, boolean writeTypeTag) throws HyracksDataException {
-        IARecordBuilder recordBuilder = new RecordBuilder();
-        ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
-
-        recordBuilder.reset(recordType);
-        recordBuilder.init();
-        if (recordType != null) {
-            for (int i = 0; i < recordType.getFieldNames().length; i++) {
-                fieldValue.reset();
-                serializers[i].serialize(instance.getValueByPos(i), fieldValue.getDataOutput());
-                recordBuilder.addField(i, fieldValue);
-            }
-            try {
-                recordBuilder.write(out, writeTypeTag);
-            } catch (IOException | AsterixException e) {
-                throw new HyracksDataException(e);
-            }
-        } else {
-            throw new NotImplementedException("Serializer for schemaless records is not implemented.");
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
     public static final int getRecordLength(byte[] serRecord, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
index 4306c76..7e0bf0f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
@@ -23,12 +23,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutablePoint;
-import org.apache.asterix.om.base.AMutableRectangle;
 import org.apache.asterix.om.base.APoint;
 import org.apache.asterix.om.base.ARectangle;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,13 +33,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ARectangleSerializerDeserializer INSTANCE = new ARectangleSerializerDeserializer();
-
-    @SuppressWarnings("unchecked")
-    private final static ISerializerDeserializer<ARectangle> rectangleSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ARECTANGLE);
-    private final static AMutableRectangle aRectangle = new AMutableRectangle(null, null);
-    private final static AMutablePoint aRectanglePoint1 = new AMutablePoint(0, 0);
-    private final static AMutablePoint aRectanglePoint2 = new AMutablePoint(0, 0);
 
     private ARectangleSerializerDeserializer() {
     }
@@ -92,30 +81,6 @@
                 return 25;
             default:
                 throw new HyracksDataException("Wrong coordinate");
-        }
-    }
-
-    public static void parse(String rectangle, DataOutput out) throws HyracksDataException {
-        try {
-            String[] points = rectangle.split(" ");
-            if (points.length != 2)
-                throw new HyracksDataException("rectangle consists of only 2 points.");
-            aRectanglePoint1.setValue(Double.parseDouble(points[0].split(",")[0]),
-                    Double.parseDouble(points[0].split(",")[1]));
-            aRectanglePoint2.setValue(Double.parseDouble(points[1].split(",")[0]),
-                    Double.parseDouble(points[1].split(",")[1]));
-            if (aRectanglePoint1.getX() > aRectanglePoint2.getX() && aRectanglePoint1.getY() > aRectanglePoint2.getY()) {
-                aRectangle.setValue(aRectanglePoint2, aRectanglePoint1);
-            } else if (aRectanglePoint1.getX() < aRectanglePoint2.getX()
-                    && aRectanglePoint1.getY() < aRectanglePoint2.getY()) {
-                aRectangle.setValue(aRectanglePoint1, aRectanglePoint2);
-            } else {
-                throw new IllegalArgumentException(
-                        "Rectangle arugment must be either (bottom left point, top right point) or (top right point, bottom left point)");
-            }
-            rectangleSerde.serialize(aRectangle, out);
-        } catch (HyracksDataException e) {
-            throw new HyracksDataException(rectangle + " can not be an instance of rectangle");
         }
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
index 272b36f..8e2da0b 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
@@ -25,20 +25,26 @@
 import org.apache.asterix.om.base.AString;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.util.string.UTF8StringUtil;
 
+/**
+ * NOTE: this class will have objection creations in each serialize/deserialize
+ * call. Therefore, in order to have efficient runtime implementations, please
+ * use <code>UTF8StringReader</code> and <code>UTF8StringWriter</code> whenever possible.
+ */
 public class AStringSerializerDeserializer implements ISerializerDeserializer<AString> {
 
     private static final long serialVersionUID = 1L;
 
-    private final UTF8StringSerializerDeserializer utf8SerDer = new UTF8StringSerializerDeserializer();
+    public static final AStringSerializerDeserializer INSTANCE = new AStringSerializerDeserializer();
 
-    public AStringSerializerDeserializer() {}
+    private AStringSerializerDeserializer() {
+    }
 
     @Override
     public AString deserialize(DataInput in) throws HyracksDataException {
         try {
-            return new AString(utf8SerDer.deserialize(in));
+            return new AString(UTF8StringUtil.readUTF8(in));
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
@@ -47,7 +53,7 @@
     @Override
     public void serialize(AString instance, DataOutput out) throws HyracksDataException {
         try {
-            utf8SerDer.serialize(instance.getStringValue(), out);
+            UTF8StringUtil.writeUTF8(instance.getStringValue(), out);
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
index a059cde..1138164 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
@@ -22,11 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
 import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -35,11 +31,6 @@
     private static final long serialVersionUID = 1L;
 
     public static final ATimeSerializerDeserializer INSTANCE = new ATimeSerializerDeserializer();
-
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<ATime> timeSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ATIME);
-    private static final AMutableTime aTime = new AMutableTime(0);
 
     private ATimeSerializerDeserializer() {
     }
@@ -62,20 +53,6 @@
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-    }
-
-    public static void parse(String time, DataOutput out) throws HyracksDataException {
-        int chrononTimeInMs;
-
-        try {
-            chrononTimeInMs = ATimeParserFactory.parseTimePart(time, 0, time.length());
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-
-        aTime.setValue(chrononTimeInMs);
-
-        timeSerde.serialize(aTime, out);
     }
 
     public static int getChronon(byte[] byteArray, int offset) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
index caf1eff..8df0b0d 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
@@ -34,6 +34,9 @@
 
     public static final AUUIDSerializerDeserializer INSTANCE = new AUUIDSerializerDeserializer();
 
+    private AUUIDSerializerDeserializer() {
+    }
+
     @Override
     public AUUID deserialize(DataInput in) throws HyracksDataException {
         long msb = Integer64SerializerDeserializer.INSTANCE.deserialize(in);
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
index e109a32..87e2ab7 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
@@ -20,7 +20,6 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.asterix.builders.UnorderedListBuilder;
@@ -31,8 +30,10 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -44,25 +45,24 @@
 
     public static final AUnorderedListSerializerDeserializer SCHEMALESS_INSTANCE = new AUnorderedListSerializerDeserializer();
 
-    private IAType itemType;
-    private AUnorderedListType unorderedlistType;
+    private final IAType itemType;
+    private final AUnorderedListType unorderedlistType;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer nontaggedSerDes;
+    private final ISerializerDeserializer deserializer;
     @SuppressWarnings("rawtypes")
-    private ISerializerDeserializer taggedSerDes;
+    private final ISerializerDeserializer serializer;
 
     private AUnorderedListSerializerDeserializer() {
-        this.itemType = null;
-        this.unorderedlistType = null;
+        this(new AUnorderedListType(BuiltinType.ANY, "unorderedlist"));
     }
 
     public AUnorderedListSerializerDeserializer(AUnorderedListType unorderedlistType) {
-        this.itemType = unorderedlistType.getItemType();
         this.unorderedlistType = unorderedlistType;
-        nontaggedSerDes = itemType.getTypeTag() == ATypeTag.ANY
+        this.itemType = unorderedlistType.getItemType();
+        serializer = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
+        deserializer = itemType.getTypeTag() == ATypeTag.ANY
                 ? AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType)
                 : AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(itemType);
-        taggedSerDes = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
     }
 
     @Override
@@ -84,6 +84,16 @@
                     fixedSize = true;
                     break;
             }
+
+            IAType currentItemType = itemType;
+            @SuppressWarnings("rawtypes")
+            ISerializerDeserializer currentDeserializer = deserializer;
+            if (itemType.getTypeTag() == ATypeTag.ANY && typeTag != ATypeTag.ANY) {
+                currentItemType = TypeTagUtil.getBuiltinTypeByTag(typeTag);
+                currentDeserializer = AqlSerializerDeserializerProvider.INSTANCE
+                        .getNonTaggedSerializerDeserializer(currentItemType);
+            }
+
             in.readInt(); // list size
             int numberOfitems;
             numberOfitems = in.readInt();
@@ -94,13 +104,12 @@
                         in.readInt();
                 }
                 for (int i = 0; i < numberOfitems; i++) {
-                    items.add((IAObject) nontaggedSerDes.deserialize(in));
+                    items.add((IAObject) currentDeserializer.deserialize(in));
                 }
             }
-            AUnorderedListType type = new AUnorderedListType(itemType, "orderedlist");
+            AUnorderedListType type = new AUnorderedListType(currentItemType, "unorderedlist");
             return new AUnorderedList(type, items);
-
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
@@ -115,7 +124,7 @@
         IACursor cursor = instance.getCursor();
         while (cursor.next()) {
             itemValue.reset();
-            taggedSerDes.serialize(cursor.get(), itemValue.getDataOutput());
+            serializer.serialize(cursor.get(), itemValue.getDataOutput());
             listBuilder.addItem(itemValue);
         }
         listBuilder.write(out, false);
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
index b5a6408..eb6942f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
@@ -22,12 +22,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableYearMonthDuration;
 import org.apache.asterix.om.base.AYearMonthDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -37,10 +32,8 @@
 
     public static final AYearMonthDurationSerializerDeserializer INSTANCE = new AYearMonthDurationSerializerDeserializer();
 
-    @SuppressWarnings("unchecked")
-    private static final ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
-    private static final AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
+    private AYearMonthDurationSerializerDeserializer() {
+    }
 
     @Override
     public AYearMonthDuration deserialize(DataInput in) throws HyracksDataException {
@@ -57,16 +50,6 @@
             out.writeInt(instance.getMonths());
         } catch (IOException e) {
             throw new HyracksDataException();
-        }
-    }
-
-    public void parse(String durationString, DataOutput out) throws HyracksDataException {
-        try {
-            ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
-                    ADurationParseOption.All);
-            yearMonthDurationSerde.serialize(aYearMonthDuration, out);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
         }
     }
 
diff --git a/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java b/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
index 910f633..9ca94f1 100644
--- a/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
+++ b/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
@@ -84,7 +84,7 @@
         switch (aqlType.getTypeTag()) {
             case ANY:
             case UNION: { // we could do smth better for nullable fields
-                return new AObjectSerializerDeserializer();
+                return AObjectSerializerDeserializer.INSTANCE;
             }
             default: {
                 return addTag(getNonTaggedSerializerDeserializer(aqlType), aqlType.getTypeTag());
@@ -132,10 +132,10 @@
                 return ANullSerializerDeserializer.INSTANCE;
             }
             case STRING: {
-                return new AStringSerializerDeserializer();
+                return AStringSerializerDeserializer.INSTANCE;
             }
             case BINARY: {
-                return new ABinarySerializerDeserializer();
+                return ABinarySerializerDeserializer.INSTANCE;
             }
             case TIME: {
                 return ATimeSerializerDeserializer.INSTANCE;
@@ -183,8 +183,8 @@
                 return ShortSerializerDeserializer.INSTANCE;
             }
             default: {
-                throw new NotImplementedException("No serializer/deserializer implemented for type "
-                        + aqlType.getTypeTag() + " .");
+                throw new NotImplementedException(
+                        "No serializer/deserializer implemented for type " + aqlType.getTypeTag() + " .");
             }
         }
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java b/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
index 35be4ab..bb9a7bf 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
@@ -18,11 +18,11 @@
  */
 package org.apache.asterix.om.base;
 
-import java.util.ArrayList;
+import java.util.List;
 
 public class ACollectionCursor implements IACursor {
 
-    private ArrayList<IAObject> values;
+    private List<IAObject> values;
     private int pos;
     private int size;
     private IAObject object = null;
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java b/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
index 9130c64..94416b3 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
@@ -21,19 +21,18 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.visitors.IOMVisitor;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 public class AOrderedList implements IACollection {
 
-    protected ArrayList<IAObject> values;
+    protected List<IAObject> values;
     protected AOrderedListType type;
 
     public AOrderedList(AOrderedListType type) {
@@ -41,7 +40,7 @@
         this.type = type;
     }
 
-    public AOrderedList(AOrderedListType type, ArrayList<IAObject> sequence) {
+    public AOrderedList(AOrderedListType type, List<IAObject> sequence) {
         values = sequence;
         this.type = type;
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index ece2c77..438617f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -18,13 +18,12 @@
  */
 package org.apache.asterix.om.base;
 
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.visitors.IOMVisitor;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 public class ARecord implements IAObject {
 
@@ -61,6 +60,11 @@
 
     @Override
     public boolean deepEqual(IAObject obj) {
+        return equals(obj);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
         if (!(obj instanceof ARecord)) {
             return false;
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java b/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
index f8c02b0..8f19b7e 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
@@ -19,19 +19,19 @@
 package org.apache.asterix.om.base;
 
 import java.util.ArrayList;
-
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.visitors.IOMVisitor;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 public class AUnorderedList implements IACollection {
 
-    protected ArrayList<IAObject> values;
+    protected List<IAObject> values;
     protected AUnorderedListType type;
 
     public AUnorderedList(AUnorderedListType type) {
@@ -39,7 +39,7 @@
         this.type = type;
     }
 
-    public AUnorderedList(AUnorderedListType type, ArrayList<IAObject> sequence) {
+    public AUnorderedList(AUnorderedListType type, List<IAObject> sequence) {
         values = sequence;
         this.type = type;
     }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java b/asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
index f42def0..cf1dfe4 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
@@ -19,15 +19,12 @@
 
 package org.apache.asterix.om.pointables.base;
 
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.AsterixRuntimeException;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnorderedListType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * This class serves as the repository for the default record type and list type
@@ -41,11 +38,7 @@
     public static ARecordType NESTED_OPEN_RECORD_TYPE;
 
     static {
-        try {
-            NESTED_OPEN_RECORD_TYPE = new ARecordType("nested-open", new String[] {}, new IAType[] {}, true);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AsterixRuntimeException();
-        }
+        NESTED_OPEN_RECORD_TYPE = new ARecordType("nested-open", new String[] {}, new IAType[] {}, true);
     }
 
     // nested open list type
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
index fd10a40..a435f7a 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
@@ -21,24 +21,20 @@
 
 import java.util.Iterator;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.typecomputer.base.TypeComputerUtilities;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ClosedRecordConstructorResultType implements IResultTypeComputer {
 
@@ -62,22 +58,18 @@
         int i = 0;
         Iterator<Mutable<ILogicalExpression>> argIter = f.getArguments().iterator();
         while (argIter.hasNext()) {
-            ILogicalExpression e1 = (AbstractLogicalExpression) argIter.next().getValue();
+            ILogicalExpression e1 = argIter.next().getValue();
             if (e1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
                 ConstantExpression nameExpr = (ConstantExpression) e1;
                 fieldNames[i] = ((AString) ((AsterixConstantValue) nameExpr.getValue()).getObject()).getStringValue();
             } else {
-                throw new AlgebricksException("Field name " + i + "(" + e1
-                        + ") in call to closed-record-constructor is not a constant.");
+                throw new AlgebricksException(
+                        "Field name " + i + "(" + e1 + ") in call to closed-record-constructor is not a constant.");
             }
-            ILogicalExpression e2 = (AbstractLogicalExpression) argIter.next().getValue();
+            ILogicalExpression e2 = argIter.next().getValue();
             fieldTypes[i] = (IAType) env.getType(e2);
             i++;
         }
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, false);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, false);
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
index f742eee..5947b5f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
@@ -19,7 +19,6 @@
 
 package org.apache.asterix.om.typecomputer.impl;
 
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.AUnionType;
@@ -29,7 +28,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class NonTaggedLocalAvgTypeComputer implements IResultTypeComputer {
 
@@ -38,11 +36,9 @@
     @Override
     public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
             IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
-        try {
-            return new ARecordType(null, new String[] { "sum", "count" }, new IAType[] {
-                    AUnionType.createNullableType(BuiltinType.ADOUBLE, "OptionalDouble"), BuiltinType.AINT32 }, false);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
+        return new ARecordType(null,
+                new String[] { "sum", "count" }, new IAType[] {
+                        AUnionType.createNullableType(BuiltinType.ADOUBLE, "OptionalDouble"), BuiltinType.AINT32 },
+                false);
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
index f4c1f07..862d1cd 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
@@ -23,9 +23,6 @@
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
@@ -33,6 +30,7 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeHelper;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
@@ -40,7 +38,6 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class OpenRecordConstructorResultType implements IResultTypeComputer {
 
@@ -78,10 +75,6 @@
         IAType[] fieldTypes = new IAType[n];
         fieldNames = namesList.toArray(fieldNames);
         fieldTypes = typesList.toArray(fieldTypes);
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, true);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
index 6498be9..a9c7968 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
@@ -21,15 +21,13 @@
 
 import java.util.Iterator;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.typecomputer.base.TypeComputerUtilities;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
@@ -37,7 +35,6 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class RecordConstructorResultType implements IResultTypeComputer {
 
@@ -79,10 +76,6 @@
             }
             i++;
         }
-        try {
-            return new ARecordType(null, fieldNames, fieldTypes, isOpen);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
+        return new ARecordType(null, fieldNames, fieldTypes, isOpen);
     }
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
index 19aae54..83f4f44 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
@@ -37,7 +37,6 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class RecordMergeTypeComputer implements IResultTypeComputer {
     public static final RecordMergeTypeComputer INSTANCE = new RecordMergeTypeComputer();
@@ -119,12 +118,8 @@
         String resultTypeName = "merged(" + recType0.getTypeName() + ", " + recType1.getTypeName() + ")";
         boolean isOpen = recType0.isOpen() || recType1.isOpen();
         IAType resultType = null;
-        try {
-            resultType = new ARecordType(resultTypeName, resultFieldNames.toArray(new String[] {}),
-                    resultFieldTypes.toArray(new IAType[] {}), isOpen);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AlgebricksException(e);
-        };
+        resultType = new ARecordType(resultTypeName, resultFieldNames.toArray(new String[] {}),
+                resultFieldTypes.toArray(new IAType[] {}), isOpen);
 
         if (nullable) {
             resultType = AUnionType.createNullableType(resultType);
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index 42ea000..1062b88 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -33,7 +33,6 @@
 import org.apache.asterix.om.visitors.IOMVisitor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
@@ -66,8 +65,7 @@
      *             if there are duplicate field names or if there is an error serializing the field names
      * @throws HyracksDataException
      */
-    public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen)
-            throws AsterixException, HyracksDataException {
+    public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
         super(typeName);
         this.fieldNames = fieldNames;
         this.fieldTypes = fieldTypes;
@@ -232,11 +230,7 @@
                 newTypes[i] = type.fieldTypes[i];
             }
         }
-        try {
-            return new ARecordType(type.typeName, type.fieldNames, newTypes, type.isOpen);
-        } catch (AsterixException | HyracksException e) {
-            throw new AlgebricksException(e);
-        }
+        return new ARecordType(type.typeName, type.fieldNames, newTypes, type.isOpen);
     }
 
     @Override
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
index b4fc813..65f0098 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
@@ -22,19 +22,18 @@
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.visitors.IOMVisitor;
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.visitors.IOMVisitor;
-
 public class AUnionType extends AbstractComplexType {
 
     private static final long serialVersionUID = 1L;
-    private List<IAType> unionList;
     public static final int OPTIONAL_TYPE_INDEX_IN_UNION_LIST = 1;
+    private final List<IAType> unionList;
 
     public AUnionType(List<IAType> unionList, String typeName) {
         super(typeName);
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
index d2b06bc..6e2ffef 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
@@ -62,8 +62,5 @@
             nestedType.generateNestedDerivedTypeNames();
         }
     }
-    // public void serialize(DataOutput out) throws IOException {
-    // out.writeBoolean(isTyped());
-    // }
 
 }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
index de4b5d3..0c0fdaa 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
@@ -877,32 +877,4 @@
         return getType().getTypeTag().serialize();
     }
 
-    public static BuiltinType builtinTypeFromString(String str) throws AsterixException {
-        if (str.equals(BuiltinType.AINT32.getTypeName())) {
-            return BuiltinType.AINT32;
-        } else if (str.equals(BuiltinType.ASTRING.getTypeName())) {
-            return BuiltinType.ASTRING;
-        } else if (str.equals(BuiltinType.ADOUBLE.getTypeName())) {
-            return BuiltinType.ADOUBLE;
-        } else if (str.equals(BuiltinType.AFLOAT.getTypeName())) {
-            return BuiltinType.AFLOAT;
-        } else if (str.equals(BuiltinType.ANY.getTypeName())) {
-            return BuiltinType.ANY;
-        }
-        throw new AsterixException("No string translation for type: " + str + " .");
-    }
-
-    public static ATypeTag builtinTypeTagFromString(String str) throws AsterixException {
-        if (str.equals("int32")) {
-            return ATypeTag.INT32;
-        } else if (str.equals("string")) {
-            return ATypeTag.STRING;
-        } else if (str.equals("double")) {
-            return ATypeTag.DOUBLE;
-        } else if (str.equals("float")) {
-            return ATypeTag.FLOAT;
-        }
-        throw new AsterixException("No string translation for type: " + str + " .");
-    }
-
 }
diff --git a/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
new file mode 100644
index 0000000..c86312f
--- /dev/null
+++ b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.junit.Test;
+
+public class AOrderedListSerializerDeserializerTest {
+
+    @Test
+    public void test() {
+        // Generates types.
+        ARecordType addrRecordType = SerializerDeserializerTestUtils.generateAddressRecordType();
+        ARecordType employeeType = SerializerDeserializerTestUtils.generateEmployeeRecordType(addrRecordType);
+        AOrderedListType employeeListType = new AOrderedListType(employeeType, "employee_list");
+
+        //Generates records.
+        ARecord[] records = SerializerDeserializerTestUtils.generateRecords(addrRecordType, employeeType);
+
+        // Generates lists
+        AOrderedList[] lists = new AOrderedList[4];
+        for (int index = 0; index < lists.length; ++index) {
+            lists[index] = new AOrderedList(employeeListType, Arrays.asList(records));
+        }
+
+        AOrderedListSerializerDeserializer serde = new AOrderedListSerializerDeserializer(employeeListType);
+        // Run four test threads to serialize/deserialize lists concurrently.
+        SerializerDeserializerTestUtils.concurrentSerDeTestRun(serde, lists);
+    }
+
+}
diff --git a/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
new file mode 100644
index 0000000..843bb59
--- /dev/null
+++ b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde;
+
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.types.ARecordType;
+import org.junit.Test;
+
+public class ARecordSerializerDeserializerTest {
+
+    @Test
+    public void test() {
+        // Generates types.
+        ARecordType addrRecordType = SerializerDeserializerTestUtils.generateAddressRecordType();
+        ARecordType employeeType = SerializerDeserializerTestUtils.generateEmployeeRecordType(addrRecordType);
+
+        //Generates records.
+        ARecord[] records = SerializerDeserializerTestUtils.generateRecords(addrRecordType, employeeType);
+
+        ARecordSerializerDeserializer serde = new ARecordSerializerDeserializer(employeeType);
+        // Run four test threads to serialize/deserialize records concurrently.
+        SerializerDeserializerTestUtils.concurrentSerDeTestRun(serde, records);
+    }
+
+}
diff --git a/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
new file mode 100644
index 0000000..5153630
--- /dev/null
+++ b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AUnorderedList;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.junit.Test;
+
+public class AUnorderedListSerializerDeserializerTest {
+
+    @Test
+    public void test() {
+        // Generates types.
+        ARecordType addrRecordType = SerializerDeserializerTestUtils.generateAddressRecordType();
+        ARecordType employeeType = SerializerDeserializerTestUtils.generateEmployeeRecordType(addrRecordType);
+        AUnorderedListType employeeListType = new AUnorderedListType(employeeType, "employee_list");
+
+        //Generates records.
+        ARecord[] records = SerializerDeserializerTestUtils.generateRecords(addrRecordType, employeeType);
+
+        // Generates lists
+        AUnorderedList[] lists = new AUnorderedList[4];
+        for (int index = 0; index < lists.length; ++index) {
+            lists[index] = new AUnorderedList(employeeListType, Arrays.asList(records));
+        }
+
+        AUnorderedListSerializerDeserializer serde = new AUnorderedListSerializerDeserializer(employeeListType);
+        // Run four test threads to serialize/deserialize lists concurrently.
+        SerializerDeserializerTestUtils.concurrentSerDeTestRun(serde, lists);
+    }
+
+}
diff --git a/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
new file mode 100644
index 0000000..4d41092
--- /dev/null
+++ b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInterval;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.junit.Assert;
+
+public class SerializerDeserializerTestUtils {
+
+    public static ARecordType generateAddressRecordType() {
+        String[] addrFieldNames = new String[] { "line", "city", "state", "postcode", "duration" };
+        IAType[] addrFieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                BuiltinType.AINT16, BuiltinType.AINTERVAL };
+        return new ARecordType("addritem", addrFieldNames, addrFieldTypes, true);
+    }
+
+    public static ARecordType generateEmployeeRecordType(ARecordType addrRecordType) {
+        AOrderedListType addrListType = new AOrderedListType(addrRecordType, "address_list");
+        String[] fieldNames = new String[] { "id", "name", "addresses_history" };
+        IAType[] fieldTypes = new IAType[] { BuiltinType.AINT64, BuiltinType.ASTRING, addrListType };
+        return new ARecordType("employee", fieldNames, fieldTypes, true);
+    }
+
+    public static ARecord[] generateRecords(ARecordType addrRecordType, ARecordType employeeType) {
+        AOrderedListType addrListType = new AOrderedListType(addrRecordType, "address_history");
+        ARecord addr11 = new ARecord(addrRecordType, new IAObject[] { new AString("120 San Raman Street"),
+                new AString("Irvine"), new AString("CA"), new AInt16((short) 95051), new AInterval(0, 100, (byte) 0) });
+        ARecord addr12 = new ARecord(addrRecordType,
+                new IAObject[] { new AString("210 University Drive"), new AString("Philadelphia"), new AString("PA"),
+                        new AInt16((short) 10086), new AInterval(100, 300, (byte) 0) });
+        ARecord addr21 = new ARecord(addrRecordType,
+                new IAObject[] { new AString("1 College Street"), new AString("Seattle"), new AString("WA"),
+                        new AInt16((short) 20012), new AInterval(400, 500, (byte) 0) });
+        ARecord addr22 = new ARecord(addrRecordType,
+                new IAObject[] { new AString("20 Lindsay Avenue"), new AString("Columbus"), new AString("OH"),
+                        new AInt16((short) 30120), new AInterval(600, 900, (byte) 0) });
+        ARecord addr31 = new ARecord(addrRecordType,
+                new IAObject[] { new AString("200 14th Avenue"), new AString("Long Island"), new AString("NY"),
+                        new AInt16((short) 95011), new AInterval(12000, 14000, (byte) 0) });
+        // With nested open field addr31.
+        ARecord addr32 = new ARecord(addrRecordType,
+                new IAObject[] { new AString("51 8th Street"), new AString("Orlando"), new AString("FL"),
+                        new AInt16((short) 49045), new AInterval(190000, 200000, (byte) 0) });
+
+        ARecord record1 = new ARecord(employeeType, new IAObject[] { new AInt64(0L), new AString("Tom"),
+                new AOrderedList(addrListType, Arrays.asList(new IAObject[] { addr11, addr12 })) });
+        ARecord record2 = new ARecord(employeeType, new IAObject[] { new AInt64(1L), new AString("John"),
+                new AOrderedList(addrListType, Arrays.asList(new IAObject[] { addr21, addr22 })) });
+        ARecord record3 = new ARecord(employeeType, new IAObject[] { new AInt64(2L), new AString("Lindsay"),
+                new AOrderedList(addrListType, Arrays.asList(new IAObject[] { addr31, addr32 })) });
+        // With nested open field addr41.
+        ARecord record4 = new ARecord(employeeType, new IAObject[] { new AInt64(3L), new AString("Joshua"),
+                new AOrderedList(addrListType, Arrays.asList(new IAObject[] {})) });
+        ARecord[] records = new ARecord[] { record1, record2, record3, record4 };
+        return records;
+    }
+
+    @SuppressWarnings("rawtypes")
+    public static void concurrentSerDeTestRun(ISerializerDeserializer serde, IAObject[] records) {
+        Thread[] threads = new Thread[records.length];
+        AtomicInteger errorCount = new AtomicInteger(0);
+        for (int i = 0; i < threads.length; ++i) {
+            final int index = i;
+            threads[i] = new Thread(new Runnable() {
+
+                @SuppressWarnings("unchecked")
+                @Override
+                public void run() {
+                    try {
+                        int round = 0;
+                        while (round++ < 100000) {
+                            // serialize
+                            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+                            DataOutput dos = new DataOutputStream(bos);
+                            serde.serialize(records[index], dos);
+                            bos.close();
+
+                            // deserialize
+                            ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
+                            DataInput dis = new DataInputStream(bis);
+                            IAObject object = (IAObject) serde.deserialize(dis);
+                            bis.close();
+
+                            // asserts the equivalence of objects before and after serde.
+                            Assert.assertTrue(object.deepEqual(records[index]));
+                            Assert.assertTrue(records[index].deepEqual(object));
+                        }
+                    } catch (Exception e) {
+                        errorCount.incrementAndGet();
+                        e.printStackTrace();
+                    }
+                }
+            });
+            // Kicks off test threads.
+            threads[i].start();
+        }
+
+        // Joins all the threads.
+        try {
+            for (int i = 0; i < threads.length; ++i) {
+                threads[i].join();
+            }
+        } catch (InterruptedException e) {
+            throw new IllegalStateException(e);
+        }
+        // Asserts no failure.
+        Assert.assertTrue(errorCount.get() == 0);
+    }
+}
diff --git a/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
new file mode 100644
index 0000000..eabd747
--- /dev/null
+++ b/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.serde;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.util.Set;
+
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.junit.Test;
+import org.reflections.Reflections;
+
+public class SimpleSerializerDeserializerTest {
+
+    @SuppressWarnings("rawtypes")
+    @Test
+    public void test() {
+        Reflections reflections = new Reflections("org.apache.asterix.dataflow.data.nontagged.serde");
+        Set<Class<? extends ISerializerDeserializer>> allClasses = reflections
+                .getSubTypesOf(ISerializerDeserializer.class);
+
+        for (Class<? extends ISerializerDeserializer> cl : allClasses) {
+            String className = cl.getName();
+            if (className.endsWith("ARecordSerializerDeserializer")
+                    || className.endsWith("AUnorderedListSerializerDeserializer")
+                    || className.endsWith("AOrderedListSerializerDeserializer")) {
+                // Serializer/Deserializer for complex types can have (immutable) states.
+                continue;
+            }
+
+            // Verifies the class does not have non-static fields.
+            for (Field field : cl.getDeclaredFields()) {
+                if (!java.lang.reflect.Modifier.isStatic(field.getModifiers())) {
+                    throw new IllegalStateException(
+                            "The serializer/deserializer " + cl.getName() + " is not stateless!");
+                }
+            }
+
+            // Verifies the class follows the singleton pattern.
+            for (Constructor constructor : cl.getDeclaredConstructors()) {
+                if (!java.lang.reflect.Modifier.isPrivate(constructor.getModifiers())) {
+                    throw new IllegalStateException("The serializer/deserializer " + cl.getName()
+                            + " is not implemented as a singleton class!");
+                }
+            }
+        }
+    }
+}
diff --git a/asterix-runtime/pom.xml b/asterix-runtime/pom.xml
index a5dedb0..2713dd9 100644
--- a/asterix-runtime/pom.xml
+++ b/asterix-runtime/pom.xml
@@ -176,6 +176,12 @@
 		         <groupId>org.apache.hyracks</groupId>
 		         <artifactId>hyracks-api</artifactId>
 	        </dependency>
+	    <dependency>
+            <groupId>com.e-movimento.tinytools</groupId>
+            <artifactId>privilegedaccessor</artifactId>
+            <version>1.2.2</version>
+            <scope>test</scope>
+        </dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
index 8c8723b..b61f3b2 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
@@ -23,7 +23,6 @@
 import java.io.IOException;
 
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
@@ -51,7 +50,6 @@
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ICopySerializableAggregateFunction;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -103,10 +101,13 @@
         }
     }
 
+    @Override
     public abstract void step(IFrameTupleReference tuple, byte[] state, int start, int len) throws AlgebricksException;
 
+    @Override
     public abstract void finish(byte[] state, int start, int len, DataOutput result) throws AlgebricksException;
 
+    @Override
     public abstract void finishPartial(byte[] state, int start, int len, DataOutput result) throws AlgebricksException;
 
     protected abstract void processNull(byte[] state, int start);
@@ -128,8 +129,8 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new AlgebricksException("Unexpected type " + typeTag + " in aggregation input stream. Expected type "
-                    + aggType + ".");
+            throw new AlgebricksException(
+                    "Unexpected type " + typeTag + " in aggregation input stream. Expected type " + aggType + ".");
         } else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
             aggType = typeTag;
         }
@@ -175,20 +176,16 @@
         state[start + AGG_TYPE_OFFSET] = aggType.serialize();
     }
 
-    protected void finishPartialResults(byte[] state, int start, int len, DataOutput result) throws AlgebricksException {
+    protected void finishPartialResults(byte[] state, int start, int len, DataOutput result)
+            throws AlgebricksException {
         double sum = BufferSerDeUtil.getDouble(state, start + SUM_OFFSET);
         long count = BufferSerDeUtil.getLong(state, start + COUNT_OFFSET);
         ATypeTag aggType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(state[start + AGG_TYPE_OFFSET]);
         if (recordEval == null) {
-            ARecordType recType;
-            try {
-                recType = new ARecordType(null, new String[] { "sum", "count" }, new IAType[] { BuiltinType.ADOUBLE,
-                        BuiltinType.AINT64 }, false);
-            } catch (AsterixException | HyracksDataException e) {
-                throw new AlgebricksException(e);
-            }
-            recordEval = new ClosedRecordConstructorEval(recType, new ICopyEvaluator[] { evalSum, evalCount },
-                    avgBytes, result);
+            ARecordType recType = new ARecordType(null, new String[] { "sum", "count" },
+                    new IAType[] { BuiltinType.ADOUBLE, BuiltinType.AINT64 }, false);
+            recordEval = new ClosedRecordConstructorEval(recType, new ICopyEvaluator[] { evalSum, evalCount }, avgBytes,
+                    result);
         }
 
         try {
@@ -242,8 +239,8 @@
                 int offset1 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, SUM_FIELD_ID, nullBitmapSize,
                         false);
                 sum += ADoubleSerializerDeserializer.getDouble(serBytes, offset1);
-                int offset2 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, COUNT_FIELD_ID,
-                        nullBitmapSize, false);
+                int offset2 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, COUNT_FIELD_ID, nullBitmapSize,
+                        false);
                 count += AInt64SerializerDeserializer.getLong(serBytes, offset2);
 
                 BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
index 6100fb4..a4f9968 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
@@ -23,7 +23,6 @@
 import java.io.IOException;
 
 import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
@@ -51,7 +50,6 @@
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
@@ -96,15 +94,8 @@
         eval = args[0].createEvaluator(inputVal);
         out = output.getDataOutput();
 
-        ARecordType tmpRecType;
-        try {
-            tmpRecType = new ARecordType(null, new String[] { "sum", "count" }, new IAType[] { BuiltinType.ADOUBLE,
-                    BuiltinType.AINT64 }, false);
-        } catch (AsterixException | HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
-
-        recType = tmpRecType;
+        recType = new ARecordType(null, new String[] { "sum", "count" },
+                new IAType[] { BuiltinType.ADOUBLE, BuiltinType.AINT64 }, false);
         recordEval = new ClosedRecordConstructorEval(recType, new ICopyEvaluator[] { evalSum, evalCount }, avgBytes,
                 out);
     }
@@ -116,10 +107,13 @@
         count = 0;
     }
 
+    @Override
     public abstract void step(IFrameTupleReference tuple) throws AlgebricksException;
 
+    @Override
     public abstract void finish() throws AlgebricksException;
 
+    @Override
     public abstract void finishPartial() throws AlgebricksException;
 
     protected abstract void processNull();
@@ -137,8 +131,8 @@
         } else if (aggType == ATypeTag.SYSTEM_NULL) {
             aggType = typeTag;
         } else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
-            throw new AlgebricksException("Unexpected type " + typeTag + " in aggregation input stream. Expected type "
-                    + aggType + ".");
+            throw new AlgebricksException(
+                    "Unexpected type " + typeTag + " in aggregation input stream. Expected type " + aggType + ".");
         } else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
             aggType = typeTag;
         }
@@ -229,8 +223,8 @@
                 int offset1 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, SUM_FIELD_ID, nullBitmapSize,
                         false);
                 sum += ADoubleSerializerDeserializer.getDouble(serBytes, offset1);
-                int offset2 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, COUNT_FIELD_ID,
-                        nullBitmapSize, false);
+                int offset2 = ARecordSerializerDeserializer.getFieldOffsetById(serBytes, COUNT_FIELD_ID, nullBitmapSize,
+                        false);
                 count += AInt64SerializerDeserializer.getLong(serBytes, offset2);
                 break;
             }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
index e7e90ad..e3c793a 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
@@ -42,6 +42,7 @@
 
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new IsNullDescriptor();
         }
@@ -60,7 +61,7 @@
                     private DataOutput out = output.getDataOutput();
                     private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
                     private ICopyEvaluator eval = args[0].createEvaluator(argOut);
-                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+                    private final AObjectSerializerDeserializer aObjSerDer = AObjectSerializerDeserializer.INSTANCE;
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
index 47ae0d6..40fac14 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
@@ -42,6 +42,7 @@
 
     private final static byte SER_SYSTEM_NULL_TYPE_TAG = ATypeTag.SYSTEM_NULL.serialize();
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new IsSystemNullDescriptor();
         }
@@ -60,13 +61,14 @@
                     private DataOutput out = output.getDataOutput();
                     private ArrayBackedValueStorage argOut = new ArrayBackedValueStorage();
                     private ICopyEvaluator eval = args[0].createEvaluator(argOut);
-                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+                    private final AObjectSerializerDeserializer aObjSerDer = AObjectSerializerDeserializer.INSTANCE;
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
                         argOut.reset();
                         eval.evaluate(tuple);
-                        boolean isSystemNull = argOut.getByteArray()[argOut.getStartOffset()] == SER_SYSTEM_NULL_TYPE_TAG;
+                        boolean isSystemNull = argOut.getByteArray()[argOut
+                                .getStartOffset()] == SER_SYSTEM_NULL_TYPE_TAG;
                         ABoolean res = isSystemNull ? ABoolean.TRUE : ABoolean.FALSE;
                         try {
                             aObjSerDer.serialize(res, out);
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index aebf186..bec1f79 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import org.apache.asterix.fuzzyjoin.IntArray;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -39,7 +40,6 @@
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.common.DoubleArray;
 import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
-import org.apache.asterix.fuzzyjoin.IntArray;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -56,6 +56,7 @@
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SpatialIntersectDescriptor();
         }
@@ -84,12 +85,12 @@
                     private final DoubleArray trianglesY0 = new DoubleArray();
                     private final DoubleArray trianglesX1 = new DoubleArray();
                     private final DoubleArray trianglesY1 = new DoubleArray();
-                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
+                    private final AObjectSerializerDeserializer aBooleanSerDer = AObjectSerializerDeserializer.INSTANCE;
 
                     private boolean pointOnLine(double pX, double pY, double startX, double startY, double endX,
                             double endY) throws HyracksDataException {
-                        double crossProduct = SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY, endX
-                                - startX);
+                        double crossProduct = SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY,
+                                endX - startX);
                         if (Math.abs(crossProduct) > SpatialUtils.doubleEpsilon()) { // crossProduct != 0
                             return false;
                         }
@@ -257,7 +258,8 @@
                         return false;
                     }
 
-                    private boolean lineRectangleIntersection(byte[] bytes0, byte[] bytes1) throws HyracksDataException {
+                    private boolean lineRectangleIntersection(byte[] bytes0, byte[] bytes1)
+                            throws HyracksDataException {
                         double startX1 = ADoubleSerializerDeserializer.getDouble(bytes0,
                                 ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
                         double startY1 = ADoubleSerializerDeserializer.getDouble(bytes0,
@@ -434,7 +436,7 @@
 
                     private boolean triangleTriangleIntersection(DoubleArray trianglesX0, DoubleArray trianglesY0,
                             int triangleId0, DoubleArray trianglesX1, DoubleArray trianglesY1, int triangleId1)
-                            throws HyracksDataException { // separating axis theorem
+                                    throws HyracksDataException { // separating axis theorem
 
                         for (int side = 0; side < 3; side++) {
                             spatialUtils.findNormals(trianglesX0, trianglesY0, triangleId0, side);
@@ -763,15 +765,21 @@
                                 case POINT:
                                     switch (tag1) {
                                         case POINT:
-                                            if (ascDoubleComp.compare(outInput0.getByteArray(),
-                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.X), 8,
-                                                    outInput1.getByteArray(),
-                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.X), 8) == 0) {
-                                                if (ascDoubleComp.compare(outInput0.getByteArray(),
-                                                        APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y),
-                                                        8, outInput1.getByteArray(),
-                                                        APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y),
-                                                        8) == 0) {
+                                            if (ascDoubleComp
+                                                    .compare(outInput0.getByteArray(),
+                                                            APointSerializerDeserializer.getCoordinateOffset(
+                                                                    Coordinate.X),
+                                                            8, outInput1.getByteArray(), APointSerializerDeserializer
+                                                                    .getCoordinateOffset(Coordinate.X),
+                                                            8) == 0) {
+                                                if (ascDoubleComp
+                                                        .compare(outInput0.getByteArray(),
+                                                                APointSerializerDeserializer.getCoordinateOffset(
+                                                                        Coordinate.Y),
+                                                                8, outInput1.getByteArray(),
+                                                                APointSerializerDeserializer
+                                                                        .getCoordinateOffset(Coordinate.Y),
+                                                                8) == 0) {
                                                     res = true;
                                                 }
                                             }
@@ -784,18 +792,18 @@
                                                     outInput0.getByteArray(),
                                                     APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
 
-                                            double startX = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double startX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
 
                                             res = pointOnLine(pX, pY, startX, startY, endX, endY);
                                             break;
@@ -812,11 +820,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case LINE:
@@ -829,49 +835,49 @@
                                                     outInput1.getByteArray(),
                                                     APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
 
-                                            double startX = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double startX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
 
                                             res = pointOnLine(pX, pY, startX, startY, endX, endY);
                                             break;
                                         case LINE:
-                                            double startX1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY1 = ADoubleSerializerDeserializer.getDouble(outInput0
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+                                            double startX1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY1 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput0.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
 
-                                            double startX2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.X));
-                                            double startY2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getStartPointCoordinateOffset(Coordinate.Y));
-                                            double endX2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.X));
-                                            double endY2 = ADoubleSerializerDeserializer.getDouble(outInput1
-                                                    .getByteArray(), ALineSerializerDeserializer
-                                                    .getEndPointCoordinateOffset(Coordinate.Y));
-                                            res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2,
-                                                    startY2, endX2, endY2);
+                                            double startX2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY2 = ADoubleSerializerDeserializer
+                                                    .getDouble(outInput1.getByteArray(), ALineSerializerDeserializer
+                                                            .getEndPointCoordinateOffset(Coordinate.Y));
+                                            res = lineLineIntersection(startX1, startY1, endX1, endY1, startX2, startY2,
+                                                    endX2, endY2);
                                             break;
                                         case POLYGON:
                                             res = linePolygonIntersection(outInput0.getByteArray(),
@@ -889,11 +895,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case POLYGON:
@@ -991,11 +995,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case CIRCLE:
@@ -1023,11 +1025,9 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case RECTANGLE:
@@ -1078,25 +1078,22 @@
                                             res = false;
                                             break;
                                         default:
-                                            throw new NotImplementedException(
-                                                    AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                            + ": does not support the type: "
-                                                            + tag1
-                                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                            throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                                    .getName() + ": does not support the type: " + tag1
+                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                                     }
                                     break;
                                 case NULL:
                                     res = false;
                                     break;
                                 default:
-                                    throw new NotImplementedException(
-                                            AsterixBuiltinFunctions.SPATIAL_INTERSECT.getName()
-                                                    + ": does not support the type: " + tag1
-                                                    + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
+                                    throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_INTERSECT
+                                            .getName() + ": does not support the type: " + tag1
+                                            + "; it is only implemented for POINT, ALINE, POLYGON, and CIRCLE.");
                             }
 
                             ABoolean aResult = res ? (ABoolean.TRUE) : (ABoolean.FALSE);
-                            aObjSerDer.serialize(aResult, out);
+                            aBooleanSerDer.serialize(aResult, out);
                         } catch (HyracksDataException hde) {
                             throw new AlgebricksException(hde);
                         }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index cdecce5..509f104 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -25,7 +25,6 @@
 
 import org.apache.asterix.builders.RecordBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
@@ -50,6 +49,7 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.string.UTF8StringReader;
 
 //The record merge evaluator is used to combine two records with no matching fieldnames
 //If both records have the same fieldname for a non-record field anywhere in the schema, the merge will fail
@@ -88,20 +88,10 @@
 
             private static final long serialVersionUID = 1L;
 
-            @SuppressWarnings("unchecked")
-            private final ISerializerDeserializer<ANull> nullSerDe = AqlSerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(BuiltinType.ANULL);
-            private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
-
             @Override
             public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
-                final ARecordType recType;
-                try {
-                    recType = new ARecordType(outRecType.getTypeName(), outRecType.getFieldNames(),
-                            outRecType.getFieldTypes(), outRecType.isOpen());
-                } catch (AsterixException | HyracksDataException e) {
-                    throw new IllegalStateException();
-                }
+                final ARecordType recType = new ARecordType(outRecType.getTypeName(), outRecType.getFieldNames(),
+                        outRecType.getFieldTypes(), outRecType.isOpen());
 
                 final PointableAllocator pa = new PointableAllocator();
                 final IVisitablePointable vp0 = pa.allocateRecordValue(inRecType0);
@@ -114,6 +104,11 @@
                 final ICopyEvaluator eval1 = args[1].createEvaluator(abvs1);
 
                 return new ICopyEvaluator() {
+                    @SuppressWarnings("unchecked")
+                    final ISerializerDeserializer<ANull> nullSerDe = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+                    final UTF8StringReader reader = new UTF8StringReader();
+
                     private final Stack<RecordBuilder> rbStack = new Stack<RecordBuilder>();
                     private final ArrayBackedValueStorage tabvs = new ArrayBackedValueStorage();
                     private final ByteArrayAccessibleOutputStream nameOutputStream = new ByteArrayAccessibleOutputStream();
@@ -220,7 +215,7 @@
                         nameOutputStream.write(fieldNamePointable.getByteArray(),
                                 fieldNamePointable.getStartOffset() + 1, fieldNamePointable.getLength());
                         namedis.reset();
-                        String fieldName = aStringSerDer.deserialize(namedis).getStringValue();
+                        String fieldName = reader.readUTF(namedis);
 
                         //Add the merged field
                         if (combinedType.isClosedField(fieldName)) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
index 6e51311..6e4c175 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
@@ -32,7 +32,6 @@
 import org.apache.asterix.builders.RecordBuilderFactory;
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
@@ -302,8 +301,7 @@
                 if (checkType(ATypeTag.INTERVAL, objectType)) {
                     if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                         if (admLexer.next() == AdmLexer.TOKEN_STRING_LITERAL) {
-                            AIntervalSerializerDeserializer.parseDate(admLexer.getLastTokenImage(), out);
-
+                            parseDateInterval(admLexer.getLastTokenImage(), out);
                             if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_CLOSE) {
                                 break;
                             }
@@ -316,8 +314,7 @@
                 if (checkType(ATypeTag.INTERVAL, objectType)) {
                     if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                         if (admLexer.next() == AdmLexer.TOKEN_STRING_LITERAL) {
-                            AIntervalSerializerDeserializer.parseTime(admLexer.getLastTokenImage(), out);
-
+                            parseTimeInterval(admLexer.getLastTokenImage(), out);
                             if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_CLOSE) {
                                 break;
                             }
@@ -330,8 +327,7 @@
                 if (checkType(ATypeTag.INTERVAL, objectType)) {
                     if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
                         if (admLexer.next() == AdmLexer.TOKEN_STRING_LITERAL) {
-                            AIntervalSerializerDeserializer.parseDatetime(admLexer.getLastTokenImage(), out);
-
+                            parseDateTimeInterval(admLexer.getLastTokenImage(), out);
                             if (admLexer.next() == AdmLexer.TOKEN_CONSTRUCTOR_CLOSE) {
                                 break;
                             }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
index 4ce8ee8..794097f 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AInterval;
 import org.apache.asterix.om.base.ALine;
 import org.apache.asterix.om.base.AMutableBinary;
 import org.apache.asterix.om.base.AMutableCircle;
@@ -47,6 +48,7 @@
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.base.AMutableInt64;
 import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.om.base.AMutableInterval;
 import org.apache.asterix.om.base.AMutableLine;
 import org.apache.asterix.om.base.AMutablePoint;
 import org.apache.asterix.om.base.AMutablePoint3D;
@@ -68,6 +70,7 @@
 import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import org.apache.asterix.om.base.temporal.ATimeParserFactory;
 import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -104,6 +107,7 @@
     protected AMutablePoint aPoint2 = new AMutablePoint(0, 0);
     protected AMutableLine aLine = new AMutableLine(null, null);
     protected AMutableDate aDate = new AMutableDate(0);
+    protected final AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
 
     // Serializers
     @SuppressWarnings("unchecked")
@@ -182,6 +186,9 @@
     @SuppressWarnings("unchecked")
     protected final static ISerializerDeserializer<ALine> lineSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ALINE);
+    @SuppressWarnings("unchecked")
+    private static final ISerializerDeserializer<AInterval> intervalSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINTERVAL);
 
     protected String filename;
 
@@ -352,4 +359,163 @@
         aBinary.setValue(base64Parser.getByteArray(), 0, base64Parser.getLength());
         binarySerde.serialize(aBinary, out);
     }
+
+    protected void parseDateTimeInterval(String interval, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMsStart = 0;
+        long chrononTimeInMsEnd = 0;
+        try {
+            // the starting point for parsing (so for the accessor)
+            int startOffset = 0;
+            int endOffset, timeSeperatorOffsetInDatetimeString;
+
+            // Get the index for the comma
+            int commaIndex = interval.indexOf(',');
+            if (commaIndex < 1) {
+                throw new AlgebricksException("comma is missing for a string of interval");
+            }
+
+            endOffset = commaIndex - 1;
+            timeSeperatorOffsetInDatetimeString = interval.indexOf('T');
+
+            if (timeSeperatorOffsetInDatetimeString < 0) {
+                throw new AlgebricksException(
+                        "This can not be an instance of interval: missing T for a datetime value.");
+            }
+
+            chrononTimeInMsStart = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
+
+            chrononTimeInMsStart += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
+
+            // Interval End
+            startOffset = commaIndex + 1;
+            endOffset = interval.length() - 1;
+
+            timeSeperatorOffsetInDatetimeString = interval.indexOf('T', startOffset);
+
+            if (timeSeperatorOffsetInDatetimeString < 0) {
+                throw new AlgebricksException(
+                        "This can not be an instance of interval: missing T for a datetime value.");
+            }
+
+            chrononTimeInMsEnd = parseDatePart(interval, startOffset, timeSeperatorOffsetInDatetimeString - 1);
+
+            chrononTimeInMsEnd += parseTimePart(interval, timeSeperatorOffsetInDatetimeString + 1, endOffset);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+
+        try {
+            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.DATETIME.serialize());
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e);
+        }
+
+        intervalSerde.serialize(aInterval, out);
+    }
+
+    protected void parseTimeInterval(String interval, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMsStart = 0;
+        long chrononTimeInMsEnd = 0;
+        try {
+            int startOffset = 0;
+            int endOffset;
+
+            // Get the index for the comma
+            int commaIndex = interval.indexOf(',');
+            if (commaIndex < 0) {
+                throw new AlgebricksException("comma is missing for a string of interval");
+            }
+
+            endOffset = commaIndex - 1;
+            // Interval Start
+            chrononTimeInMsStart = parseTimePart(interval, startOffset, endOffset);
+
+            if (chrononTimeInMsStart < 0) {
+                chrononTimeInMsStart += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+
+            // Interval End
+            startOffset = commaIndex + 1;
+            endOffset = interval.length() - 1;
+
+            chrononTimeInMsEnd = parseTimePart(interval, startOffset, endOffset);
+            if (chrononTimeInMsEnd < 0) {
+                chrononTimeInMsEnd += GregorianCalendarSystem.CHRONON_OF_DAY;
+            }
+
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+
+        try {
+            aInterval.setValue(chrononTimeInMsStart, chrononTimeInMsEnd, ATypeTag.TIME.serialize());
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e);
+        }
+        intervalSerde.serialize(aInterval, out);
+    }
+
+    protected void parseDateInterval(String interval, DataOutput out) throws HyracksDataException {
+        long chrononTimeInMsStart = 0;
+        long chrononTimeInMsEnd = 0;
+        try {
+            // the starting point for parsing (so for the accessor)
+            int startOffset = 0;
+            int endOffset;
+
+            // Get the index for the comma
+            int commaIndex = interval.indexOf(',');
+            if (commaIndex < 1) {
+                throw new AlgebricksException("comma is missing for a string of interval");
+            }
+
+            endOffset = commaIndex - 1;
+            chrononTimeInMsStart = parseDatePart(interval, startOffset, endOffset);
+
+            // Interval End
+            startOffset = commaIndex + 1;
+            endOffset = interval.length() - 1;
+
+            chrononTimeInMsEnd = parseDatePart(interval, startOffset, endOffset);
+
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+
+        try {
+            aInterval.setValue((chrononTimeInMsStart / GregorianCalendarSystem.CHRONON_OF_DAY),
+                    (chrononTimeInMsEnd / GregorianCalendarSystem.CHRONON_OF_DAY), ATypeTag.DATE.serialize());
+        } catch (AlgebricksException e) {
+            throw new HyracksDataException(e);
+        }
+        intervalSerde.serialize(aInterval, out);
+    }
+
+    private long parseDatePart(String interval, int startOffset, int endOffset)
+            throws AlgebricksException, HyracksDataException {
+
+        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
+            endOffset--;
+        }
+
+        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
+            startOffset++;
+        }
+
+        return ADateParserFactory.parseDatePart(interval, startOffset, endOffset - startOffset + 1);
+    }
+
+    private int parseTimePart(String interval, int startOffset, int endOffset)
+            throws AlgebricksException, HyracksDataException {
+
+        while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
+            endOffset--;
+        }
+
+        while (interval.charAt(startOffset) == '"' || interval.charAt(startOffset) == ' ') {
+            startOffset++;
+        }
+
+        return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
+    }
 }
diff --git a/asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java b/asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
new file mode 100644
index 0000000..e23e255
--- /dev/null
+++ b/asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operator.file;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.asterix.om.base.AMutableInterval;
+import org.apache.asterix.runtime.operators.file.ADMDataParser;
+import org.junit.Assert;
+import org.junit.Test;
+
+import junit.extensions.PA;
+
+public class ADMDataParserTest {
+
+    @Test
+    public void test() {
+        String[] dateIntervals = { "-9537-08-04, 9656-06-03", "-9537-04-04, 9656-06-04", "-9537-10-04, 9626-09-05" };
+        AMutableInterval[] parsedDateIntervals = new AMutableInterval[] {
+                new AMutableInterval(-4202630, 2807408, (byte) 17), new AMutableInterval(-4202752, 2807409, (byte) 17),
+                new AMutableInterval(-4202569, 2796544, (byte) 17), };
+
+        String[] timeIntervals = { "12:04:45.689Z, 12:41:59.002Z", "12:10:45.169Z, 15:37:48.736Z",
+                "04:16:42.321Z, 12:22:56.816Z" };
+        AMutableInterval[] parsedTimeIntervals = new AMutableInterval[] {
+                new AMutableInterval(43485689, 45719002, (byte) 18),
+                new AMutableInterval(43845169, 56268736, (byte) 18),
+                new AMutableInterval(15402321, 44576816, (byte) 18), };
+
+        String[] dateTimeIntervals = { "-2640-10-11T17:32:15.675Z, 4104-02-01T05:59:11.902Z",
+                "0534-12-08T08:20:31.487Z, 6778-02-16T22:40:21.653Z",
+                "2129-12-12T13:18:35.758Z, 8647-07-01T13:10:19.691Z" };
+        AMutableInterval[] parsedDateTimeIntervals = new AMutableInterval[] {
+                new AMutableInterval(-145452954464325L, 67345192751902L, (byte) 16),
+                new AMutableInterval(-45286270768513L, 151729886421653L, (byte) 16),
+                new AMutableInterval(5047449515758L, 210721439419691L, (byte) 16) };
+
+        Thread[] threads = new Thread[16];
+        AtomicInteger errorCount = new AtomicInteger(0);
+        for (int i = 0; i < threads.length; ++i) {
+            threads[i] = new Thread(new Runnable() {
+                ADMDataParser parser = new ADMDataParser();
+                ByteArrayOutputStream bos = new ByteArrayOutputStream();
+                DataOutput dos = new DataOutputStream(bos);
+
+                @Override
+                public void run() {
+                    try {
+                        int round = 0;
+                        while (round++ < 10000) {
+                            // Test parseDateInterval.
+                            for (int index = 0; index < dateIntervals.length; ++index) {
+                                PA.invokeMethod(parser, "parseDateInterval(java.lang.String, java.io.DataOutput)",
+                                        dateIntervals[index], dos);
+                                AMutableInterval aInterval = (AMutableInterval) PA.getValue(parser, "aInterval");
+                                Assert.assertTrue(aInterval.equals(parsedDateIntervals[index]));
+                            }
+
+                            // Tests parseTimeInterval.
+                            for (int index = 0; index < timeIntervals.length; ++index) {
+                                PA.invokeMethod(parser, "parseTimeInterval(java.lang.String, java.io.DataOutput)",
+                                        timeIntervals[index], dos);
+                                AMutableInterval aInterval = (AMutableInterval) PA.getValue(parser, "aInterval");
+                                Assert.assertTrue(aInterval.equals(parsedTimeIntervals[index]));
+                            }
+
+                            // Tests parseDateTimeInterval.
+                            for (int index = 0; index < dateTimeIntervals.length; ++index) {
+                                PA.invokeMethod(parser, "parseDateTimeInterval(java.lang.String, java.io.DataOutput)",
+                                        dateTimeIntervals[index], dos);
+                                AMutableInterval aInterval = (AMutableInterval) PA.getValue(parser, "aInterval");
+                                Assert.assertTrue(aInterval.equals(parsedDateTimeIntervals[index]));
+                            }
+                        }
+                    } catch (Exception e) {
+                        errorCount.incrementAndGet();
+                        e.printStackTrace();
+                    }
+                }
+            });
+            // Kicks off test threads.
+            threads[i].start();
+        }
+
+        // Joins all the threads.
+        try {
+            for (int i = 0; i < threads.length; ++i) {
+                threads[i].join();
+            }
+        } catch (InterruptedException e) {
+            throw new IllegalStateException(e);
+        }
+        // Asserts no failure.
+        Assert.assertTrue(errorCount.get() == 0);
+    }
+
+}

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 9
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#3).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
63 files changed, 988 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 1: -Verified

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/461/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jianfeng Jia (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jianfeng Jia has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 6: Code-Review+1

(1 comment)

LGTM .

So for the serialization-heavy work, it should use the UTF8StringReader/Writer instead of the AStringSerDer, right? 
We'd better add some guide comments on AStringSerDer.

https://asterix-gerrit.ics.uci.edu/#/c/522/6/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
File asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java:

Line 48:             for (Field field : cl.getDeclaredFields()) {
This is a nice test!


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/465/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 7: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/470/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 4: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/464/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 7:

@Jianfeng, comments added.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 6: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/466/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jianfeng Jia, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#7).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/pom.xml
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
A asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
65 files changed, 1,116 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/7
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jianfeng Jia, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#8).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/pom.xml
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
A asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
65 files changed, 1,116 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/8
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 1: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/557/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 5: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/465/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 3: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/463/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/462/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/471/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/461/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#6).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/pom.xml
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
A asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
65 files changed, 1,111 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/466/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/522

to look at the new patch set (#5).

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................

ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.

Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectUtil.java
M asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjects.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapter.java
M asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
M asterix-om/pom.xml
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ACircleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADateTimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt16SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AInt8SerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ALineSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APoint3DSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APointSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARectangleSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ATimeSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUUIDSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AYearMonthDurationSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ACollectionCursor.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
M asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/base/DefaultOpenFieldType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedLocalAvgTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordConstructorResultType.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
M asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializerTest.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerTestUtils.java
A asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/serde/SimpleSerializerDeserializerTest.java
M asterix-runtime/pom.xml
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/IsSystemNullDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/ADMDataParser.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/file/AbstractDataParser.java
A asterix-runtime/src/test/java/org/apache/asterix/runtime/operator/file/ADMDataParserTest.java
65 files changed, 1,111 insertions(+), 1,170 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/22/522/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 2: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/462/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1198: make ISerializerDeserializer implementations...

Posted by "Jianfeng Jia (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jianfeng Jia has posted comments on this change.

Change subject: ASTERIXDB-1198: make ISerializerDeserializer implementations in AsterixDB stateless.
......................................................................


Patch Set 8: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/522
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I808b6583db6b48a0e9000dcf0ac88c03c336c3a5
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No